From b28b492c47ce5a1357145118b9cbfeebe277db19 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 12 Sep 2018 13:51:37 +0800 Subject: [PATCH 001/470] Initial commit --- README.md | 2 ++ 1 file changed, 2 insertions(+) create mode 100644 README.md diff --git a/README.md b/README.md new file mode 100644 index 000000000..07e3802de --- /dev/null +++ b/README.md @@ -0,0 +1,2 @@ +# flinkStreamSQL +基于开源的flink,对其实时sql进行扩展 From 460bd3fa5621649091fdd89bf40abc241fdeb913 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 12 Sep 2018 14:01:37 +0800 Subject: [PATCH 002/470] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 07e3802de..490262eb7 100644 --- a/README.md +++ b/README.md @@ -1,2 +1,2 @@ # flinkStreamSQL -基于开源的flink,对其实时sql进行扩展 +基于开源的flink,对其实时sql进行扩展;主要实现了流与维表的join。 From 3dd8dd19436fd2bea2a94491380f89037be61cca Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 12 Sep 2018 17:51:46 +0800 Subject: [PATCH 003/470] dtstack upload --- .gitignore | 11 + core/pom.xml | 119 +++ .../main/java/com/dtstack/flink/sql/Main.java | 313 +++++++ .../flink/sql/MyLocalStreamEnvironment.java | 123 +++ .../flink/sql/classloader/DtClassLoader.java | 222 +++++ .../dtstack/flink/sql/enums/ColumnType.java | 67 ++ .../flink/sql/enums/ECacheContentType.java | 46 ++ .../dtstack/flink/sql/enums/ECacheType.java | 41 + .../dtstack/flink/sql/enums/ETableType.java | 44 + .../flink/sql/parser/CreateFuncParser.java | 98 +++ .../flink/sql/parser/CreateTableParser.java | 114 +++ .../com/dtstack/flink/sql/parser/IParser.java | 45 + .../flink/sql/parser/InsertSqlParser.java | 154 ++++ .../dtstack/flink/sql/parser/SqlParser.java | 134 +++ .../com/dtstack/flink/sql/parser/SqlTree.java | 79 ++ .../com/dtstack/flink/sql/side/AliasInfo.java | 51 ++ .../dtstack/flink/sql/side/AsyncReqRow.java | 223 +++++ .../dtstack/flink/sql/side/CacheMissVal.java | 40 + .../com/dtstack/flink/sql/side/FieldInfo.java | 67 ++ .../flink/sql/side/FieldReplaceInfo.java | 63 ++ .../com/dtstack/flink/sql/side/JoinInfo.java | 193 +++++ .../com/dtstack/flink/sql/side/JoinScope.java | 105 +++ .../flink/sql/side/ParserJoinField.java | 133 +++ .../dtstack/flink/sql/side/SideSQLParser.java | 217 +++++ .../dtstack/flink/sql/side/SideSqlExec.java | 576 +++++++++++++ .../dtstack/flink/sql/side/SideTableInfo.java | 98 +++ .../flink/sql/side/StreamSideFactory.java | 52 ++ .../flink/sql/side/cache/AbsSideCache.java | 46 ++ .../flink/sql/side/cache/CacheObj.java | 62 ++ .../flink/sql/side/cache/LRUSideCache.java | 70 ++ .../flink/sql/sink/IStreamSinkGener.java | 34 + .../flink/sql/sink/StreamSinkFactory.java | 77 ++ .../flink/sql/source/IStreamSourceGener.java | 44 + .../flink/sql/source/StreamSourceFactory.java | 86 ++ .../flink/sql/table/AbsSideTableParser.java | 75 ++ .../flink/sql/table/AbsSourceParser.java | 69 ++ .../flink/sql/table/AbsTableParser.java | 122 +++ .../sql/table/ITableFieldDealHandler.java | 34 + .../flink/sql/table/SourceTableInfo.java | 104 +++ .../dtstack/flink/sql/table/TableInfo.java | 130 +++ .../sql/table/TableInfoParserFactory.java | 121 +++ .../flink/sql/table/TargetTableInfo.java | 33 + .../sql/threadFactory/DTThreadFactory.java | 59 ++ .../com/dtstack/flink/sql/util/ByteUtils.java | 174 ++++ .../com/dtstack/flink/sql/util/ClassUtil.java | 178 ++++ .../flink/sql/util/ConfigConstrant.java | 53 ++ .../com/dtstack/flink/sql/util/DateUtil.java | 773 ++++++++++++++++++ .../dtstack/flink/sql/util/DtStringUtil.java | 140 ++++ .../com/dtstack/flink/sql/util/FlinkUtil.java | 286 +++++++ .../com/dtstack/flink/sql/util/MD5Utils.java | 64 ++ .../com/dtstack/flink/sql/util/MathUtil.java | 95 +++ .../dtstack/flink/sql/util/PluginUtil.java | 124 +++ .../CustomerWaterMarkerForLong.java | 65 ++ .../CustomerWaterMarkerForTimeStamp.java | 66 ++ .../sql/watermarker/WaterMarkerAssigner.java | 84 ++ .../flink/sql/side/SideSqlExecTest.java | 192 +++++ .../flink/sql/side/TestSideSqlParser.java | 59 ++ elasticsearch/elasticsearch-sink/pom.xml | 112 +++ .../sink/elasticsearch/CustomerSinkFunc.java | 111 +++ .../sink/elasticsearch/ElasticsearchSink.java | 175 ++++ .../flink/sql/sink/elasticsearch/EsUtil.java | 65 ++ .../table/ElasticsearchSinkParser.java | 62 ++ .../table/ElasticsearchTableInfo.java | 106 +++ elasticsearch/pom.xml | 19 + hbase/hbase-side/doc/readme.txt | 57 ++ hbase/hbase-side/pom.xml | 100 +++ .../sql/side/hbase/HbaseAsyncReqRow.java | 246 ++++++ .../sql/side/hbase/Md5ReplaceOperator.java | 43 + .../flink/sql/side/hbase/ReplaceInfo.java | 62 ++ .../flink/sql/side/hbase/ReplaceOperator.java | 57 ++ .../flink/sql/side/hbase/RowKeyBuilder.java | 140 ++++ .../sql/side/hbase/enums/EReplaceOpType.java | 32 + .../sql/side/hbase/enums/EReplaceType.java | 33 + .../rowkeydealer/AbsRowKeyModeDealer.java | 110 +++ .../PreRowKeyModeDealerDealer.java | 146 ++++ .../rowkeydealer/RowKeyEqualModeDealer.java | 122 +++ .../sql/side/hbase/table/HbaseSideParser.java | 117 +++ .../side/hbase/table/HbaseSideTableInfo.java | 156 ++++ .../sql/side/hbase/utils/HbaseUtils.java | 74 ++ hbase/hbase-sink/pom.xml | 99 +++ .../sql/sink/hbase/HbaseOutputFormat.java | 209 +++++ .../flink/sql/sink/hbase/HbaseSink.java | 103 +++ .../flink/sql/sink/hbase/HbaseUtil.java | 67 ++ .../sql/sink/hbase/table/HbaseSinkParser.java | 63 ++ .../sql/sink/hbase/table/HbaseTableInfo.java | 138 ++++ hbase/pom.xml | 20 + kafka09/kafka09-source/pom.xml | 107 +++ .../kafka09/CustomerJsonDeserialization.java | 114 +++ .../sql/source/kafka09/Kafka09Source.java | 83 ++ .../kafka09/table/Kafka09SourceParser.java | 51 ++ .../kafka09/table/Kafka09SourceTableInfo.java | 120 +++ .../test/java/com/dtstack/flinkx/AppTest.java | 58 ++ kafka09/pom.xml | 21 + launcher/job/sideSql.txt | 72 ++ launcher/pom.xml | 34 + .../sql/launcher/ClusterClientFactory.java | 210 +++++ .../flink/sql/launcher/ClusterMode.java | 35 + .../flink/sql/launcher/LauncherMain.java | 64 ++ .../sql/launcher/LauncherOptionParser.java | 150 ++++ .../flink/sql/launcher/LauncherOptions.java | 48 ++ mysql/mysql-side/pom.xml | 108 +++ .../sql/side/mysql/MysqlAsyncReqRow.java | 340 ++++++++ .../sql/side/mysql/table/MysqlSideParser.java | 67 ++ .../side/mysql/table/MysqlSideTableInfo.java | 99 +++ mysql/mysql-sink/pom.xml | 111 +++ .../dtstack/flink/sql/sink/mysql/DBSink.java | 176 ++++ .../flink/sql/sink/mysql/MysqlSink.java | 98 +++ .../sink/mysql/RetractJDBCOutputFormat.java | 354 ++++++++ .../sql/sink/mysql/table/MysqlSinkParser.java | 55 ++ .../sql/sink/mysql/table/MysqlTableInfo.java | 139 ++++ .../test/java/com/dtstack/flinkx/AppTest.java | 58 ++ mysql/pom.xml | 20 + pom.xml | 38 + 113 files changed, 12621 insertions(+) create mode 100644 .gitignore create mode 100644 core/pom.xml create mode 100644 core/src/main/java/com/dtstack/flink/sql/Main.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/classloader/DtClassLoader.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/enums/ColumnType.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/enums/ECacheContentType.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/enums/ECacheType.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/enums/ETableType.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/parser/IParser.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/AliasInfo.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/CacheMissVal.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/FieldInfo.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/FieldReplaceInfo.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/JoinScope.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/cache/AbsSideCache.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/cache/CacheObj.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/cache/LRUSideCache.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/sink/IStreamSinkGener.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/source/IStreamSourceGener.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/table/ITableFieldDealHandler.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/table/TargetTableInfo.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/threadFactory/DTThreadFactory.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/util/ByteUtils.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/util/ConfigConstrant.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/util/DateUtil.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/util/MD5Utils.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/util/MathUtil.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java create mode 100644 core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java create mode 100644 core/src/test/java/com/dtstack/flink/sql/side/TestSideSqlParser.java create mode 100644 elasticsearch/elasticsearch-sink/pom.xml create mode 100644 elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java create mode 100644 elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java create mode 100644 elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/EsUtil.java create mode 100644 elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.java create mode 100644 elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java create mode 100644 elasticsearch/pom.xml create mode 100644 hbase/hbase-side/doc/readme.txt create mode 100644 hbase/hbase-side/pom.xml create mode 100644 hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java create mode 100644 hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/Md5ReplaceOperator.java create mode 100644 hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceInfo.java create mode 100644 hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceOperator.java create mode 100644 hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java create mode 100644 hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceOpType.java create mode 100644 hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceType.java create mode 100644 hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbsRowKeyModeDealer.java create mode 100644 hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java create mode 100644 hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java create mode 100644 hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java create mode 100644 hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideTableInfo.java create mode 100644 hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseUtils.java create mode 100644 hbase/hbase-sink/pom.xml create mode 100644 hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java create mode 100644 hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseSink.java create mode 100644 hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseUtil.java create mode 100644 hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java create mode 100644 hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java create mode 100644 hbase/pom.xml create mode 100644 kafka09/kafka09-source/pom.xml create mode 100644 kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/CustomerJsonDeserialization.java create mode 100644 kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/Kafka09Source.java create mode 100644 kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/table/Kafka09SourceParser.java create mode 100644 kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/table/Kafka09SourceTableInfo.java create mode 100644 kafka09/kafka09-source/src/test/java/com/dtstack/flinkx/AppTest.java create mode 100644 kafka09/pom.xml create mode 100644 launcher/job/sideSql.txt create mode 100644 launcher/pom.xml create mode 100644 launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java create mode 100644 launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterMode.java create mode 100644 launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java create mode 100644 launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java create mode 100644 launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java create mode 100644 mysql/mysql-side/pom.xml create mode 100644 mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java create mode 100644 mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java create mode 100644 mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java create mode 100644 mysql/mysql-sink/pom.xml create mode 100644 mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java create mode 100644 mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java create mode 100644 mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java create mode 100644 mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java create mode 100644 mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlTableInfo.java create mode 100644 mysql/mysql-sink/src/test/java/com/dtstack/flinkx/AppTest.java create mode 100644 mysql/pom.xml create mode 100644 pom.xml diff --git a/.gitignore b/.gitignore new file mode 100644 index 000000000..5e8b08446 --- /dev/null +++ b/.gitignore @@ -0,0 +1,11 @@ +target +.idea/ +/.idea/* +/target +target +.class +.project +.classpath +*.eclipse.* +*.iml +plugins/ diff --git a/core/pom.xml b/core/pom.xml new file mode 100644 index 000000000..6dad09524 --- /dev/null +++ b/core/pom.xml @@ -0,0 +1,119 @@ + + + flink.sql + com.dtstack.flinkx + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.core + jar + + core + http://maven.apache.org + + + UTF-8 + core + + + + + junit + junit + 4.12 + test + + + + org.apache.flink + flink-table_2.11 + ${flink.version} + + + + org.apache.flink + flink-core + ${flink.version} + + + + org.apache.flink + flink-streaming-java_2.11 + ${flink.version} + + + + org.apache.flink + flink-scala_2.11 + ${flink.version} + + + + org.apache.flink + flink-streaming-scala_2.11 + ${flink.version} + + + + + + + src/main/java/ + + + src/main/resources/ + + + + + org.apache.maven.plugins + maven-jar-plugin + + target/classes/ + + + + com.dtstack.flink.sql.Main + + false + + + . + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java new file mode 100644 index 000000000..541e87909 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -0,0 +1,313 @@ +/* + * 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 com.dtstack.flink.sql; + +import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.parser.CreateFuncParser; +import com.dtstack.flink.sql.parser.InsertSqlParser; +import com.dtstack.flink.sql.side.SideSqlExec; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.parser.SqlParser; +import com.dtstack.flink.sql.parser.SqlTree; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.sink.StreamSinkFactory; +import com.dtstack.flink.sql.source.StreamSourceFactory; +import com.dtstack.flink.sql.watermarker.WaterMarkerAssigner; +import com.dtstack.flink.sql.util.FlinkUtil; +import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.CommandLineParser; +import org.apache.commons.cli.DefaultParser; +import org.apache.commons.cli.Options; +import org.apache.commons.io.Charsets; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import org.apache.flink.calcite.shaded.com.google.common.base.Strings; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.calcite.shaded.com.google.common.collect.Sets; +import org.apache.flink.client.program.ContextEnvironment; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamContextEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.sinks.TableSink; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.net.URL; +import java.net.URLClassLoader; +import java.net.URLDecoder; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +/** + * Date: 2018/6/26 + * Company: www.dtstack.com + * @author xuchao + */ + +public class Main { + + private static final ObjectMapper objMapper = new ObjectMapper(); + + private static final Logger LOG = LoggerFactory.getLogger(Main.class); + + private static final String LOCAL_MODE = "local"; + + private static final int failureRate = 3; + + private static final int failureInterval = 6; //min + + private static final int delayInterval = 10; //sec + + public static void main(String[] args) throws Exception { + + Options options = new Options(); + options.addOption("sql", true, "sql config"); + options.addOption("name", true, "job name"); + options.addOption("addjar", true, "add jar"); + options.addOption("localSqlPluginPath", true, "local sql plugin path"); + options.addOption("remoteSqlPluginPath", true, "remote sql plugin path"); + options.addOption("confProp", true, "env properties"); + options.addOption("mode", true, "deploy mode"); + + CommandLineParser parser = new DefaultParser(); + CommandLine cl = parser.parse(options, args); + String sql = cl.getOptionValue("sql"); + String name = cl.getOptionValue("name"); + String addJarListStr = cl.getOptionValue("addjar"); + String localSqlPluginPath = cl.getOptionValue("localSqlPluginPath"); + String remoteSqlPluginPath = cl.getOptionValue("remoteSqlPluginPath"); + String deployMode = cl.getOptionValue("mode"); + String confProp = cl.getOptionValue("confProp"); + + Preconditions.checkNotNull(sql, "it requires input parameters sql"); + Preconditions.checkNotNull(name, "it requires input parameters name"); + Preconditions.checkNotNull(localSqlPluginPath, "it requires input parameters localSqlPluginPath"); + Preconditions.checkNotNull(remoteSqlPluginPath, "it requires input parameters remoteSqlPluginPath"); + + sql = URLDecoder.decode(sql, Charsets.UTF_8.name()); + SqlParser.setLocalSqlPluginRoot(localSqlPluginPath); + + List addJarFileList = Lists.newArrayList(); + if(!Strings.isNullOrEmpty(addJarListStr)){ + addJarListStr = URLDecoder.decode(addJarListStr, Charsets.UTF_8.name()); + addJarFileList = objMapper.readValue(addJarListStr, List.class); + } + + ClassLoader threadClassLoader = Thread.currentThread().getContextClassLoader(); + DtClassLoader dtClassLoader = new DtClassLoader(new URL[]{}, threadClassLoader); + Thread.currentThread().setContextClassLoader(dtClassLoader); + + URLClassLoader parentClassloader = null; + if(!LOCAL_MODE.equals(deployMode)){ + parentClassloader = (URLClassLoader) threadClassLoader.getParent(); + }else{ + parentClassloader = dtClassLoader; + } + + confProp = URLDecoder.decode(confProp, Charsets.UTF_8.toString()); + Properties confProperties = PluginUtil.jsonStrToObject(confProp, Properties.class); + StreamExecutionEnvironment env = getStreamExeEnv(confProperties, deployMode); + StreamTableEnvironment tableEnv = StreamTableEnvironment.getTableEnvironment(env); + + List jarURList = Lists.newArrayList(); + SqlTree sqlTree = SqlParser.parseSql(sql); + + //Get External jar to load + for(String addJarPath : addJarFileList){ + File tmpFile = new File(addJarPath); + jarURList.add(tmpFile.toURI().toURL()); + } + + Map sideTableMap = Maps.newHashMap(); + Map registerTableCache = Maps.newHashMap(); + + //register udf + registerUDF(sqlTree, jarURList, parentClassloader, tableEnv); + //register table schema + registerTable(sqlTree, env, tableEnv, localSqlPluginPath, remoteSqlPluginPath, sideTableMap, registerTableCache); + + SideSqlExec sideSqlExec = new SideSqlExec(); + sideSqlExec.setLocalSqlPluginPath(localSqlPluginPath); + + for (InsertSqlParser.SqlParseResult result : sqlTree.getExecSqlList()) { + if(LOG.isInfoEnabled()){ + LOG.info("exe-sql:\n" + result.getExecSql()); + } + + boolean isSide = false; + + for(String tableName : result.getSourceTableList()){ + if(sideTableMap.containsKey(tableName)){ + isSide = true; + break; + } + } + + if(isSide){ + //sql-dimensional table contains the dimension table of execution + sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache); + }else{ + tableEnv.sqlUpdate(result.getExecSql()); + } + } + + if(env instanceof MyLocalStreamEnvironment) { + List urlList = new ArrayList<>(); + urlList.addAll(Arrays.asList(dtClassLoader.getURLs())); + ((MyLocalStreamEnvironment) env).setClasspaths(urlList); + } + + env.execute(name); + } + + /** + * This part is just to add classpath for the jar when reading remote execution, and will not submit jar from a local + * @param env + * @param classPathSet + * @throws NoSuchFieldException + * @throws IllegalAccessException + */ + private static void addEnvClassPath(StreamExecutionEnvironment env, Set classPathSet) throws NoSuchFieldException, IllegalAccessException { + if(env instanceof StreamContextEnvironment){ + Field field = env.getClass().getDeclaredField("ctx"); + field.setAccessible(true); + ContextEnvironment contextEnvironment= (ContextEnvironment) field.get(env); + for(URL url : classPathSet){ + contextEnvironment.getClasspaths().add(url); + } + } + } + + private static void registerUDF(SqlTree sqlTree, List jarURList, URLClassLoader parentClassloader, + StreamTableEnvironment tableEnv) + throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException { + //register urf + URLClassLoader classLoader = null; + List funcList = sqlTree.getFunctionList(); + for (CreateFuncParser.SqlParserResult funcInfo : funcList) { + //classloader + if (classLoader == null) { + classLoader = FlinkUtil.loadExtraJar(jarURList, parentClassloader); + } + classLoader.loadClass(funcInfo.getClassName()); + FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName().toUpperCase(), + tableEnv, classLoader); + } + } + + + private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv, + String localSqlPluginPath, String remoteSqlPluginPath, + Map sideTableMap, Map registerTableCache) throws Exception { + Set classPathSet = Sets.newHashSet(); + WaterMarkerAssigner waterMarkerAssigner = new WaterMarkerAssigner(); + for (TableInfo tableInfo : sqlTree.getTableInfoMap().values()) { + + if (tableInfo instanceof SourceTableInfo) { + + SourceTableInfo sourceTableInfo = (SourceTableInfo) tableInfo; + Table table = StreamSourceFactory.getStreamSource(sourceTableInfo, env, tableEnv, localSqlPluginPath); + tableEnv.registerTable(sourceTableInfo.getAdaptName(), table); + //Note --- parameter conversion function can not be used inside a function of the type of polymerization + //Create table in which the function is arranged only need adaptation sql + String adaptSql = sourceTableInfo.getAdaptSelectSql(); + Table adaptTable = adaptSql == null ? table : tableEnv.sqlQuery(adaptSql); + + RowTypeInfo typeInfo = new RowTypeInfo(adaptTable.getSchema().getTypes(), adaptTable.getSchema().getColumnNames()); + DataStream adaptStream = tableEnv.toAppendStream(adaptTable, typeInfo); + String fields = String.join(",", typeInfo.getFieldNames()); + + if(waterMarkerAssigner.checkNeedAssignWaterMarker(sourceTableInfo)){ + adaptStream = waterMarkerAssigner.assignWaterMarker(adaptStream, typeInfo, sourceTableInfo.getEventTimeField(), sourceTableInfo.getMaxOutOrderness()); + fields += ",ROWTIME.ROWTIME"; + }else{ + fields += ",PROCTIME.PROCTIME"; + } + //tableEnv.registerDataStream(tableInfo.getName(), adaptStream, fields); + Table regTable = tableEnv.fromDataStream(adaptStream, fields); + tableEnv.registerTable(tableInfo.getName(), regTable); + registerTableCache.put(tableInfo.getName(), regTable); + classPathSet.add(PluginUtil.getRemoteJarFilePath(tableInfo.getType(), SourceTableInfo.SOURCE_SUFFIX, remoteSqlPluginPath)); + } else if (tableInfo instanceof TargetTableInfo) { + + TableSink tableSink = StreamSinkFactory.getTableSink((TargetTableInfo) tableInfo, localSqlPluginPath); + TypeInformation[] flinkTypes = FlinkUtil.transformTypes(tableInfo.getFieldClasses()); + tableEnv.registerTableSink(tableInfo.getName(), tableInfo.getFields(), flinkTypes, tableSink); + classPathSet.add( PluginUtil.getRemoteJarFilePath(tableInfo.getType(), TargetTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); + } else if(tableInfo instanceof SideTableInfo){ + + sideTableMap.put(tableInfo.getName(), (SideTableInfo) tableInfo); + classPathSet.add(PluginUtil.getRemoteJarFilePath(tableInfo.getType(), SideTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); + }else { + throw new RuntimeException("not support table type:" + tableInfo.getType()); + } + } + + //The plug-in information corresponding to the table is loaded into the classPath env + addEnvClassPath(env, classPathSet); + } + + private static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws IOException { + StreamExecutionEnvironment env = !LOCAL_MODE.equals(deployMode) ? + StreamExecutionEnvironment.getExecutionEnvironment() : + new MyLocalStreamEnvironment(); + + env.setParallelism(FlinkUtil.getEnvParallelism(confProperties)); + + if(FlinkUtil.getMaxEnvParallelism(confProperties) > 0){ + env.setMaxParallelism(FlinkUtil.getMaxEnvParallelism(confProperties)); + } + + if(FlinkUtil.getBufferTimeoutMillis(confProperties) > 0){ + env.setBufferTimeout(FlinkUtil.getBufferTimeoutMillis(confProperties)); + } + + env.setRestartStrategy(RestartStrategies.failureRateRestart( + failureRate, + Time.of(failureInterval, TimeUnit.MINUTES), + Time.of(delayInterval, TimeUnit.SECONDS) + )); + + FlinkUtil.setStreamTimeCharacteristic(env, confProperties); + FlinkUtil.openCheckpoint(env, confProperties); + + return env; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java b/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java new file mode 100644 index 000000000..54ddaa647 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java @@ -0,0 +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 com.dtstack.flink.sql; + +import org.apache.flink.api.common.InvalidProgramException; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; +import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.URL; +import java.util.Collections; +import java.util.List; + +/** + * Local Stream Environment + * + * Company: www.dtstack.com + * @author huyifan.zju@163.com + */ +public class MyLocalStreamEnvironment extends StreamExecutionEnvironment { + private static final Logger LOG = LoggerFactory.getLogger(LocalStreamEnvironment.class); + + /** The configuration to use for the local cluster. */ + private final Configuration conf; + + public List getClasspaths() { + return classpaths; + } + + public void setClasspaths(List classpaths) { + this.classpaths = classpaths; + } + + private List classpaths = Collections.emptyList(); + + /** + * Creates a new local stream environment that uses the default configuration. + */ + public MyLocalStreamEnvironment() { + this(null); + } + + /** + * Creates a new local stream environment that configures its local executor with the given configuration. + * + * @param config The configuration used to configure the local executor. + */ + public MyLocalStreamEnvironment(Configuration config) { + if (!ExecutionEnvironment.areExplicitEnvironmentsAllowed()) { + throw new InvalidProgramException( + "The LocalStreamEnvironment cannot be used when submitting a program through a client, " + + "or running in a TestEnvironment context."); + } + + this.conf = config == null ? new Configuration() : config; + } + + /** + * Executes the JobGraph of the on a mini cluster of CLusterUtil with a user + * specified name. + * + * @param jobName + * name of the job + * @return The result of the job execution, containing elapsed time and accumulators. + */ + @Override + public JobExecutionResult execute(String jobName) throws Exception { + // transform the streaming program into a JobGraph + StreamGraph streamGraph = getStreamGraph(); + streamGraph.setJobName(jobName); + + JobGraph jobGraph = streamGraph.getJobGraph(); + jobGraph.setClasspaths(classpaths); + + Configuration configuration = new Configuration(); + configuration.addAll(jobGraph.getJobConfiguration()); + + configuration.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, -1L); + configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, jobGraph.getMaximumParallelism()); + + // add (and override) the settings with what the user defined + configuration.addAll(this.conf); + + if (LOG.isInfoEnabled()) { + LOG.info("Running job on local embedded Flink mini cluster"); + } + + LocalFlinkMiniCluster exec = new LocalFlinkMiniCluster(configuration, true); + try { + exec.start(); + return exec.submitJobAndWait(jobGraph, getConfig().isSysoutLoggingEnabled()); + } + finally { + transformations.clear(); + exec.stop(); + } + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/classloader/DtClassLoader.java b/core/src/main/java/com/dtstack/flink/sql/classloader/DtClassLoader.java new file mode 100644 index 000000000..420369954 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/classloader/DtClassLoader.java @@ -0,0 +1,222 @@ +/* + * 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 com.dtstack.flink.sql.classloader; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import sun.misc.CompoundEnumeration; + +import java.io.IOException; +import java.net.URL; +import java.net.URLClassLoader; +import java.net.URLStreamHandlerFactory; +import java.util.Collections; +import java.util.Enumeration; +import java.util.LinkedHashSet; + +/** + * + * Custom class loader ---> priority access from the current loader class + * Date: 2017/6/18 + * Company: www.dtstack.com + * @author xuchao + */ + +public class DtClassLoader extends URLClassLoader { + + private static Logger log = LoggerFactory.getLogger(DtClassLoader.class); + + private static final String CLASS_FILE_SUFFIX = ".class"; + + /** + * The parent class loader. + */ + protected ClassLoader parent; + + private boolean hasExternalRepositories = false; + + public DtClassLoader(URL[] urls, ClassLoader parent) { + super(urls, parent); + this.parent = parent; + } + + public DtClassLoader(URL[] urls) { + super(urls); + } + + public DtClassLoader(URL[] urls, ClassLoader parent, URLStreamHandlerFactory factory) { + super(urls, parent, factory); + } + + @Override + public Class loadClass(String name) throws ClassNotFoundException { + return this.loadClass(name, false); + } + + @Override + protected Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + synchronized (getClassLoadingLock(name)) { + if (log.isDebugEnabled()){ + log.debug("loadClass(" + name + ", " + resolve + ")"); + } + Class clazz = null; + + // (0.1) Check our previously loaded class cache + clazz = findLoadedClass(name); + if (clazz != null) { + if (log.isDebugEnabled()){ + log.debug(" Returning class from cache"); + } + if (resolve){ + resolveClass(clazz); + } + return (clazz); + } + + // (2) Search local repositories + if (log.isDebugEnabled()){ + log.debug(" Searching local repositories"); + } + try { + clazz = findClass(name); + if (clazz != null) { + if (log.isDebugEnabled()){ + log.debug(" Loading class from local repository"); + } + if (resolve){ + resolveClass(clazz); + } + return (clazz); + } + } catch (ClassNotFoundException e) { + // Ignore + } + + if (log.isDebugEnabled()){ + log.debug(" Delegating to parent classloader at end: " + parent); + } + + try { + clazz = Class.forName(name, false, parent); + if (clazz != null) { + if (log.isDebugEnabled()){ + log.debug(" Loading class from parent"); + } + if (resolve){ + resolveClass(clazz); + } + return (clazz); + } + } catch (ClassNotFoundException e) { + // Ignore + } + } + + throw new ClassNotFoundException(name); + } + + + @Override + public URL getResource(String name) { + + if (log.isDebugEnabled()){ + log.debug("getResource(" + name + ")"); + } + + URL url = null; + + // (2) Search local repositories + url = findResource(name); + if (url != null) { + if (log.isDebugEnabled()){ + log.debug(" --> Returning '" + url.toString() + "'"); + } + return (url); + } + + // (3) Delegate to parent unconditionally if not already attempted + url = parent.getResource(name); + if (url != null) { + if (log.isDebugEnabled()){ + log.debug(" --> Returning '" + url.toString() + "'"); + } + return (url); + } + + // (4) Resource was not found + if (log.isDebugEnabled()){ + log.debug(" --> Resource not found, returning null"); + } + return (null); + } + + @Override + public void addURL(URL url) { + super.addURL(url); + hasExternalRepositories = true; + } + + /** + * FIXME 需要测试 + * @param name + * @return + * @throws IOException + */ + @Override + public Enumeration getResources(String name) throws IOException { + @SuppressWarnings("unchecked") + Enumeration[] tmp = (Enumeration[]) new Enumeration[1]; + tmp[0] = findResources(name);//优先使用当前类的资源 + + if(!tmp[0].hasMoreElements()){//只有子classLoader找不到任何资源才会调用原生的方法 + return super.getResources(name); + } + + return new CompoundEnumeration<>(tmp); + } + + @Override + public Enumeration findResources(String name) throws IOException { + + if (log.isDebugEnabled()){ + log.debug("findResources(" + name + ")"); + } + + LinkedHashSet result = new LinkedHashSet<>(); + + Enumeration superResource = super.findResources(name); + + while (superResource.hasMoreElements()){ + result.add(superResource.nextElement()); + } + + // Adding the results of a call to the superclass + if (hasExternalRepositories) { + Enumeration otherResourcePaths = super.findResources(name); + while (otherResourcePaths.hasMoreElements()) { + result.add(otherResourcePaths.nextElement()); + } + } + + return Collections.enumeration(result); + } + +} diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/ColumnType.java b/core/src/main/java/com/dtstack/flink/sql/enums/ColumnType.java new file mode 100644 index 000000000..749bbc907 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/enums/ColumnType.java @@ -0,0 +1,67 @@ +/* + * 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. + */ + + + +/** + * 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 com.dtstack.flink.sql.enums; + +/** + * Define standard column type for all the readers or writers that do not + * have special types of their own + * + * Company: www.dtstack.com + * @author huyifan.zju@163.com + */ +public enum ColumnType { + STRING, VARCHAR, CHAR, + INT, MEDIUMINT, TINYINT, DATETIME, SMALLINT, BIGINT, + DOUBLE, FLOAT, + BOOLEAN, + DATE, TIMESTAMP, DECIMAL; + + public static ColumnType fromString(String type) { + if(type == null) { + throw new RuntimeException("null ColumnType!"); + } + + if(type.toUpperCase().startsWith("DECIMAL")) { + return DECIMAL; + } + + return valueOf(type.toUpperCase()); + } + +} diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/ECacheContentType.java b/core/src/main/java/com/dtstack/flink/sql/enums/ECacheContentType.java new file mode 100644 index 000000000..66160d820 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/enums/ECacheContentType.java @@ -0,0 +1,46 @@ +/* + * 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 com.dtstack.flink.sql.enums; + +/** + * Reason: + * Date: 2018/9/10 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public enum ECacheContentType { + + MissVal(0), + SingleLine(1), + MultiLine(2); + + int type; + + ECacheContentType(int type){ + this.type = type; + } + + public int getType(){ + return this.type; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/ECacheType.java b/core/src/main/java/com/dtstack/flink/sql/enums/ECacheType.java new file mode 100644 index 000000000..4cd6aaf4f --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/enums/ECacheType.java @@ -0,0 +1,41 @@ +/* + * 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 com.dtstack.flink.sql.enums; + +/** + * Reason: + * Date: 2018/8/2 + * Company: www.dtstack.com + * @author xuchao + */ +public enum ECacheType { + NONE, LRU; + + public static boolean isValid(String type){ + for(ECacheType tmpType : ECacheType.values()){ + if(tmpType.name().equalsIgnoreCase(type)){ + return true; + } + } + + return false; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/ETableType.java b/core/src/main/java/com/dtstack/flink/sql/enums/ETableType.java new file mode 100644 index 000000000..3ca08f63c --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/enums/ETableType.java @@ -0,0 +1,44 @@ +/* + * 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 com.dtstack.flink.sql.enums; + +/** + * Reason: + * Date: 2018/7/5 + * Company: www.dtstack.com + * @author xuchao + */ +public enum ETableType { + //源表 + SOURCE(1), + //目的表 + SINK(2); + + int type; + + ETableType(int type){ + this.type = type; + } + + public int getType() { + return type; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java new file mode 100644 index 000000000..9b7017743 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java @@ -0,0 +1,98 @@ +/* + * 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 com.dtstack.flink.sql.parser; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * 解析创建自定义方法sql + * Date: 2018/6/26 + * Company: www.dtstack.com + * @author xuchao + */ + +public class CreateFuncParser implements IParser { + + private static final String funcPatternStr = "(?i)\\s*create\\s+(scala|table)\\s+function\\s+(\\S+)\\s+WITH\\s+(\\S+)"; + + private static final Pattern funcPattern = Pattern.compile(funcPatternStr); + + @Override + public boolean verify(String sql) { + return funcPattern.matcher(sql).find(); + } + + @Override + public void parseSql(String sql, SqlTree sqlTree) { + Matcher matcher = funcPattern.matcher(sql); + if(matcher.find()){ + String type = matcher.group(1); + String funcName = matcher.group(2); + String className = matcher.group(3); + SqlParserResult result = new SqlParserResult(); + result.setType(type); + result.setName(funcName); + result.setClassName(className); + sqlTree.addFunc(result); + } + } + + + public static CreateFuncParser newInstance(){ + return new CreateFuncParser(); + } + + public static class SqlParserResult{ + + private String name; + + private String className; + + private String type; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public String getClassName() { + return className; + } + + public void setClassName(String className) { + this.className = className; + } + + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + } + + +} diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java new file mode 100644 index 000000000..72443f301 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java @@ -0,0 +1,114 @@ +/* + * 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 com.dtstack.flink.sql.parser; + +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; + +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * 解析创建表结构sql + * Date: 2018/6/26 + * Company: www.dtstack.com + * @author xuchao + */ + +public class CreateTableParser implements IParser { + + private static final String PATTERN_STR = "(?i)create\\s+table\\s+(\\S+)\\s*\\((.+)\\)\\s*with\\s*\\((.+)\\)"; + + private static final Pattern PATTERN = Pattern.compile(PATTERN_STR); + + public static CreateTableParser newInstance(){ + return new CreateTableParser(); + } + + @Override + public boolean verify(String sql) { + return PATTERN.matcher(sql).find(); + } + + @Override + public void parseSql(String sql, SqlTree sqlTree) { + Matcher matcher = PATTERN.matcher(sql); + if(matcher.find()){ + String tableName = matcher.group(1).toUpperCase(); + String fieldsInfoStr = matcher.group(2); + String propsStr = matcher.group(3); + Map props = parseProp(propsStr); + + SqlParserResult result = new SqlParserResult(); + result.setTableName(tableName); + result.setFieldsInfoStr(fieldsInfoStr); + result.setPropMap(props); + + sqlTree.addPreDealTableInfo(tableName, result); + } + } + + private Map parseProp(String propsStr){ + String[] strs = propsStr.trim().split("'\\s*,"); + Map propMap = Maps.newHashMap(); + for(int i=0; i propMap; + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String getFieldsInfoStr() { + return fieldsInfoStr; + } + + public void setFieldsInfoStr(String fieldsInfoStr) { + this.fieldsInfoStr = fieldsInfoStr; + } + + public Map getPropMap() { + return propMap; + } + + public void setPropMap(Map propMap) { + this.propMap = propMap; + } + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/IParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/IParser.java new file mode 100644 index 000000000..87f91d223 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/parser/IParser.java @@ -0,0 +1,45 @@ +/* + * 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 com.dtstack.flink.sql.parser; + +/** + * Reason: + * Date: 2018/6/26 + * Company: www.dtstack.com + * @author xuchao + */ + +public interface IParser { + + /** + * 是否满足该解析类型 + * @param sql + * @return + */ + boolean verify(String sql); + + /*** + * 解析sql + * @param sql + * @param sqlTree + */ + void parseSql(String sql, SqlTree sqlTree); +} diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java new file mode 100644 index 000000000..e3d65f420 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -0,0 +1,154 @@ +/* + * 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 com.dtstack.flink.sql.parser; + +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlInsert; +import org.apache.calcite.sql.SqlJoin; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlSelect; +import org.apache.calcite.sql.parser.SqlParseException; +import org.apache.calcite.sql.parser.SqlParser; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + +import java.util.List; + +import static org.apache.calcite.sql.SqlKind.IDENTIFIER; + +/** + * 解析flink sql + * sql 只支持 insert 开头的 + * Date: 2018/6/22 + * Company: www.dtstack.com + * @author xuchao + */ + +public class InsertSqlParser implements IParser { + + @Override + public boolean verify(String sql) { + return StringUtils.isNotBlank(sql) && sql.trim().toLowerCase().startsWith("insert"); + } + + public static InsertSqlParser newInstance(){ + InsertSqlParser parser = new InsertSqlParser(); + return parser; + } + + @Override + public void parseSql(String sql, SqlTree sqlTree) { + SqlParser sqlParser = SqlParser.create(sql); + SqlNode sqlNode = null; + try { + sqlNode = sqlParser.parseStmt(); + } catch (SqlParseException e) { + throw new RuntimeException("", e); + } + + SqlParseResult sqlParseResult = new SqlParseResult(); + parseNode(sqlNode, sqlParseResult); + sqlParseResult.setExecSql(sqlNode.toString()); + sqlTree.addExecSql(sqlParseResult); + } + + private static void parseNode(SqlNode sqlNode, SqlParseResult sqlParseResult){ + SqlKind sqlKind = sqlNode.getKind(); + switch (sqlKind){ + case INSERT: + SqlNode sqlTarget = ((SqlInsert)sqlNode).getTargetTable(); + SqlNode sqlSource = ((SqlInsert)sqlNode).getSource(); + sqlParseResult.addTargetTable(sqlTarget.toString()); + parseNode(sqlSource, sqlParseResult); + break; + case SELECT: + SqlNode sqlFrom = ((SqlSelect)sqlNode).getFrom(); + if(sqlFrom.getKind() == IDENTIFIER){ + sqlParseResult.addSourceTable(sqlFrom.toString()); + }else{ + parseNode(sqlFrom, sqlParseResult); + } + break; + case JOIN: + SqlNode leftNode = ((SqlJoin)sqlNode).getLeft(); + SqlNode rightNode = ((SqlJoin)sqlNode).getRight(); + + if(leftNode.getKind() == IDENTIFIER){ + sqlParseResult.addSourceTable(leftNode.toString()); + }else{ + parseNode(leftNode, sqlParseResult); + } + + if(rightNode.getKind() == IDENTIFIER){ + sqlParseResult.addSourceTable(rightNode.toString()); + }else{ + parseNode(rightNode, sqlParseResult); + } + break; + case AS: + //不解析column,所以 as 相关的都是表 + SqlNode identifierNode = ((SqlBasicCall)sqlNode).getOperands()[0]; + if(identifierNode.getKind() != IDENTIFIER){ + parseNode(identifierNode, sqlParseResult); + }else { + sqlParseResult.addSourceTable(identifierNode.toString()); + } + break; + default: + //do nothing + break; + } + } + + public static class SqlParseResult { + + private List sourceTableList = Lists.newArrayList(); + + private List targetTableList = Lists.newArrayList(); + + private String execSql; + + public void addSourceTable(String sourceTable){ + sourceTableList.add(sourceTable); + } + + public void addTargetTable(String targetTable){ + targetTableList.add(targetTable); + } + + public List getSourceTableList() { + return sourceTableList; + } + + public List getTargetTableList() { + return targetTableList; + } + + public String getExecSql() { + return execSql; + } + + public void setExecSql(String execSql) { + this.execSql = execSql; + } + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java new file mode 100644 index 000000000..82f33d2b0 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java @@ -0,0 +1,134 @@ +/* + * 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 com.dtstack.flink.sql.parser; + +import com.dtstack.flink.sql.enums.ETableType; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.TableInfoParserFactory; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.base.Strings; + +import java.util.List; + +/** + * Reason: + * Date: 2018/6/22 + * Company: www.dtstack.com + * @author xuchao + */ + +public class SqlParser { + + private static final char SQL_DELIMITER = ';'; + + private static String LOCAL_SQL_PLUGIN_ROOT; + + private static List sqlParserList = Lists.newArrayList(CreateFuncParser.newInstance(), + CreateTableParser.newInstance(), InsertSqlParser.newInstance()); + + public static void setLocalSqlPluginRoot(String localSqlPluginRoot){ + LOCAL_SQL_PLUGIN_ROOT = localSqlPluginRoot; + } + + /** + * ------flink 支持的 sql 语法包括-------- + * CREATE TABLE sls_stream() with (); + * CREATE (TABLE|SCALA) FUNCTION fcnName WITH com.dtstack.com; + * insert into tb1 select * from tb2; + * @param sql + */ + public static SqlTree parseSql(String sql) throws Exception { + + if(StringUtils.isBlank(sql)){ + throw new RuntimeException("sql is not null"); + } + + if(LOCAL_SQL_PLUGIN_ROOT == null){ + throw new RuntimeException("need to set local sql plugin root"); + } + + sql = sql.replaceAll("--.*", "") + .replaceAll("\r\n", " ") + .replaceAll("\n", " ") + .replace("\t", " ").trim(); + + List sqlArr = DtStringUtil.splitIgnoreQuota(sql, SQL_DELIMITER); + SqlTree sqlTree = new SqlTree(); + + for(String childSql : sqlArr){ + + + if(Strings.isNullOrEmpty(childSql)){ + continue; + } + + boolean result = false; + for(IParser sqlParser : sqlParserList){ + if(!sqlParser.verify(childSql)){ + continue; + } + + sqlParser.parseSql(childSql, sqlTree); + result = true; + } + + if(!result){ + throw new RuntimeException(String.format("%s:Syntax does not support,the format of SQL like insert into tb1 select * from tb2.", childSql)); + } + } + + //解析exec-sql + if(sqlTree.getExecSqlList().size() == 0){ + throw new RuntimeException("sql中没有可执行语句"); + } + + for(InsertSqlParser.SqlParseResult result : sqlTree.getExecSqlList()){ + List sourceTableList = result.getSourceTableList(); + List targetTableList = result.getTargetTableList(); + + for(String tableName : sourceTableList){ + CreateTableParser.SqlParserResult createTableResult = sqlTree.getPreDealTableMap().get(tableName); + if(createTableResult == null){ + throw new RuntimeException("can't find table " + tableName); + } + + TableInfo tableInfo = TableInfoParserFactory.parseWithTableType(ETableType.SOURCE.getType(), + createTableResult, LOCAL_SQL_PLUGIN_ROOT); + sqlTree.addTableInfo(tableName, tableInfo); + } + + for(String tableName : targetTableList){ + CreateTableParser.SqlParserResult createTableResult = sqlTree.getPreDealTableMap().get(tableName); + if(createTableResult == null){ + throw new RuntimeException("can't find table " + tableName); + } + + TableInfo tableInfo = TableInfoParserFactory.parseWithTableType(ETableType.SINK.getType(), + createTableResult, LOCAL_SQL_PLUGIN_ROOT); + sqlTree.addTableInfo(tableName, tableInfo); + } + } + + return sqlTree; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java b/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java new file mode 100644 index 000000000..6ba524818 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java @@ -0,0 +1,79 @@ +/* + * 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 com.dtstack.flink.sql.parser; + + +import com.dtstack.flink.sql.table.TableInfo; +import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.collect.Maps; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + +import java.util.List; +import java.util.Map; + +/** + * 解析sql获得的对象结构 + * Date: 2018/6/25 + * Company: www.dtstack.com + * @author xuchao + */ + +public class SqlTree { + + private List functionList = Lists.newArrayList(); + + private Map preDealTableMap = Maps.newHashMap(); + + private Map tableInfoMap = Maps.newLinkedHashMap(); + + private List execSqlList = Lists.newArrayList(); + + public List getFunctionList() { + return functionList; + } + + public Map getPreDealTableMap() { + return preDealTableMap; + } + + public List getExecSqlList() { + return execSqlList; + } + + public void addFunc(CreateFuncParser.SqlParserResult func){ + functionList.add(func); + } + + public void addPreDealTableInfo(String tableName, CreateTableParser.SqlParserResult table){ + preDealTableMap.put(tableName, table); + } + + public void addExecSql(InsertSqlParser.SqlParseResult execSql){ + execSqlList.add(execSql); + } + + public Map getTableInfoMap() { + return tableInfoMap; + } + + public void addTableInfo(String tableName, TableInfo tableInfo){ + tableInfoMap.put(tableName, tableInfo); + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AliasInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/AliasInfo.java new file mode 100644 index 000000000..d572a7252 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/AliasInfo.java @@ -0,0 +1,51 @@ +/* + * 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 com.dtstack.flink.sql.side; + +/** + * Reason: + * Date: 2018/7/24 + * Company: www.dtstack.com + * @author xuchao + */ + +public class AliasInfo { + + private String name; + + private String alias; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public String getAlias() { + return alias; + } + + public void setAlias(String alias) { + this.alias = alias; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java new file mode 100644 index 000000000..2007b3a0f --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java @@ -0,0 +1,223 @@ +/* + * 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 com.dtstack.flink.sql.side; + +import com.dtstack.flink.sql.enums.ECacheType; +import com.dtstack.flink.sql.side.cache.AbsSideCache; +import com.dtstack.flink.sql.side.cache.CacheObj; +import com.dtstack.flink.sql.side.cache.LRUSideCache; +import org.apache.calcite.sql.JoinType; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.streaming.api.functions.async.RichAsyncFunction; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * 所有的继承接口命名规则:类型 + "AsyncReqRow" 比如==》MysqlAsyncReqRow + * 当前只支持Left join / inner join(join) + * FIXME 不支持right join + * Date: 2018/7/9 + * Company: www.dtstack.com + * @author xuchao + */ + +public abstract class AsyncReqRow extends RichAsyncFunction { + + private static final Logger LOG = LoggerFactory.getLogger(AsyncReqRow.class); + + private static final long serialVersionUID = 2098635244857937717L; + + protected RowTypeInfo rowTypeInfo; + + protected List outFieldInfoList; + + protected List equalFieldList = Lists.newArrayList(); + + protected List equalValIndex = Lists.newArrayList(); + + protected String sqlCondition = ""; + + protected String sideSelectFields = ""; + + protected JoinType joinType; + + //key:返回值位置,返回值在输入数据中的索引位置 + protected Map inFieldIndex = Maps.newHashMap(); + + protected Map sideFieldIndex = Maps.newHashMap(); + + protected SideTableInfo sideTableInfo; + + //TODO 需要指定类型 + protected AbsSideCache sideCache; + + public AsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, + SideTableInfo sideTableInfo){ + this.rowTypeInfo = rowTypeInfo; + this.outFieldInfoList = outFieldInfoList; + this.joinType = joinInfo.getJoinType(); + this.sideTableInfo = sideTableInfo; + parseSelectFields(joinInfo); + buildEqualInfo(joinInfo, sideTableInfo); + } + + private void initCache(){ + if(sideTableInfo.getCacheType() == null || ECacheType.NONE.name().equalsIgnoreCase(sideTableInfo.getCacheType())){ + return; + } + + if(ECacheType.LRU.name().equalsIgnoreCase(sideTableInfo.getCacheType())){ + sideCache = new LRUSideCache(sideTableInfo); + }else{ + throw new RuntimeException("not support side cache with type:" + sideTableInfo.getCacheType()); + } + + sideCache.initCache(); + } + + protected CacheObj getFromCache(String key){ + return sideCache.getFromCache(key); + } + + protected void putCache(String key, CacheObj value){ + sideCache.putCache(key, value); + } + + protected boolean openCache(){ + return sideCache != null; + } + + public void parseSelectFields(JoinInfo joinInfo){ + String sideTableName = joinInfo.getSideTableName(); + String nonSideTableName = joinInfo.getNonSideTable(); + List fields = Lists.newArrayList(); + + int sideIndex = 0; + for( int i=0; i resultFuture){ + if(joinType == JoinType.LEFT){ + //保留left 表数据 + Row row = fillData(input, null); + resultFuture.complete(Collections.singleton(row)); + }else{ + resultFuture.complete(null); + } + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + initCache(); + } + + @Override + public void close() throws Exception { + super.close(); + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/CacheMissVal.java b/core/src/main/java/com/dtstack/flink/sql/side/CacheMissVal.java new file mode 100644 index 000000000..ebcbddcbb --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/CacheMissVal.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 com.dtstack.flink.sql.side; + +import com.dtstack.flink.sql.enums.ECacheContentType; +import com.dtstack.flink.sql.side.cache.CacheObj; + +/** + * 仅仅用来标记未命中的维表数据 + * Date: 2018/8/28 + * Company: www.dtstack.com + * @author xuchao + */ + +public class CacheMissVal { + + private static CacheObj missObj = CacheObj.buildCacheObj(ECacheContentType.MissVal, null); + + public static CacheObj getMissKeyObj(){ + return missObj; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/FieldInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/FieldInfo.java new file mode 100644 index 000000000..f8ba77383 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/FieldInfo.java @@ -0,0 +1,67 @@ +/* + * 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 com.dtstack.flink.sql.side; + +import org.apache.flink.api.common.typeinfo.TypeInformation; + +import java.io.Serializable; + +/** + * Reason: + * Date: 2018/7/23 + * Company: www.dtstack.com + * @author xuchao + */ + +public class FieldInfo implements Serializable { + + private static final long serialVersionUID = -1L; + + private String table; + + private String fieldName; + + private TypeInformation typeInformation; + + public String getTable() { + return table; + } + + public void setTable(String table) { + this.table = table; + } + + public String getFieldName() { + return fieldName; + } + + public void setFieldName(String fieldName) { + this.fieldName = fieldName; + } + + public TypeInformation getTypeInformation() { + return typeInformation; + } + + public void setTypeInformation(TypeInformation typeInformation) { + this.typeInformation = typeInformation; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/FieldReplaceInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/FieldReplaceInfo.java new file mode 100644 index 000000000..bc716ddaa --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/FieldReplaceInfo.java @@ -0,0 +1,63 @@ +/* + * 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 com.dtstack.flink.sql.side; + +import org.apache.flink.calcite.shaded.com.google.common.collect.HashBasedTable; + +/** + * Reason: + * Date: 2018/8/30 + * Company: www.dtstack.com + * @author xuchao + */ + +public class FieldReplaceInfo { + + private HashBasedTable mappingTable; + + private String targetTableName = null; + + private String targetTableAlias = null; + + public void setMappingTable(HashBasedTable mappingTable) { + this.mappingTable = mappingTable; + } + + public HashBasedTable getMappingTable() { + return mappingTable; + } + + public String getTargetTableName() { + return targetTableName; + } + + public void setTargetTableName(String targetTableName) { + this.targetTableName = targetTableName; + } + + public String getTargetTableAlias() { + return targetTableAlias; + } + + public void setTargetTableAlias(String targetTableAlias) { + this.targetTableAlias = targetTableAlias; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java new file mode 100644 index 000000000..03dbde5a6 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java @@ -0,0 +1,193 @@ +/* + * 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 com.dtstack.flink.sql.side; + +import org.apache.calcite.sql.JoinType; +import org.apache.calcite.sql.SqlNode; +import org.apache.flink.calcite.shaded.com.google.common.base.Strings; + +import java.io.Serializable; + +/** + * Join信息 + * Date: 2018/7/24 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class JoinInfo implements Serializable { + + private static final long serialVersionUID = -1L; + + //左表是否是维表 + private boolean leftIsSideTable; + + //右表是否是维表 + private boolean rightIsSideTable; + + private String leftTableName; + + private String leftTableAlias; + + private String rightTableName; + + private String rightTableAlias; + + private SqlNode leftNode; + + private SqlNode rightNode; + + private SqlNode condition; + + private SqlNode selectFields; + + private SqlNode selectNode; + + private JoinType joinType; + + public String getSideTableName(){ + if(leftIsSideTable){ + return leftTableAlias; + } + + return rightTableAlias; + } + + public String getNonSideTable(){ + if(leftIsSideTable){ + return rightTableAlias; + } + + return leftTableAlias; + } + + public String getNewTableName(){ + //兼容左边表是as 的情况 + String leftStr = leftTableName; + leftStr = Strings.isNullOrEmpty(leftStr) ? leftTableAlias : leftStr; + return leftStr + "_" + rightTableName; + } + + public String getNewTableAlias(){ + return leftTableAlias + "_" + rightTableAlias; + } + + public boolean isLeftIsSideTable() { + return leftIsSideTable; + } + + public void setLeftIsSideTable(boolean leftIsSideTable) { + this.leftIsSideTable = leftIsSideTable; + } + + public boolean isRightIsSideTable() { + return rightIsSideTable; + } + + public void setRightIsSideTable(boolean rightIsSideTable) { + this.rightIsSideTable = rightIsSideTable; + } + + public String getLeftTableName() { + return leftTableName; + } + + public void setLeftTableName(String leftTableName) { + this.leftTableName = leftTableName; + } + + public String getRightTableName() { + return rightTableName; + } + + public void setRightTableName(String rightTableName) { + this.rightTableName = rightTableName; + } + + public SqlNode getLeftNode() { + return leftNode; + } + + public void setLeftNode(SqlNode leftNode) { + this.leftNode = leftNode; + } + + public SqlNode getRightNode() { + return rightNode; + } + + public void setRightNode(SqlNode rightNode) { + this.rightNode = rightNode; + } + + public SqlNode getCondition() { + return condition; + } + + public void setCondition(SqlNode condition) { + this.condition = condition; + } + + public SqlNode getSelectFields() { + return selectFields; + } + + public void setSelectFields(SqlNode selectFields) { + this.selectFields = selectFields; + } + + public boolean checkIsSide(){ + return isLeftIsSideTable() || isRightIsSideTable(); + } + + public String getLeftTableAlias() { + return leftTableAlias; + } + + public void setLeftTableAlias(String leftTableAlias) { + this.leftTableAlias = leftTableAlias; + } + + public String getRightTableAlias() { + return rightTableAlias; + } + + public void setRightTableAlias(String rightTableAlias) { + this.rightTableAlias = rightTableAlias; + } + + public SqlNode getSelectNode() { + return selectNode; + } + + public void setSelectNode(SqlNode selectNode) { + this.selectNode = selectNode; + } + + public JoinType getJoinType() { + return joinType; + } + + public void setJoinType(JoinType joinType) { + this.joinType = joinType; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/JoinScope.java b/core/src/main/java/com/dtstack/flink/sql/side/JoinScope.java new file mode 100644 index 000000000..ba07e714a --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/JoinScope.java @@ -0,0 +1,105 @@ +/* + * 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 com.dtstack.flink.sql.side; + + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2018/7/20 + * Company: www.dtstack.com + * @author xuchao + */ + +public class JoinScope { + + private List children = Lists.newArrayList(); + + private Map aliasMap = Maps.newHashMap(); + + public void addScope(ScopeChild scopeChild){ + children.add(scopeChild); + aliasMap.put(scopeChild.getAlias(), scopeChild); + } + + public ScopeChild getScope(String tableAlias){ + return aliasMap.get(tableAlias); + } + + public List getChildren() { + return children; + } + + public TypeInformation getFieldType(String tableName, String fieldName){ + ScopeChild scopeChild = aliasMap.get(tableName); + if(scopeChild == null){ + throw new RuntimeException("can't find "); + } + + RowTypeInfo rowTypeInfo = scopeChild.getRowTypeInfo(); + int index = rowTypeInfo.getFieldIndex(fieldName); + if(index == -1){ + throw new RuntimeException("can't find field: " + fieldName); + } + + return rowTypeInfo.getTypeAt(index); + } + + public static class ScopeChild{ + + private String alias; + + private String tableName; + + private RowTypeInfo rowTypeInfo; + + public String getAlias() { + return alias; + } + + public void setAlias(String alias) { + this.alias = alias; + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public RowTypeInfo getRowTypeInfo() { + return rowTypeInfo; + } + + public void setRowTypeInfo(RowTypeInfo rowTypeInfo) { + this.rowTypeInfo = rowTypeInfo; + } + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java b/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java new file mode 100644 index 000000000..db6cd608f --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java @@ -0,0 +1,133 @@ +/* + * 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 com.dtstack.flink.sql.side; + +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlSelect; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.Iterator; +import java.util.List; + +/** + *FIXME 需要考虑是直接返回所有的字段然后在外层再包裹原先的查询? + * Date: 2018/7/20 + * Company: www.dtstack.com + * @author xuchao + */ + +public class ParserJoinField { + + /** + * 需要解析出selectlist和where中的字段信息 + * @return + */ + public static List getRowTypeInfo(SqlNode sqlNode, JoinScope scope, boolean getAll){ + + if(sqlNode.getKind() != SqlKind.SELECT){ + throw new RuntimeException("------not select node--------\n" + sqlNode.toString()); + } + + List fieldInfoList = Lists.newArrayList(); + if(getAll){ + return getAllField(scope); + } + + SqlSelect sqlSelect = (SqlSelect)sqlNode; + SqlNodeList sqlNodeList = sqlSelect.getSelectList(); + for(SqlNode fieldNode : sqlNodeList.getList()){ + SqlIdentifier identifier = (SqlIdentifier)fieldNode; + if(!identifier.isStar()) { + System.out.println(identifier); + String tableName = identifier.getComponent(0).getSimple(); + String fieldName = identifier.getComponent(1).getSimple(); + TypeInformation type = scope.getFieldType(tableName, fieldName); + FieldInfo fieldInfo = new FieldInfo(); + fieldInfo.setTable(tableName); + fieldInfo.setFieldName(fieldName); + fieldInfo.setTypeInformation(type); + fieldInfoList.add(fieldInfo); + } else { + //处理 + System.out.println("----------"); + int identifierSize = identifier.names.size(); + + switch(identifierSize) { + case 1: + fieldInfoList.addAll(getAllField(scope)); + default: + SqlIdentifier tableIdentify = identifier.skipLast(1); + JoinScope.ScopeChild scopeChild = scope.getScope(tableIdentify.getSimple()); + if(scopeChild == null){ + throw new RuntimeException("can't find table alias " + tableIdentify.getSimple()); + } + + RowTypeInfo field = scopeChild.getRowTypeInfo(); + String[] fieldNames = field.getFieldNames(); + TypeInformation[] types = field.getFieldTypes(); + for(int i=0; i< field.getTotalFields(); i++){ + String fieldName = fieldNames[i]; + TypeInformation type = types[i]; + FieldInfo fieldInfo = new FieldInfo(); + fieldInfo.setTable(tableIdentify.getSimple()); + fieldInfo.setFieldName(fieldName); + fieldInfo.setTypeInformation(type); + fieldInfoList.add(fieldInfo); + } + } + } + } + + return fieldInfoList; + } + + private static List getAllField(JoinScope scope){ + Iterator prefixId = scope.getChildren().iterator(); + List fieldInfoList = Lists.newArrayList(); + while(true) { + JoinScope.ScopeChild resolved; + RowTypeInfo field; + if(!prefixId.hasNext()) { + return fieldInfoList; + } + + resolved = (JoinScope.ScopeChild)prefixId.next(); + field = resolved.getRowTypeInfo(); + String[] fieldNames = field.getFieldNames(); + TypeInformation[] types = field.getFieldTypes(); + for(int i=0; i< field.getTotalFields(); i++){ + String fieldName = fieldNames[i]; + TypeInformation type = types[i]; + FieldInfo fieldInfo = new FieldInfo(); + fieldInfo.setTable(resolved.getAlias()); + fieldInfo.setFieldName(fieldName); + fieldInfo.setTypeInformation(type); + fieldInfoList.add(fieldInfo); + } + } + } + +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java new file mode 100644 index 000000000..6342f70a0 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -0,0 +1,217 @@ +/* + * 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 com.dtstack.flink.sql.side; + +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.calcite.sql.JoinType; +import org.apache.calcite.sql.SqlAsOperator; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlInsert; +import org.apache.calcite.sql.SqlJoin; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlSelect; +import org.apache.calcite.sql.parser.SqlParseException; +import org.apache.calcite.sql.parser.SqlParser; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.flink.calcite.shaded.com.google.common.base.Strings; +import org.apache.flink.calcite.shaded.com.google.common.collect.Queues; + +import java.util.Queue; +import java.util.Set; + +import static org.apache.calcite.sql.SqlKind.*; + +/** + * 解析sql,获取维表的执行信息 + * Date: 2018/7/24 + * Company: www.dtstack.com + * @author xuchao + */ + +public class SideSQLParser { + + public Queue getExeQueue(String exeSql, Set sideTableSet) throws SqlParseException { + exeSql = DtStringUtil.replaceIgnoreQuota(exeSql, "`", ""); + System.out.println("---exeSql---"); + System.out.println(exeSql); + Queue queueInfo = Queues.newLinkedBlockingQueue(); + SqlParser sqlParser = SqlParser.create(exeSql); + SqlNode sqlNode = sqlParser.parseStmt(); + parseSql(sqlNode, sideTableSet, queueInfo); + queueInfo.offer(sqlNode); + return queueInfo; + } + + private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue queueInfo){ + SqlKind sqlKind = sqlNode.getKind(); + switch (sqlKind){ + case INSERT: + SqlNode sqlSource = ((SqlInsert)sqlNode).getSource(); + return parseSql(sqlSource, sideTableSet, queueInfo); + case SELECT: + SqlNode sqlFrom = ((SqlSelect)sqlNode).getFrom(); + if(sqlFrom.getKind() != IDENTIFIER){ + Object result = parseSql(sqlFrom, sideTableSet, queueInfo); + if(result instanceof JoinInfo){ + dealSelectResultWithJoinInfo((JoinInfo)result, (SqlSelect) sqlNode, queueInfo); + }else if(result instanceof AliasInfo){ + String tableName = ((AliasInfo) result).getName(); + if(sideTableSet.contains(tableName)){ + throw new RuntimeException("side-table must be used in join operator"); + } + } + }else{ + String tableName = ((SqlIdentifier)sqlFrom).getSimple(); + if(sideTableSet.contains(tableName)){ + throw new RuntimeException("side-table must be used in join operator"); + } + } + break; + case JOIN: + return dealJoinNode((SqlJoin) sqlNode, sideTableSet, queueInfo); + case AS: + SqlNode info = ((SqlBasicCall)sqlNode).getOperands()[0]; + SqlNode alias = ((SqlBasicCall) sqlNode).getOperands()[1]; + String infoStr; + + if(info.getKind() == IDENTIFIER){ + infoStr = info.toString(); + }else{ + infoStr = parseSql(info, sideTableSet, queueInfo).toString(); + } + + AliasInfo aliasInfo = new AliasInfo(); + aliasInfo.setName(infoStr); + aliasInfo.setAlias(alias.toString()); + + return aliasInfo; + } + + return ""; + } + + private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue queueInfo){ + SqlNode leftNode = joinNode.getLeft(); + SqlNode rightNode = joinNode.getRight(); + JoinType joinType = joinNode.getJoinType(); + String leftTbName = ""; + String leftTbAlias = ""; + + if(leftNode.getKind() == IDENTIFIER){ + leftTbName = leftNode.toString(); + }else if(leftNode.getKind() == JOIN){ + Object leftNodeJoinInfo = parseSql(leftNode, sideTableSet, queueInfo); + System.out.println(leftNodeJoinInfo); + }else if(leftNode.getKind() == AS){ + AliasInfo aliasInfo = (AliasInfo) parseSql(leftNode, sideTableSet, queueInfo); + leftTbName = aliasInfo.getName(); + leftTbAlias = aliasInfo.getAlias(); + }else{ + throw new RuntimeException("---not deal---"); + } + + boolean leftIsSide = checkIsSideTable(leftTbName, sideTableSet); + if(leftIsSide){ + throw new RuntimeException("side-table must be at the right of join operator"); + } + + String rightTableName = ""; + String rightTableAlias = ""; + + if(rightNode.getKind() == IDENTIFIER){ + rightTableName = rightNode.toString(); + }else{ + AliasInfo aliasInfo = (AliasInfo)parseSql(rightNode, sideTableSet, queueInfo); + rightTableName = aliasInfo.getName(); + rightTableAlias = aliasInfo.getAlias(); + } + + boolean rightIsSide = checkIsSideTable(rightTableName, sideTableSet); + if(joinType == JoinType.RIGHT){ + throw new RuntimeException("side join not support join type of right[current support inner join and left join]"); + } + + JoinInfo tableInfo = new JoinInfo(); + tableInfo.setLeftTableName(leftTbName); + tableInfo.setRightTableName(rightTableName); + tableInfo.setLeftTableAlias(leftTbAlias); + tableInfo.setRightTableAlias(rightTableAlias); + tableInfo.setLeftIsSideTable(leftIsSide); + tableInfo.setRightIsSideTable(rightIsSide); + tableInfo.setLeftNode(leftNode); + tableInfo.setRightNode(rightNode); + tableInfo.setJoinType(joinType); + tableInfo.setCondition(joinNode.getCondition()); + + return tableInfo; + } + + + private void dealSelectResultWithJoinInfo(JoinInfo joinInfo, SqlSelect sqlNode, Queue queueInfo){ + //SideJoinInfo重命名 + if(joinInfo.checkIsSide()){ + joinInfo.setSelectFields(sqlNode.getSelectList()); + joinInfo.setSelectNode(sqlNode); + if(joinInfo.isRightIsSideTable()){ + //判断left是不是一个简单表 + if(joinInfo.getLeftNode().toString().contains("SELECT")){ + queueInfo.offer(joinInfo.getLeftNode()); + } + + queueInfo.offer(joinInfo); + }else{ + //判断right是不是一个简单表 + if(joinInfo.getRightNode().getKind() == SELECT){ + queueInfo.offer(joinInfo.getLeftNode()); + } + + queueInfo.offer(joinInfo); + } + + //更新from 节点 + SqlOperator operator = new SqlAsOperator(); + SqlParserPos sqlParserPos = new SqlParserPos(0, 0); + String joinLeftTableName = joinInfo.getLeftTableName(); + String joinLeftTableAlias = joinInfo.getLeftTableAlias(); + joinLeftTableName = Strings.isNullOrEmpty(joinLeftTableName) ? joinLeftTableAlias : joinLeftTableName; + String newTableName = joinLeftTableName + "_" + joinInfo.getRightTableName(); + String newTableAlias = joinInfo.getLeftTableAlias() + "_" + joinInfo.getRightTableAlias(); + SqlIdentifier sqlIdentifier = new SqlIdentifier(newTableName, null, sqlParserPos); + SqlIdentifier sqlIdentifierAlias = new SqlIdentifier(newTableAlias, null, sqlParserPos); + SqlNode[] sqlNodes = new SqlNode[2]; + sqlNodes[0] = sqlIdentifier; + sqlNodes[1] = sqlIdentifierAlias; + SqlBasicCall sqlBasicCall = new SqlBasicCall(operator, sqlNodes, sqlParserPos); + sqlNode.setFrom(sqlBasicCall); + } + } + + private boolean checkIsSideTable(String tableName, Set sideTableList){ + if(sideTableList.contains(tableName)){ + return true; + } + + return false; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java new file mode 100644 index 000000000..f290e88b8 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -0,0 +1,576 @@ +/* + * 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 com.dtstack.flink.sql.side; + +import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlDataTypeSpec; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlInsert; +import org.apache.calcite.sql.SqlJoin; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlLiteral; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlSelect; +import org.apache.calcite.sql.fun.SqlCase; +import org.apache.calcite.sql.parser.SqlParseException; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.HashBasedTable; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.streaming.api.datastream.AsyncDataStream; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.java.StreamTableEnvironment; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.TimeUnit; + +import static org.apache.calcite.sql.SqlKind.*; + +/** + * Reason: + * Date: 2018/7/24 + * Company: www.dtstack.com + * @author xuchao + */ + +public class SideSqlExec { + + private String localSqlPluginPath = null; + + private SideSQLParser sideSQLParser = new SideSQLParser(); + + public void exec(String sql, Map sideTableMap, StreamTableEnvironment tableEnv, + Map tableCache) + throws Exception { + + if(localSqlPluginPath == null){ + throw new RuntimeException("need to set localSqlPluginPath"); + } + + Map localTableCache = Maps.newHashMap(tableCache); + Queue exeQueue = sideSQLParser.getExeQueue(sql, sideTableMap.keySet()); + Object pollObj = null; + + //TODO 需要清理 + boolean preIsSideJoin = false; + List replaceInfoList = Lists.newArrayList(); + + while((pollObj = exeQueue.poll()) != null){ + + if(pollObj instanceof SqlNode){ + SqlNode pollSqlNode = (SqlNode) pollObj; + + if(preIsSideJoin){ + preIsSideJoin = false; + for(FieldReplaceInfo replaceInfo : replaceInfoList){ + replaceFieldName(pollSqlNode, replaceInfo.getMappingTable(), replaceInfo.getTargetTableName(), replaceInfo.getTargetTableAlias()); + } + } + + if(pollSqlNode.getKind() == INSERT){ + tableEnv.sqlUpdate(pollSqlNode.toString()); + }else if(pollSqlNode.getKind() == AS){ + AliasInfo aliasInfo = parseASNode(pollSqlNode); + Table table = tableEnv.sql(aliasInfo.getName()); + tableEnv.registerTable(aliasInfo.getAlias(), table); + localTableCache.put(aliasInfo.getAlias(), table); + } + + }else if (pollObj instanceof JoinInfo){ + preIsSideJoin = true; + JoinInfo joinInfo = (JoinInfo) pollObj; + + JoinScope joinScope = new JoinScope(); + JoinScope.ScopeChild leftScopeChild = new JoinScope.ScopeChild(); + leftScopeChild.setAlias(joinInfo.getLeftTableAlias()); + leftScopeChild.setTableName(joinInfo.getLeftTableName()); + + Table leftTable = getTableFromCache(localTableCache, joinInfo.getLeftTableAlias(), joinInfo.getLeftTableName()); + RowTypeInfo leftTypeInfo = new RowTypeInfo(leftTable.getSchema().getTypes(), leftTable.getSchema().getColumnNames()); + leftScopeChild.setRowTypeInfo(leftTypeInfo); + + JoinScope.ScopeChild rightScopeChild = new JoinScope.ScopeChild(); + rightScopeChild.setAlias(joinInfo.getRightTableAlias()); + rightScopeChild.setTableName(joinInfo.getRightTableName()); + SideTableInfo sideTableInfo = sideTableMap.get(joinInfo.getRightTableName()); + if(sideTableInfo == null){ + sideTableInfo = sideTableMap.get(joinInfo.getRightTableName()); + } + + if(sideTableInfo == null){ + throw new RuntimeException("can't not find side table:" + joinInfo.getRightTableName()); + } + + if(!checkJoinCondition(joinInfo.getCondition(), joinInfo.getRightTableAlias(), sideTableInfo.getPrimaryKeys())){ + throw new RuntimeException("ON condition must contain all equal fields!!!"); + } + + rightScopeChild.setRowTypeInfo(sideTableInfo.getRowTypeInfo()); + + joinScope.addScope(leftScopeChild); + joinScope.addScope(rightScopeChild); + + //获取两个表的所有字段 + List sideJoinFieldInfo = ParserJoinField.getRowTypeInfo(joinInfo.getSelectNode(), joinScope, true); + + String leftTableAlias = joinInfo.getLeftTableAlias(); + Table targetTable = localTableCache.get(leftTableAlias); + if(targetTable == null){ + targetTable = localTableCache.get(joinInfo.getLeftTableName()); + } + + RowTypeInfo typeInfo = new RowTypeInfo(targetTable.getSchema().getTypes(), targetTable.getSchema().getColumnNames()); + DataStream adaptStream = tableEnv.toAppendStream(targetTable, org.apache.flink.types.Row.class); + + //join side table 之前先 keyby ===>减少 维表在各个async 的缓存大小 + if(sideTableInfo.isPartitionedJoin()){ + List leftJoinColList = getConditionFields(joinInfo.getCondition(), joinInfo.getLeftTableAlias()); + String[] leftJoinColArr = new String[leftJoinColList.size()]; + leftJoinColArr = leftJoinColList.toArray(leftJoinColArr); + adaptStream = adaptStream.keyBy(leftJoinColArr); + } + + AsyncReqRow asyncDbReq = loadAsyncReq(sideTableInfo.getType(), localSqlPluginPath, typeInfo, joinInfo, sideJoinFieldInfo, sideTableInfo); + //TODO 并行度应该设置为多少?超时时间设置? capacity设置? + DataStream dsOut = AsyncDataStream.orderedWait(adaptStream, asyncDbReq, 10000, TimeUnit.MILLISECONDS, 10) + .setParallelism(sideTableInfo.getParallelism()); + + HashBasedTable mappingTable = HashBasedTable.create(); + RowTypeInfo sideOutTypeInfo = buildOutRowTypeInfo(sideJoinFieldInfo, mappingTable); + dsOut.getTransformation().setOutputType(sideOutTypeInfo); + String targetTableName = joinInfo.getNewTableName(); + String targetTableAlias = joinInfo.getNewTableAlias(); + + FieldReplaceInfo replaceInfo = new FieldReplaceInfo(); + replaceInfo.setMappingTable(mappingTable); + replaceInfo.setTargetTableName(targetTableName); + replaceInfo.setTargetTableAlias(targetTableAlias); + + replaceInfoList.add(replaceInfo); + + tableEnv.registerDataStream(joinInfo.getNewTableName(), dsOut, String.join(",", sideOutTypeInfo.getFieldNames())); + } + } + + } + + public AliasInfo parseASNode(SqlNode sqlNode) throws SqlParseException { + SqlKind sqlKind = sqlNode.getKind(); + if(sqlKind != AS){ + throw new RuntimeException(sqlNode + " is not 'as' operator"); + } + + SqlNode info = ((SqlBasicCall)sqlNode).getOperands()[0]; + SqlNode alias = ((SqlBasicCall) sqlNode).getOperands()[1]; + + AliasInfo aliasInfo = new AliasInfo(); + aliasInfo.setName(info.toString()); + aliasInfo.setAlias(alias.toString()); + + return aliasInfo; + } + + public RowTypeInfo buildOutRowTypeInfo(List sideJoinFieldInfo, HashBasedTable mappingTable){ + TypeInformation[] sideOutTypes = new TypeInformation[sideJoinFieldInfo.size()]; + String[] sideOutNames = new String[sideJoinFieldInfo.size()]; + for(int i=0; i mappingTable, String targetTableName, String tableAlias) { + SqlKind sqlKind = sqlNode.getKind(); + switch (sqlKind) { + case INSERT: + SqlNode sqlSource = ((SqlInsert) sqlNode).getSource(); + replaceFieldName(sqlSource, mappingTable, targetTableName, tableAlias); + break; + case AS: + SqlNode asNode = ((SqlBasicCall)sqlNode).getOperands()[0]; + replaceFieldName(asNode, mappingTable, targetTableName, tableAlias); + break; + case SELECT: + SqlSelect sqlSelect = (SqlSelect) filterNodeWithTargetName(sqlNode, targetTableName); + if(sqlSelect == null){ + return; + } + + SqlNode sqlSource1 = sqlSelect.getFrom(); + if(sqlSource1.getKind() == AS){ + String tableName = ((SqlBasicCall)sqlSource1).getOperands()[0].toString(); + if(tableName.equalsIgnoreCase(targetTableName)){ + SqlNodeList sqlSelectList = sqlSelect.getSelectList(); + SqlNode whereNode = sqlSelect.getWhere(); + SqlNodeList sqlGroup = sqlSelect.getGroup(); + + //TODO 暂时不处理having + SqlNode sqlHaving = sqlSelect.getHaving(); + + List newSelectNodeList = Lists.newArrayList(); + for( int i=0; i replaceNodeList = replaceSelectStarFieldName(selectNode, mappingTable, tableAlias); + newSelectNodeList.addAll(replaceNodeList); + continue; + } + + SqlNode replaceNode = replaceSelectFieldName(selectNode, mappingTable, tableAlias); + if(replaceNode == null){ + continue; + } + + //sqlSelectList.set(i, replaceNode); + newSelectNodeList.add(replaceNode); + } + + SqlNodeList newSelectList = new SqlNodeList(newSelectNodeList, sqlSelectList.getParserPosition()); + sqlSelect.setSelectList(newSelectList); + + //where + if(whereNode != null){ + SqlNode[] sqlNodeList = ((SqlBasicCall)whereNode).getOperands(); + for(int i =0; i mappingTable, String tableAlias){ + if(groupNode.getKind() == IDENTIFIER){ + SqlIdentifier sqlIdentifier = (SqlIdentifier) groupNode; + String mappingFieldName = mappingTable.get(sqlIdentifier.getComponent(0).getSimple(), sqlIdentifier.getComponent(1).getSimple()); + sqlIdentifier = sqlIdentifier.setName(0, tableAlias); + return sqlIdentifier.setName(1, mappingFieldName); + }else if(groupNode instanceof SqlBasicCall){ + SqlBasicCall sqlBasicCall = (SqlBasicCall) groupNode; + for(int i=0; i outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + String pathOfType = sideType + "side"; + String pluginJarPath = PluginUtil.getJarFileDirPath(pathOfType, sqlRootDir); + DtClassLoader dtClassLoader = (DtClassLoader) classLoader; + PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); + String className = PluginUtil.getSqlSideClassName(sideType, "side"); + return dtClassLoader.loadClass(className).asSubclass(AsyncReqRow.class) + .getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, SideTableInfo.class).newInstance(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + public void setLocalSqlPluginPath(String localSqlPluginPath){ + this.localSqlPluginPath = localSqlPluginPath; + } + + private Table getTableFromCache(Map localTableCache, String tableAlias, String tableName){ + Table table = localTableCache.get(tableAlias); + if(table == null){ + table = localTableCache.get(tableName); + } + + if(table == null){ + throw new RuntimeException("not register table " + tableName); + } + + return table; + } + + private List replaceSelectStarFieldName(SqlNode selectNode, HashBasedTable mappingTable, String tableAlias){ + SqlIdentifier sqlIdentifier = (SqlIdentifier) selectNode; + List sqlNodes = Lists.newArrayList(); + if(sqlIdentifier.isStar()){//处理 [* or table.*] + int identifierSize = sqlIdentifier.names.size(); + Collection columns = null; + if(identifierSize == 1){ + columns = mappingTable.values(); + }else{ + columns = mappingTable.row(sqlIdentifier.names.get(0)).values(); + } + + for(String colAlias : columns){ + SqlParserPos sqlParserPos = new SqlParserPos(0, 0); + List columnInfo = Lists.newArrayList(); + columnInfo.add(tableAlias); + columnInfo.add(colAlias); + SqlIdentifier sqlIdentifierAlias = new SqlIdentifier(columnInfo, sqlParserPos); + sqlNodes.add(sqlIdentifierAlias); + } + + return sqlNodes; + }else{ + throw new RuntimeException("is not a star select field." + selectNode); + } + } + + private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable mappingTable, String tableAlias){ + if(selectNode.getKind() == AS){ + SqlNode leftNode = ((SqlBasicCall)selectNode).getOperands()[0]; + SqlNode replaceNode = replaceSelectFieldName(leftNode, mappingTable, tableAlias); + if(replaceNode != null){ + ((SqlBasicCall)selectNode).getOperands()[0] = replaceNode; + } + + return selectNode; + }else if(selectNode.getKind() == IDENTIFIER){ + SqlIdentifier sqlIdentifier = (SqlIdentifier) selectNode; + + if(sqlIdentifier.names.size() == 1){ + return null; + } + + String mappingFieldName = mappingTable.get(sqlIdentifier.getComponent(0).getSimple(), sqlIdentifier.getComponent(1).getSimple()); + if(mappingFieldName == null){ + throw new RuntimeException("can't find mapping fieldName:" + selectNode.toString() ); + } + + sqlIdentifier = sqlIdentifier.setName(0, tableAlias); + sqlIdentifier = sqlIdentifier.setName(1, mappingFieldName); + return sqlIdentifier; + }else if(selectNode.getKind() == LITERAL || selectNode.getKind() == LITERAL_CHAIN){//字面含义 + return selectNode; + }else if(selectNode.getKind() == OTHER_FUNCTION + || selectNode.getKind() == DIVIDE + || selectNode.getKind() == CAST + || selectNode.getKind() == SUM + || selectNode.getKind() == AVG + || selectNode.getKind() == MAX + || selectNode.getKind() == MIN + || selectNode.getKind() == TRIM + || selectNode.getKind() == TIMES + || selectNode.getKind() == PLUS + || selectNode.getKind() == IN + || selectNode.getKind() == OR + || selectNode.getKind() == AND + || selectNode.getKind() == COUNT + || selectNode.getKind() == SUM + || selectNode.getKind() == SUM0 + || selectNode.getKind() == LEAD + || selectNode.getKind() == LAG + || selectNode.getKind() == EQUALS + || selectNode.getKind() == NOT_EQUALS + || selectNode.getKind() == MINUS + || selectNode.getKind() == TUMBLE + || selectNode.getKind() == TUMBLE_START + || selectNode.getKind() == TUMBLE_END + || selectNode.getKind() == SESSION + || selectNode.getKind() == SESSION_START + || selectNode.getKind() == SESSION_END + || selectNode.getKind() == BETWEEN + || selectNode.getKind() == IS_NULL + || selectNode.getKind() == IS_NOT_NULL + ){ + SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; + for(int i=0; i primaryKeys){ + + List conditionFields = getConditionFields(conditionNode, sideTableAlias); + if(CollectionUtils.isEqualCollection(conditionFields, primaryKeys)){ + return true; + } + + return false; + } + + public List getConditionFields(SqlNode conditionNode, String specifyTableName){ + List sqlNodeList = Lists.newArrayList(); + if(conditionNode.getKind() == SqlKind.AND){ + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); + }else{ + sqlNodeList.add(conditionNode); + } + + List conditionFields = Lists.newArrayList(); + for(SqlNode sqlNode : sqlNodeList){ + if(sqlNode.getKind() != SqlKind.EQUALS){ + throw new RuntimeException("not equal operator."); + } + + SqlIdentifier left = (SqlIdentifier)((SqlBasicCall)sqlNode).getOperands()[0]; + SqlIdentifier right = (SqlIdentifier)((SqlBasicCall)sqlNode).getOperands()[1]; + + String leftTableName = left.getComponent(0).getSimple(); + String rightTableName = right.getComponent(0).getSimple(); + + String tableCol = ""; + if(leftTableName.equalsIgnoreCase(specifyTableName)){ + tableCol = left.getComponent(1).getSimple(); + }else if(rightTableName.equalsIgnoreCase(specifyTableName)){ + tableCol = right.getComponent(1).getSimple(); + }else{ + throw new RuntimeException(String.format("side table:%s join condition is wrong", specifyTableName)); + } + + conditionFields.add(tableCol); + } + + return conditionFields; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java new file mode 100644 index 000000000..c840eee10 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java @@ -0,0 +1,98 @@ +/* + * 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 com.dtstack.flink.sql.side; + +import com.dtstack.flink.sql.table.TableInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; + +import java.io.Serializable; + +/** + * Reason: + * Date: 2018/7/25 + * Company: www.dtstack.com + * @author xuchao + */ + +public abstract class SideTableInfo extends TableInfo implements Serializable { + + public static final String TARGET_SUFFIX = "Side"; + + public static final String CACHE_KEY = "cache"; + + public static final String CACHE_SIZE_KEY = "cacheSize"; + + public static final String CACHE_TTLMS_KEY = "cacheTTLMs"; + + public static final String PARTITIONED_JOIN_KEY = "partitionedJoin"; + + private String cacheType = "none";//None or LRU + + private int cacheSize = 10000; + + private long cacheTimeout = 60 * 1000;//默认1分钟 + + private boolean partitionedJoin = false; + + public RowTypeInfo getRowTypeInfo(){ + Class[] fieldClass = getFieldClasses(); + TypeInformation[] types = new TypeInformation[fieldClass.length]; + String[] fieldNames = getFields(); + for(int i=0; i sideParser = dtClassLoader.loadClass(className); + if(!AbsSideTableParser.class.isAssignableFrom(sideParser)){ + throw new RuntimeException("class " + sideParser.getName() + " not subClass of AbsSideTableParser"); + } + + return sideParser.asSubclass(AbsTableParser.class).newInstance(); + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/cache/AbsSideCache.java b/core/src/main/java/com/dtstack/flink/sql/side/cache/AbsSideCache.java new file mode 100644 index 000000000..757f91600 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/cache/AbsSideCache.java @@ -0,0 +1,46 @@ +/* + * 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 com.dtstack.flink.sql.side.cache; + +import com.dtstack.flink.sql.side.SideTableInfo; + +/** + * Reason: + * Date: 2018/9/10 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public abstract class AbsSideCache { + + protected SideTableInfo sideTableInfo; + + public AbsSideCache(SideTableInfo sideTableInfo){ + this.sideTableInfo = sideTableInfo; + } + + public abstract void initCache(); + + public abstract CacheObj getFromCache(String key); + + public abstract void putCache(String key, CacheObj value); +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/cache/CacheObj.java b/core/src/main/java/com/dtstack/flink/sql/side/cache/CacheObj.java new file mode 100644 index 000000000..57ff7cab8 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/cache/CacheObj.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 com.dtstack.flink.sql.side.cache; + +import com.dtstack.flink.sql.enums.ECacheContentType; + +/** + * Reason: + * Date: 2018/9/10 + * Company: www.dtstack.com + * @author xuchao + */ + +public class CacheObj { + + private ECacheContentType type; + + private Object content; + + private CacheObj(ECacheContentType type, Object content){ + this.type = type; + this.content = content; + } + + public static CacheObj buildCacheObj(ECacheContentType type, Object content){ + return new CacheObj(type, content); + } + + public ECacheContentType getType() { + return type; + } + + public void setType(ECacheContentType type) { + this.type = type; + } + + public Object getContent() { + return content; + } + + public void setContent(Object content) { + this.content = content; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/cache/LRUSideCache.java b/core/src/main/java/com/dtstack/flink/sql/side/cache/LRUSideCache.java new file mode 100644 index 000000000..475536a4c --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/cache/LRUSideCache.java @@ -0,0 +1,70 @@ +/* + * 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 com.dtstack.flink.sql.side.cache; + +import com.dtstack.flink.sql.side.SideTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.cache.Cache; +import org.apache.flink.calcite.shaded.com.google.common.cache.CacheBuilder; + +import java.util.concurrent.TimeUnit; + +/** + * Reason: + * Date: 2018/9/10 + * Company: www.dtstack.com + * @author xuchao + */ + +public class LRUSideCache extends AbsSideCache{ + + protected transient Cache cache; + + public LRUSideCache(SideTableInfo sideTableInfo) { + super(sideTableInfo); + } + + @Override + public void initCache() { + //当前只有LRU + cache = CacheBuilder.newBuilder() + .maximumSize(sideTableInfo.getCacheSize()) + .expireAfterWrite(sideTableInfo.getCacheTimeout(), TimeUnit.MILLISECONDS) + .build(); + } + + @Override + public CacheObj getFromCache(String key) { + if(cache == null){ + return null; + } + + return cache.getIfPresent(key); + } + + @Override + public void putCache(String key, CacheObj value) { + if(cache == null){ + return; + } + + cache.put(key, value); + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/sink/IStreamSinkGener.java b/core/src/main/java/com/dtstack/flink/sql/sink/IStreamSinkGener.java new file mode 100644 index 000000000..3cfd48f1b --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/sink/IStreamSinkGener.java @@ -0,0 +1,34 @@ +/* + * 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 com.dtstack.flink.sql.sink; + +import com.dtstack.flink.sql.table.TargetTableInfo; + +/** + * Reason: + * Date: 2017/7/31 + * Company: www.dtstack.com + * @author xuchao + */ +public interface IStreamSinkGener { + + T genStreamSink(TargetTableInfo targetTableInfo); +} diff --git a/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java b/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java new file mode 100644 index 000000000..cc7ed0179 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java @@ -0,0 +1,77 @@ +/* + * 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 com.dtstack.flink.sql.sink; + +import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.flink.table.sinks.TableSink; + +/** + * 根据指定的sink type 加载jar,并初始化对象 + * Date: 2017/3/10 + * Company: www.dtstack.com + * @author xuchao + */ + +public class StreamSinkFactory { + + public static String CURR_TYPE = "sink"; + + public static AbsTableParser getSqlParser(String resultType, String sqlRootDir) throws Exception { + String parserType = resultType + CURR_TYPE.substring(0, 1).toUpperCase() + CURR_TYPE.substring(1); + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + String pluginJarPath = PluginUtil.getJarFileDirPath(resultType + CURR_TYPE, sqlRootDir); + DtClassLoader dtClassLoader = (DtClassLoader) classLoader; + PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); + String className = PluginUtil.getSqlParserClassName(resultType, CURR_TYPE); + Class targetParser = dtClassLoader.loadClass(className); + if(!AbsTableParser.class.isAssignableFrom(targetParser)){ + throw new RuntimeException("class " + targetParser.getName() + " not subClass of AbsTableParser"); + } + + return targetParser.asSubclass(AbsTableParser.class).newInstance(); + } + + public static TableSink getTableSink(TargetTableInfo targetTableInfo, String localSqlRootDir) throws Exception { + + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + if(!(classLoader instanceof DtClassLoader)){ + throw new RuntimeException("it's not a correct classLoader instance, it's type must be DtClassLoader!"); + } + + String resultType = targetTableInfo.getType(); + String pluginJarDirPath = PluginUtil.getJarFileDirPath(resultType + CURR_TYPE, localSqlRootDir); + String className = PluginUtil.getGenerClassName(resultType, CURR_TYPE); + + DtClassLoader dtClassLoader = (DtClassLoader) classLoader; + PluginUtil.addPluginJar(pluginJarDirPath, dtClassLoader); + Class sinkClass = dtClassLoader.loadClass(className); + if(!IStreamSinkGener.class.isAssignableFrom(sinkClass)){ + throw new RuntimeException("class " + sinkClass + " not subClass of IStreamSinkGener"); + } + + IStreamSinkGener streamSinkGener = sinkClass.asSubclass(IStreamSinkGener.class).newInstance(); + Object result = streamSinkGener.genStreamSink(targetTableInfo); + return (TableSink) result; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/source/IStreamSourceGener.java b/core/src/main/java/com/dtstack/flink/sql/source/IStreamSourceGener.java new file mode 100644 index 000000000..530900dfe --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/source/IStreamSourceGener.java @@ -0,0 +1,44 @@ +/* + * 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 com.dtstack.flink.sql.source; + +import com.dtstack.flink.sql.table.SourceTableInfo; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.java.StreamTableEnvironment; + +/** + * Reason: + * Date: 2017/8/2 + * Company: www.dtstack.com + * @author xuchao + */ +public interface IStreamSourceGener { + + /** + * 获取输入源 + * @param sourceTableInfo + * @param env + * @param tableEnv + * @return + */ + T genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv); + +} diff --git a/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java b/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java new file mode 100644 index 000000000..458d8bc69 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.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 com.dtstack.flink.sql.source; + + +import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.table.AbsSourceParser; +import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.java.StreamTableEnvironment; + +/** + * 创建streamTableSource + * Date: 2017/3/10 + * Company: www.dtstack.com + * @author xuchao + */ + +public class StreamSourceFactory { + + private static final String CURR_TYPE = "source"; + + + public static AbsSourceParser getSqlParser(String resultType, String sqlRootDir) throws Exception { + + String parserType = resultType + CURR_TYPE.substring(0, 1).toUpperCase() + CURR_TYPE.substring(1); + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + String pluginJarPath = PluginUtil.getJarFileDirPath(resultType +CURR_TYPE, sqlRootDir); + DtClassLoader dtClassLoader = (DtClassLoader) classLoader; + PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); + String className = PluginUtil.getSqlParserClassName(resultType, CURR_TYPE); + Class sourceParser = dtClassLoader.loadClass(className); + if(!AbsSourceParser.class.isAssignableFrom(sourceParser)){ + throw new RuntimeException("class " + sourceParser.getName() + " not subClass of AbsSourceParser"); + } + + return sourceParser.asSubclass(AbsSourceParser.class).newInstance(); + } + + /** + * 根据指定的类型构造数据源 + * @param sourceTableInfo + * @return + */ + public static Table getStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, + StreamTableEnvironment tableEnv, String sqlRootDir) throws Exception { + + String sourceTypeStr = sourceTableInfo.getType(); + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + + String pluginJarPath = PluginUtil.getJarFileDirPath(sourceTypeStr + CURR_TYPE, sqlRootDir); + String className = PluginUtil.getGenerClassName(sourceTypeStr, CURR_TYPE); + + DtClassLoader dtClassLoader = (DtClassLoader) classLoader; + PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); + Class sourceClass = dtClassLoader.loadClass(className); + + if(!IStreamSourceGener.class.isAssignableFrom(sourceClass)){ + throw new RuntimeException("class " + sourceClass.getName() + " not subClass of IStreamSourceGener"); + } + + IStreamSourceGener sourceGener = sourceClass.asSubclass(IStreamSourceGener.class).newInstance(); + Object object = sourceGener.genStreamSource(sourceTableInfo, env, tableEnv); + return (Table) object; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java new file mode 100644 index 000000000..82c9af96e --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java @@ -0,0 +1,75 @@ +/* + * 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 com.dtstack.flink.sql.table; + +import com.dtstack.flink.sql.enums.ECacheType; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/8/2 + * Company: www.dtstack.com + * @author xuchao + */ + +public abstract class AbsSideTableParser extends AbsTableParser { + + //解析create table的属性==>获取cache信息 + protected void parseCacheProp(SideTableInfo sideTableInfo, Map props){ + if(props.containsKey(SideTableInfo.CACHE_KEY.toLowerCase())){ + String cacheType = MathUtil.getString(props.get(SideTableInfo.CACHE_KEY.toLowerCase())); + if(cacheType == null){ + return; + } + + if(!ECacheType.isValid(cacheType)){ + throw new RuntimeException("can't not support cache type :" + cacheType); + } + + sideTableInfo.setCacheType(cacheType); + if(props.containsKey(SideTableInfo.CACHE_SIZE_KEY.toLowerCase())){ + Integer cacheSize = MathUtil.getIntegerVal(props.get(SideTableInfo.CACHE_SIZE_KEY.toLowerCase())); + if(cacheSize < 0){ + throw new RuntimeException("cache size need > 0."); + } + sideTableInfo.setCacheSize(cacheSize); + } + + if(props.containsKey(SideTableInfo.CACHE_TTLMS_KEY.toLowerCase())){ + Long cacheTTLMS = MathUtil.getLongVal(props.get(SideTableInfo.CACHE_TTLMS_KEY.toLowerCase())); + if(cacheTTLMS < 1000){ + throw new RuntimeException("cache time out need > 1000 ms."); + } + sideTableInfo.setCacheTimeout(cacheTTLMS); + } + + if(props.containsKey(SideTableInfo.PARTITIONED_JOIN_KEY.toLowerCase())){ + Boolean partitionedJoinKey = MathUtil.getBoolean(props.get(SideTableInfo.PARTITIONED_JOIN_KEY.toLowerCase())); + if(partitionedJoinKey){ + sideTableInfo.setPartitionedJoin(true); + } + } + } + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java new file mode 100644 index 000000000..aeb77679c --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java @@ -0,0 +1,69 @@ +/* + * 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 com.dtstack.flink.sql.table; + +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/7/4 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public abstract class AbsSourceParser extends AbsTableParser { + + private static final String VIRTUAL_KEY = "virtualFieldKey"; + + private static final String WATERMARK_KEY = "waterMarkKey"; + + private static Pattern virtualFieldKeyPattern = Pattern.compile("(?i)^(\\S+\\([^\\)]+\\))\\s+AS\\s+(\\w+)$"); + + private static Pattern waterMarkKeyPattern = Pattern.compile("(?i)^\\s*WATERMARK\\s+FOR\\s+(\\S+)\\s+AS\\s+withOffset\\(\\s*(\\S+)\\s*,\\s*(\\d+)\\s*\\)$"); + + static { + keyPatternMap.put(VIRTUAL_KEY, virtualFieldKeyPattern); + keyPatternMap.put(WATERMARK_KEY, waterMarkKeyPattern); + + keyHandlerMap.put(VIRTUAL_KEY, AbsSourceParser::dealVirtualField); + keyHandlerMap.put(WATERMARK_KEY, AbsSourceParser::dealWaterMark); + } + + static void dealVirtualField(Matcher matcher, TableInfo tableInfo){ + SourceTableInfo sourceTableInfo = (SourceTableInfo) tableInfo; + String fieldName = matcher.group(2); + String expression = matcher.group(1); + sourceTableInfo.addVirtualField(fieldName, expression); + } + + static void dealWaterMark(Matcher matcher, TableInfo tableInfo){ + SourceTableInfo sourceTableInfo = (SourceTableInfo) tableInfo; + String eventTimeField = matcher.group(1); + //FIXME 暂时不解析第二个row_time_field参数 + Integer offset = MathUtil.getIntegerVal(matcher.group(3)); + sourceTableInfo.setEventTimeField(eventTimeField); + sourceTableInfo.setMaxOutOrderness(offset); + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java new file mode 100644 index 000000000..84ccc0779 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.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 com.dtstack.flink.sql.table; + +import com.dtstack.flink.sql.util.ClassUtil; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/7/4 + * Company: www.dtstack.com + * @author xuchao + */ + +public abstract class AbsTableParser { + + private static final String PRIMARY_KEY = "primaryKey"; + + private static Pattern primaryKeyPattern = Pattern.compile("(?i)PRIMARY\\s+KEY\\s*\\((.*)\\)"); + + public static Map keyPatternMap = Maps.newHashMap(); + + public static Map keyHandlerMap = Maps.newHashMap(); + + static { + keyPatternMap.put(PRIMARY_KEY, primaryKeyPattern); + keyHandlerMap.put(PRIMARY_KEY, AbsTableParser::dealPrimaryKey); + } + + protected boolean fieldNameNeedsUpperCase() { + return true; + } + + public abstract TableInfo getTableInfo(String tableName, String fieldsInfo, Map props); + + public boolean dealKeyPattern(String fieldRow, TableInfo tableInfo){ + for(Map.Entry keyPattern : keyPatternMap.entrySet()){ + Pattern pattern = keyPattern.getValue(); + String key = keyPattern.getKey(); + Matcher matcher = pattern.matcher(fieldRow); + if(matcher.find()){ + ITableFieldDealHandler handler = keyHandlerMap.get(key); + if(handler == null){ + throw new RuntimeException("parse field [" + fieldRow + "] error."); + } + + handler.dealPrimaryKey(matcher, tableInfo); + return true; + } + } + + return false; + } + + public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ + + String[] fieldRows = DtStringUtil.splitIgnoreQuotaBrackets(fieldsInfo, ","); + for(String fieldRow : fieldRows){ + fieldRow = fieldRow.trim(); + if(fieldNameNeedsUpperCase()) { + fieldRow = fieldRow.toUpperCase(); + } + + boolean isMatcherKey = dealKeyPattern(fieldRow, tableInfo); + + if(isMatcherKey){ + continue; + } + + String[] filedInfoArr = fieldRow.split("\\s+"); + if(filedInfoArr.length < 2){ + throw new RuntimeException(String.format("table [%s] field [%s] format error.", tableInfo.getName(), fieldRow)); + } + + //兼容可能在fieldName中出现空格的情况 + String[] filedNameArr = new String[filedInfoArr.length - 1]; + System.arraycopy(filedInfoArr, 0, filedNameArr, 0, filedInfoArr.length - 1); + String fieldName = String.join(" ", filedNameArr); + String fieldType = filedInfoArr[filedInfoArr.length - 1 ].trim(); + //Class fieldClass = ClassUtil.stringConvertClass(filedInfoArr[1].trim()); + Class fieldClass = ClassUtil.stringConvertClass(fieldType); + + tableInfo.addField(fieldName); + tableInfo.addFieldClass(fieldClass); + tableInfo.addFieldType(fieldType); + } + + tableInfo.finish(); + } + + public static void dealPrimaryKey(Matcher matcher, TableInfo tableInfo){ + String primaryFields = matcher.group(1); + String[] splitArry = primaryFields.split(","); + List primaryKes = Lists.newArrayList(splitArry); + tableInfo.setPrimaryKeys(primaryKes); + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/table/ITableFieldDealHandler.java b/core/src/main/java/com/dtstack/flink/sql/table/ITableFieldDealHandler.java new file mode 100644 index 000000000..db804ea34 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/table/ITableFieldDealHandler.java @@ -0,0 +1,34 @@ +/* + * 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 com.dtstack.flink.sql.table; + +import java.util.regex.Matcher; + +/** + * Reason: + * Date: 2018/7/4 + * Company: www.dtstack.com + * @author xuchao + */ +public interface ITableFieldDealHandler { + + void dealPrimaryKey(Matcher matcher, TableInfo tableInfo); +} diff --git a/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java new file mode 100644 index 000000000..ed342e002 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java @@ -0,0 +1,104 @@ +/* + * 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 com.dtstack.flink.sql.table; + +import org.apache.flink.calcite.shaded.com.google.common.base.Strings; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/6/25 + * Company: www.dtstack.com + * @author xuchao + */ + +public abstract class SourceTableInfo extends TableInfo { + + public static final String SOURCE_SUFFIX = "Source"; + + private String eventTimeField; + + private Integer maxOutOrderness = 10; + + private Map virtualFields = Maps.newHashMap(); + + @Override + public boolean check() { + return true; + } + + public String getEventTimeField() { + return eventTimeField; + } + + public void setEventTimeField(String eventTimeField) { + this.eventTimeField = eventTimeField; + } + + public int getMaxOutOrderness() { + return maxOutOrderness; + } + + public void setMaxOutOrderness(Integer maxOutOrderness) { + if(maxOutOrderness == null){ + return; + } + + this.maxOutOrderness = maxOutOrderness; + } + + public Map getVirtualFields() { + return virtualFields; + } + + public void setVirtualFields(Map virtualFields) { + this.virtualFields = virtualFields; + } + + public void addVirtualField(String fieldName, String expression){ + virtualFields.put(fieldName, expression); + } + + public String getAdaptSelectSql(){ + String fields = String.join(",", getFields()); + String virtualFieldsStr = ""; + + if(virtualFields.size() == 0){ + return null; + } + + for(Map.Entry entry : virtualFields.entrySet()){ + virtualFieldsStr += entry.getValue() +" AS " + entry.getKey() + ","; + } + + if(!Strings.isNullOrEmpty(virtualFieldsStr)){ + fields += "," + virtualFieldsStr.substring(0, virtualFieldsStr.lastIndexOf(",")); + } + + return String.format("select %s from %s", fields, getAdaptName()); + } + + public String getAdaptName(){ + return getName() + "_adapt"; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java new file mode 100644 index 000000000..71bea466f --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java @@ -0,0 +1,130 @@ +/* + * 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 com.dtstack.flink.sql.table; + +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.List; + +/** + * Reason: + * Date: 2018/6/22 + * Company: www.dtstack.com + * @author xuchao + */ + +public abstract class TableInfo { + + public static final String PARALLELISM_KEY = "parallelism"; + + private String name; + + private String type; + + private String[] fields; + + private String[] fieldTypes; + + private Class[] fieldClasses; + + private final List fieldList = Lists.newArrayList(); + + private final List fieldTypeList = Lists.newArrayList(); + + private final List fieldClassList = Lists.newArrayList(); + + private List primaryKeys; + + private Integer parallelism = 1; + + public String[] getFieldTypes() { + return fieldTypes; + } + + public abstract boolean check(); + + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + public String[] getFields() { + return fields; + } + + public Class[] getFieldClasses() { + return fieldClasses; + } + + public List getPrimaryKeys() { + return primaryKeys; + } + + public void setPrimaryKeys(List primaryKeys) { + this.primaryKeys = primaryKeys; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public Integer getParallelism() { + return parallelism; + } + + public void setParallelism(Integer parallelism) { + if(parallelism == null){ + return; + } + + if(parallelism <= 0){ + throw new RuntimeException("Abnormal parameter settings: parallelism > 0"); + } + + this.parallelism = parallelism; + } + + public void addField(String fieldName){ + fieldList.add(fieldName); + } + + public void addFieldClass(Class fieldClass){ + fieldClassList.add(fieldClass); + } + + public void addFieldType(String fieldType){ + fieldTypeList.add(fieldType); + } + + + public void finish(){ + this.fields = fieldList.toArray(new String[fieldList.size()]); + this.fieldClasses = fieldClassList.toArray(new Class[fieldClassList.size()]); + this.fieldTypes = fieldTypeList.toArray(new String[fieldTypeList.size()]); + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java new file mode 100644 index 000000000..73e255340 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java @@ -0,0 +1,121 @@ +/* + * 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 com.dtstack.flink.sql.table; + +import com.dtstack.flink.sql.enums.ETableType; +import com.dtstack.flink.sql.parser.CreateTableParser; +import com.dtstack.flink.sql.side.StreamSideFactory; +import com.dtstack.flink.sql.sink.StreamSinkFactory; +import com.dtstack.flink.sql.source.StreamSourceFactory; +import com.dtstack.flink.sql.util.MathUtil; +import org.apache.flink.calcite.shaded.com.google.common.base.Strings; +import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.collect.Maps; + +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * 解析创建表语句获得具体的表结构信息 + * Date: 2018/6/25 + * Company: www.dtstack.com + * @author xuchao + */ + +public class TableInfoParserFactory { + + private final static String TYPE_KEY = "type"; + + private final static String SIDE_TABLE_SIGN = "(?i)^PERIOD\\s+FOR\\s+SYSTEM_TIME$"; + + private final static Pattern SIDE_PATTERN = Pattern.compile(SIDE_TABLE_SIGN); + + private static Map sourceTableInfoMap = Maps.newConcurrentMap(); + + private static Map targetTableInfoMap = Maps.newConcurrentMap(); + + private static Map sideTableInfoMap = Maps.newConcurrentMap(); + + //加载插件中的解析 + public static TableInfo parseWithTableType(int tableType, CreateTableParser.SqlParserResult parserResult, + String localPluginRoot) throws Exception { + AbsTableParser absTableParser = null; + Map props = parserResult.getPropMap(); + String type = MathUtil.getString(props.get(TYPE_KEY)); + + if(Strings.isNullOrEmpty(type)){ + throw new RuntimeException("create table statement requires property of type"); + } + + if(tableType == ETableType.SOURCE.getType()){ + boolean isSideTable = checkIsSideTable(parserResult.getFieldsInfoStr()); + + if(!isSideTable){ + absTableParser = sourceTableInfoMap.get(type); + if(absTableParser == null){ + absTableParser = StreamSourceFactory.getSqlParser(type, localPluginRoot); + sourceTableInfoMap.put(type, absTableParser); + } + }else{ + absTableParser = sideTableInfoMap.get(type); + if(absTableParser == null){ + absTableParser = StreamSideFactory.getSqlParser(type, localPluginRoot); + sideTableInfoMap.put(type, absTableParser); + } + } + + }else if(tableType == ETableType.SINK.getType()){ + absTableParser = targetTableInfoMap.get(type); + if(absTableParser == null){ + absTableParser = StreamSinkFactory.getSqlParser(type, localPluginRoot); + targetTableInfoMap.put(type, absTableParser); + } + } + + if(absTableParser == null){ + throw new RuntimeException(String.format("not support %s type of table", type)); + } + + Map prop = Maps.newHashMap(); + + //屏蔽大小写 + parserResult.getPropMap().forEach((key,val) -> prop.put(key.toLowerCase(), val)); + + return absTableParser.getTableInfo(parserResult.getTableName(), parserResult.getFieldsInfoStr(), prop); + } + + /** + * 如果表字段中包含PERIOD FOR SYSTEM_TIME则认为是一个维表 + * @param tableField + * @return + */ + private static boolean checkIsSideTable(String tableField){ + String[] fieldInfos = tableField.split(","); + for(String field : fieldInfos){ + Matcher matcher = SIDE_PATTERN.matcher(field.trim()); + if(matcher.find()){ + return true; + } + } + + return false; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TargetTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/TargetTableInfo.java new file mode 100644 index 000000000..a9a846707 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/table/TargetTableInfo.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 com.dtstack.flink.sql.table; + +/** + * Reason: + * Date: 2018/6/25 + * Company: www.dtstack.com + * @author xuchao + */ + +public abstract class TargetTableInfo extends TableInfo { + + public static final String TARGET_SUFFIX = "Sink"; +} diff --git a/core/src/main/java/com/dtstack/flink/sql/threadFactory/DTThreadFactory.java b/core/src/main/java/com/dtstack/flink/sql/threadFactory/DTThreadFactory.java new file mode 100644 index 000000000..1c16581ef --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/threadFactory/DTThreadFactory.java @@ -0,0 +1,59 @@ +/* + * 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 com.dtstack.flink.sql.threadFactory; + +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * company: www.dtstack.com + * author: sishu.yss + * create: 2018/09/12 + */ +public class DTThreadFactory implements ThreadFactory { + private final static AtomicInteger POOL_NUMBER = new AtomicInteger(1); + private final static AtomicInteger THREAD_NUMBER = new AtomicInteger(1); + private final ThreadGroup group; + private final String namePrefix; + + public DTThreadFactory(String factoryName) { + SecurityManager s = System.getSecurityManager(); + group = (s != null) ? s.getThreadGroup() : + Thread.currentThread().getThreadGroup(); + namePrefix = factoryName + "-pool-" + + POOL_NUMBER.getAndIncrement() + + "-thread-"; + } + + @Override + public Thread newThread(Runnable r) { + Thread t = new Thread(group, r, + namePrefix + THREAD_NUMBER.getAndIncrement(), + 0); + if (t.isDaemon()) { + t.setDaemon(false); + } + if (t.getPriority() != Thread.NORM_PRIORITY) { + t.setPriority(Thread.NORM_PRIORITY); + } + return t; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ByteUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/ByteUtils.java new file mode 100644 index 000000000..05792e044 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/util/ByteUtils.java @@ -0,0 +1,174 @@ +/* + * 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 com.dtstack.flink.sql.util; + +import org.apache.commons.io.Charsets; + +import java.nio.ByteBuffer; + +/** + * byte 转换工具 + * Date: 2018/8/28 + * Company: www.dtstack.com + * @author xuchao + */ + +public class ByteUtils { + + public static boolean toBoolean(final byte [] b) { + if (b.length != 1) { + throw new IllegalArgumentException("Array has wrong size: " + b.length); + } + return b[0] != (byte) 0; + } + + + public static String byteToString(byte[] bytes){ + return new String(bytes, Charsets.UTF_8); + } + + public static byte[] shortToByte4(short i) { + byte[] targets = new byte[2]; + targets[1] = (byte) (i & 0xFF); + targets[0] = (byte) (i >> 8 & 0xFF); + return targets; + } + + + public static Short byte2ToShort(byte[] bytes) { + + if(bytes.length != 2){ + throw new RuntimeException("byte2ToUnsignedShort input bytes length need == 2"); + } + + short high = (short) (bytes[0] & 0xFF); + short low = (short) (bytes[1] & 0xFF); + return (short)(high << 8 | low); + } + + + /*将INT类型转化为10进制byte数组(占4字节)*/ + public static byte[] int2Bytes(int num) { + byte[] byteNum = new byte[4]; + for (int ix = 0; ix < 4; ++ix) { + int offset = 32 - (ix + 1) * 8; + byteNum[ix] = (byte) ((num >> offset) & 0xff); + } + return byteNum; + } + + + /** + * byte数组转换为int整数 + * + * @param byteNum byte数组 + * @return int整数 + */ + public static int byte4ToInt(byte[] byteNum) { + + if(byteNum.length != 4){ + throw new RuntimeException("byte4ToInt input bytes length need == 4"); + } + + int num = 0; + for (int ix = 0; ix < 4; ++ix) { + num <<= 8; + num |= (byteNum[ix] & 0xff); + } + return num; + + } + + /*将长整形转化为byte数组*/ + public static byte[] long2Bytes(long num) { + byte[] byteNum = new byte[8]; + for (int ix = 0; ix < 8; ++ix) { + int offset = 64 - (ix + 1) * 8; + byteNum[ix] = (byte) ((num >> offset) & 0xff); + } + return byteNum; + } + + /*将byte数组(长度为8)转化为长整形*/ + public static long bytes2Long(byte[] byteNum) { + + if(byteNum.length != 8){ + throw new RuntimeException("bytes2Long input bytes length need == 8"); + } + + long num = 0; + for (int ix = 0; ix < 8; ++ix) { + num <<= 8; + num |= (byteNum[ix] & 0xff); + } + return num; + } + + public static byte bytes2Byte(byte[] byteNum){ + if(byteNum.length != 8){ + throw new RuntimeException("bytes2Byte input bytes length need == 1"); + } + + return byteNum[0]; + } + + /**将float转化为byte数组,占用4个字节**/ + public static byte [] float2ByteArray (float value) + { + return ByteBuffer.allocate(4).putFloat(value).array(); + } + + /** + * 将10进制byte数组转化为Float + * + * @param b 字节(至少4个字节) + * @return + */ + public static float bytes2Float(byte[] b) { + int l; + l = b[0]; + l &= 0xff; + l |= ((long) b[1] << 8); + l &= 0xffff; + l |= ((long) b[2] << 16); + l &= 0xffffff; + l |= ((long) b[3] << 24); + return Float.intBitsToFloat(l); + } + + public static byte[] double2Bytes(double d) { + long value = Double.doubleToRawLongBits(d); + byte[] byteRet = new byte[8]; + for (int i = 0; i < 8; i++) { + byteRet[i] = (byte) ((value >> 8 * i) & 0xff); + } + return byteRet; + } + + public static double bytes2Double(byte[] arr) { + long value = 0; + for (int i = 0; i < 8; i++) { + value |= ((long) (arr[i] & 0xff)) << (8 * i); + } + return Double.longBitsToDouble(value); + } + +} diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java new file mode 100644 index 000000000..008913f8f --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java @@ -0,0 +1,178 @@ +/* + * 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 com.dtstack.flink.sql.util; + +import java.sql.Date; +import java.sql.Timestamp; + +/** + * Reason: TODO ADD REASON(可选) + * Date: 2017年03月10日 下午1:16:37 + * Company: www.dtstack.com + * @author sishu.yss + */ +public class ClassUtil { + + public static Class stringConvertClass(String str) { + switch (str.toLowerCase()) { + case "boolean": + return Boolean.class; + case "int": + return Integer.class; + + case "bigint": + return Long.class; + + case "tinyint": + case "byte": + return Byte.class; + + case "short": + case "smallint": + return Short.class; + + case "char": + case "varchar": + case "string": + return String.class; + + case "float": + return Float.class; + + case "double": + return Double.class; + + case "date": + return Date.class; + + case "timestamp": + return Timestamp.class; + + } + + throw new RuntimeException("不支持 " + str + " 类型"); + } + + public static Object convertType(Object field, String fromType, String toType) { + fromType = fromType.toUpperCase(); + toType = toType.toUpperCase(); + String rowData = field.toString(); + + switch(toType) { + case "TINYINT": + return Byte.valueOf(rowData); + case "SMALLINT": + return Short.valueOf(rowData); + case "INT": + return Integer.valueOf(rowData); + case "BIGINT": + return Long.valueOf(rowData); + case "FLOAT": + return Float.valueOf(rowData); + case "DOUBLE": + return Double.valueOf(rowData); + case "STRING": + return rowData; + case "BOOLEAN": + return Boolean.valueOf(rowData); + case "DATE": + return DateUtil.columnToDate(field); + case "TIMESTAMP": + Date d = DateUtil.columnToDate(field); + return new Timestamp(d.getTime()); + default: + throw new RuntimeException("Can't convert from " + fromType + " to " + toType); + } + + } + + public static String getTypeFromClass(Class clz) { + + if(clz == Byte.class){ + return "TINYINT"; + } + else if(clz == Short.class){ + return "SMALLINT"; + } + else if(clz == Integer.class){ + return "INT"; + } + else if(clz == Long.class){ + return "BIGINT"; + } + else if(clz == String.class){ + return "STRING"; + } + else if(clz == Float.class){ + return "FLOAT"; + } + else if(clz == Double.class){ + return "DOUBLE"; + } + else if(clz == Date.class){ + return "DATE"; + } + else if(clz == Timestamp.class){ + return "TIMESTAMP"; + } + else if(clz == Boolean.class){ + return "BOOLEAN"; + } + throw new IllegalArgumentException("Unsupported data type: " + clz.getName()); + + } + + public static String getTypeFromClassName(String clzName) { + + if(clzName.equals(Byte.class.getName())){ + return "TINYINT"; + } + else if(clzName.equals(Short.class.getName())){ + return "SMALLINT"; + } + else if(clzName.equals(Integer.class.getName())){ + return "INT"; + } + else if(clzName.equals(Long.class.getName())){ + return "BIGINT"; + } + else if(clzName.equals(String.class.getName())){ + return "STRING"; + } + else if(clzName.equals(Float.class.getName())){ + return "FLOAT"; + } + else if(clzName.equals(Double.class.getName())){ + return "DOUBLE"; + } + else if(clzName.equals(Date.class.getName())){ + return "DATE"; + } + else if(clzName.equals(Timestamp.class.getName())){ + return "TIMESTAMP"; + } + else if(clzName.equals(Boolean.class.getName())){ + return "BOOLEAN"; + } + throw new IllegalArgumentException("Unsupported data type: " + clzName); + } + +} diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ConfigConstrant.java b/core/src/main/java/com/dtstack/flink/sql/util/ConfigConstrant.java new file mode 100644 index 000000000..7d37d4003 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/util/ConfigConstrant.java @@ -0,0 +1,53 @@ +/* + * 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 com.dtstack.flink.sql.util; + + +/** + * + * @author sishu.yss + * + */ +public class ConfigConstrant { + + public static final String FLINK_CHECKPOINT_INTERVAL_KEY = "sql.checkpoint.interval"; + + public static final String FLINK_CHECKPOINT_MODE_KEY = "sql.checkpoint.mode"; + + public static final String FLINK_CHECKPOINT_TIMEOUT_KEY = "sql.checkpoint.timeout"; + + public static final String FLINK_MAXCONCURRENTCHECKPOINTS_KEY = "sql.max.concurrent.checkpoints"; + + public static final String FLINK_CHECKPOINT_CLEANUPMODE_KEY = "sql.checkpoint.cleanup.mode"; + + public static final String FLINK_CHECKPOINT_DATAURI_KEY = "flinkCheckpointDataURI"; + + public static final String SQL_ENV_PARALLELISM = "sql.env.parallelism"; + + public static final String SQL_MAX_ENV_PARALLELISM = "sql.max.env.parallelism"; + + public static final String MR_JOB_PARALLELISM = "mr.job.parallelism"; + + public static final String SQL_BUFFER_TIMEOUT_MILLIS = "sql.buffer.timeout.millis"; + + public static final String FLINK_TIME_CHARACTERISTIC_KEY = "time.characteristic"; + +} diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DateUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DateUtil.java new file mode 100644 index 000000000..5bfa2f203 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/util/DateUtil.java @@ -0,0 +1,773 @@ +/* + * 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 com.dtstack.flink.sql.util; + +import java.sql.Timestamp; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Calendar; +import java.util.Date; +import java.util.Locale; +import java.util.SimpleTimeZone; + + +/** + * + * Reason: TODO ADD REASON(可选) + * Date: 2017年03月10日 下午1:16:37 + * Company: www.dtstack.com + * @author sishu.yss + * + */ +public class DateUtil { + + static final String timeZone = "GMT+8"; + static final String datetimeFormat = "yyyy-MM-dd HH:mm:ss"; + static final String dateFormat = "yyyy-MM-dd"; + static final String timeFormat = "HH:mm:ss"; + static final SimpleDateFormat datetimeFormatter = new SimpleDateFormat(datetimeFormat); + static final SimpleDateFormat dateFormatter = new SimpleDateFormat(dateFormat); + static final SimpleDateFormat timeFormatter = new SimpleDateFormat(timeFormat); + + public static java.sql.Date columnToDate(Object column) { + if(column instanceof String) { + return new java.sql.Date(stringToDate((String)column).getTime()); + } else if (column instanceof Integer) { + Integer rawData = (Integer) column; + return new java.sql.Date(rawData.longValue()); + } else if (column instanceof Long) { + Long rawData = (Long) column; + return new java.sql.Date(rawData.longValue()); + } else if (column instanceof java.sql.Date) { + return (java.sql.Date) column; + } else if(column instanceof Timestamp) { + Timestamp ts = (Timestamp) column; + return new java.sql.Date(ts.getTime()); + } + throw new IllegalArgumentException("Can't convert " + column.getClass().getName() + " to Date"); + } + + public static Date stringToDate(String strDate) { + if(strDate == null){ + return null; + } + try { + return datetimeFormatter.parse(strDate); + } catch (ParseException ignored) { + } + + try { + return dateFormatter.parse(strDate); + } catch (ParseException ignored) { + } + + try { + return timeFormatter.parse(strDate); + } catch (ParseException ignored) { + } + + throw new RuntimeException("can't parse date"); + } + + /** + * + * + * @param day Long 时间 + * @return long + */ + public static long getTodayStart(long day) { + long firstDay = 0L; + Calendar cal = Calendar.getInstance(); + if (("" + day).length() > 10) { + cal.setTime(new Date(day)); + } else { + cal.setTime(new Date(day * 1000)); + } + cal.set(Calendar.HOUR_OF_DAY, 0); + cal.set(Calendar.MINUTE, 0); + cal.set(Calendar.SECOND, 0); + cal.set(Calendar.MILLISECOND, 0); + firstDay = cal.getTimeInMillis() / 1000; + return firstDay; + } + + /** + * + * @param day Long 时间 + * @param scope + * @return + */ + public static long getTodayStart(long day,String scope) { + if(scope.equals("MS")){ + return getTodayStart(day)*1000; + }else if(scope.equals("S")){ + return getTodayStart(day); + }else{ + return getTodayStart(day); + } + } + + /** + * + * @param day Long 时间 + * @return long + */ + public static long getNextDayStart(long day) { + long daySpanMill = 86400000L; + long nextDay = 0L; + Calendar cal = Calendar.getInstance(); + if (("" + day).length() > 10) { + cal.setTime(new Date(day)); + } else { + cal.setTime(new Date(day * 1000)); + } + cal.set(Calendar.HOUR_OF_DAY, 0); + cal.set(Calendar.MINUTE, 0); + cal.set(Calendar.SECOND, 0); + cal.set(Calendar.MILLISECOND, 0); + nextDay = (cal.getTimeInMillis() + daySpanMill) / 1000; + return nextDay; + } + + /** + * + * @param day Long 时间 + * @param scope String 级别
"MS":毫秒级
"S":秒级 + * @return + */ + public static long getNextDayStart(long day,String scope) { + if(scope.equals("MS")){ + return getNextDayStart(day)*1000; + }else if(scope.equals("S")){ + return getNextDayStart(day); + }else{ + return getNextDayStart(day); + } + } + + + /** + * + * @param day + * @return + */ + public static long getMonthFirst(long day) { + long firstDay = 0L; + Calendar cal = Calendar.getInstance(); + cal.setTime(new Date(day * 1000)); + cal.set(Calendar.DAY_OF_MONTH, 1); + cal.set(Calendar.HOUR_OF_DAY, 0); + cal.set(Calendar.MINUTE, 0); + cal.set(Calendar.SECOND, 0); + cal.set(Calendar.MILLISECOND, 0); + firstDay = cal.getTimeInMillis() / 1000; + return firstDay; + } + + /** + * @param day + * @return + */ + public static int getMonth(long day) { + Calendar cal = Calendar.getInstance(); + cal.setTime(new Date(day * 1000)); + return cal.get(Calendar.MONTH) + 1; + } + + /** + * + * @author yumo.lck + */ + public static int getYear(long day) { + Calendar cal = Calendar.getInstance(); + cal.setTime(new Date(day * 1000)); + return cal.get(Calendar.YEAR); + } + + /** + * + * @param day + * @return + */ + public static long getWeekFirst(long day) { + long firstDay = 0L; + Calendar cal = Calendar.getInstance(); + cal.setTime(new Date(day * 1000)); + cal.setFirstDayOfWeek(Calendar.MONDAY); + cal.set(Calendar.DAY_OF_WEEK, Calendar.MONDAY); + cal.set(Calendar.HOUR_OF_DAY, 0); + cal.set(Calendar.MINUTE, 0); + cal.set(Calendar.SECOND, 0); + cal.set(Calendar.MILLISECOND, 0); + firstDay = cal.getTimeInMillis() / 1000; + return firstDay; + } + + /** + * 根据某个日期时间戳秒值,获取所在周在一年中是第几周. + * + * @param day + * @return + */ + public static int getWeekOfYear(long day) { + Calendar cal = Calendar.getInstance(); + cal.setTime(new Date(day * 1000)); + return cal.get(Calendar.WEEK_OF_YEAR); + } + + /** + * + * @param day + * @param inFormat + * @param outFormat + * @return String + * @throws ParseException + */ + public static String getYesterdayByString(String day, String inFormat, String outFormat){ + try { + SimpleDateFormat sdf = new SimpleDateFormat(inFormat); + Date date = sdf.parse(day); + Calendar calendar = Calendar.getInstance(); + calendar.setTime(date); + int calendarDay = calendar.get(Calendar.DATE); + calendar.set(Calendar.DATE, calendarDay - 1); + String dayBefore = new SimpleDateFormat(outFormat).format(calendar.getTime()); + return dayBefore; + } catch (ParseException e) { + return null; + } + } + + /** + * + * @param day + * @param inFormat + * @param outFormat + * @return String + * @throws ParseException + */ + public static String getTomorrowByString(String day, String inFormat, String outFormat) throws ParseException { + SimpleDateFormat sdf = new SimpleDateFormat(inFormat); + Date date = sdf.parse(day); + Calendar calendar = Calendar.getInstance(); + calendar.setTime(date); + int calendarDay = calendar.get(Calendar.DATE); + calendar.set(Calendar.DATE, calendarDay + 1); + String dayBefore = new SimpleDateFormat(outFormat).format(calendar.getTime()); + return dayBefore; + } + + /** + * + * @param date + * @return Date + * @throws ParseException + */ + public static Date getTomorrowByDate(Date date) throws ParseException { + Calendar calendar = Calendar.getInstance(); + calendar.setTime(date); + int calendarDay = calendar.get(Calendar.DATE); + calendar.set(Calendar.DATE, calendarDay + 1); + return calendar.getTime(); + } + + /** + * + * @param day + * @param inFormat + * @param outFormat + * @return String + * @throws ParseException + */ + public static String get30DaysBeforeByString(String day, String inFormat, String outFormat) throws ParseException { + SimpleDateFormat sdf = new SimpleDateFormat(inFormat); + Date date = sdf.parse(day); + Calendar calendar = Calendar.getInstance(); + calendar.setTime(date); + int calendarDay = calendar.get(Calendar.DATE); + calendar.set(Calendar.DATE, calendarDay - 30); + return new SimpleDateFormat(outFormat).format(calendar.getTime()); + } + + /** + * + * @param day + * @param inFormat + * @param outFormat + * @return String + * @throws ParseException + */ + public static String get30DaysLaterByString(String day, String inFormat, String outFormat) throws ParseException { + SimpleDateFormat sdf = new SimpleDateFormat(inFormat); + Date date = sdf.parse(day); + Calendar calendar = Calendar.getInstance(); + calendar.setTime(date); + int calendarDay = calendar.get(Calendar.DATE); + calendar.set(Calendar.DATE, calendarDay + 30); + String dayBefore = new SimpleDateFormat(outFormat).format(calendar.getTime()); + return dayBefore; + } + + + /** + * + * @param day + * @param inFormat + * @param outFormat + * @return String + * @throws ParseException + */ + public static String getDateStrTOFormat(String day, String inFormat, String outFormat) throws ParseException { + SimpleDateFormat sdf = new SimpleDateFormat(inFormat); + Date date = sdf.parse(day); + Calendar calendar = Calendar.getInstance(); + calendar.setTime(date); + String dayBefore = new SimpleDateFormat(outFormat).format(calendar.getTime()); + return dayBefore; + } + + public static long getDateMillTOFormat(String day, String inFormat) throws ParseException { + SimpleDateFormat sdf = new SimpleDateFormat(inFormat); + Date date = sdf.parse(day); + Calendar calendar = Calendar.getInstance(); + calendar.setTime(date); + return calendar.getTimeInMillis()/1000; + } + + /** + * + * @author sishu.yss + * @param year + * @param month + * @return + */ + public static long getFirstDay4Month(int year, int month) { + long firstDay = 0L; + Calendar cal = Calendar.getInstance(); + cal.set(Calendar.YEAR, year); + cal.set(Calendar.MONTH, month - 1); + cal.set(Calendar.DAY_OF_MONTH, 1); + cal.set(Calendar.HOUR_OF_DAY, 0); + cal.set(Calendar.MINUTE, 0); + cal.set(Calendar.SECOND, 0); + cal.set(Calendar.MILLISECOND, 0); + firstDay = cal.getTimeInMillis() / 1000; + return firstDay; + } + + /** + * + * @author yumo.lck + * @param year + * @param month + * @return + */ + public static long getLastDay4Month(int year, int month) { + long lastDay = 0L; + Calendar cal = Calendar.getInstance(); + cal.set(Calendar.YEAR, year); + cal.set(Calendar.MONTH, month); + //1 represents a zero next month, can be seen as the end of the first day of the month most one day, but the data table on the last day of the zero point on the line + cal.set(Calendar.DAY_OF_MONTH, 0); + cal.set(Calendar.HOUR_OF_DAY, 0); + cal.set(Calendar.MINUTE, 0); + cal.set(Calendar.SECOND, 0); + cal.set(Calendar.MILLISECOND, 0); + lastDay = cal.getTimeInMillis() / 1000; + return lastDay; + } + + /** + * + * @author yumo.lck + * @param chooseFirstDay + */ + + public static long getBeforeMonthDay(long day, boolean chooseFirstDay) { + long chooseDay = 0L; + int currentMonth = getMonth(day); + int currentYear = getYear(day); + if (currentMonth > 1) { + currentMonth--; + } else { + currentYear--; + currentMonth = 12; + } + if (chooseFirstDay) { + chooseDay = getFirstDay4Month(currentYear, currentMonth); + return chooseDay; + } else { + chooseDay = getLastDay4Month(currentYear, currentMonth); + return chooseDay; + } + + } + + /** + * @return long + */ + public static long getMillByOneDay() { + Calendar cal = Calendar.getInstance(); + cal.set(Calendar.HOUR_OF_DAY, 0); + cal.set(Calendar.MINUTE, 0); + cal.set(Calendar.SECOND, 0); + cal.set(Calendar.MILLISECOND, 0); + return cal.getTimeInMillis() / 1000; + } + + /** + * + * @return long + */ + public static long getMillByYesDay() { + Calendar cal = Calendar.getInstance(); + cal.set(Calendar.DATE, cal.get(Calendar.DATE) - 1); + cal.set(Calendar.HOUR_OF_DAY, 0); + cal.set(Calendar.MINUTE, 0); + cal.set(Calendar.SECOND, 0); + cal.set(Calendar.MILLISECOND, 0); + return cal.getTimeInMillis() / 1000; + } + + /** + * + * @return + */ + public static long getMillByLastWeekDay() { + Calendar cal = Calendar.getInstance(); + cal.set(Calendar.DATE, cal.get(Calendar.DATE) - 7); + cal.set(Calendar.HOUR_OF_DAY, 0); + cal.set(Calendar.MINUTE, 0); + cal.set(Calendar.SECOND, 0); + cal.set(Calendar.MILLISECOND, 0); + return cal.getTimeInMillis() / 1000; + } + + /** + * @return long + */ + public static long getMillByDay(int severalDays,String condition) { + int dateT=0; + Calendar cal = Calendar.getInstance(); + if(condition==null){ + return getMillToDay(cal,dateT); + } + if(condition.equals("-")){ + dateT = (cal.get(Calendar.DATE) - severalDays); + return getMillToDay(cal,dateT); + } + if(condition.equals("+")){ + dateT = (cal.get(Calendar.DATE) + severalDays); + return getMillToDay(cal,dateT); + } + return getMillToDay(cal,dateT); + } + + /** + * @return long + */ + public static long getStampByDay(int severalDays,String condition) { + int dateT=0; + Calendar cal = Calendar.getInstance(); + if(condition==null){ + return getStampToDay(cal,dateT); + } + if(condition.equals("-")){ + dateT = (cal.get(Calendar.DATE) - severalDays); + return getStampToDay(cal,dateT); + } + if(condition.equals("+")){ + dateT = (cal.get(Calendar.DATE) + severalDays); + return getStampToDay(cal,dateT); + } + return getStampToDay(cal,dateT); + } + /** + * @return long + */ + public static long getMillByDay(){ + return getMillByDay(0,null); + } + + /** + * @param cal Calendar + * @param dateT Integer + * @return long + */ + public static long getMillToDay(Calendar cal,int dateT){ + if(dateT!=0){ + cal.set(Calendar.DATE, dateT); + } + cal.set(Calendar.HOUR_OF_DAY, 0); + cal.set(Calendar.MINUTE, 0); + cal.set(Calendar.SECOND, 0); + cal.set(Calendar.MILLISECOND, 0); + return cal.getTimeInMillis()/1000; + } + + /** + * @param cal Calendar + * @param dateT Integer + * @return long + */ + public static long getStampToDay(Calendar cal,int dateT){ + if(dateT!=0){ + cal.set(Calendar.DATE, dateT); + } + return cal.getTimeInMillis(); + } + + public static String getToday() { + Calendar cal = Calendar.getInstance(); + return cal.get(1) + "年" + cal.get(2) + "月" + cal.get(3) + "日"; + } + + /** + * @param day + * @return format time + */ + public static String getDate(long day, String format) { + Calendar cal = Calendar.getInstance(); + if (("" + day).length() > 10) { + cal.setTime(new Date(day)); + } else { + cal.setTime(new Date(day * 1000)); + } + SimpleDateFormat sf = new SimpleDateFormat(format); + return sf.format(cal.getTime()); + } + + /** + * + * @param date + * @return + */ + public static String getDate(Date date, String format) { + SimpleDateFormat sf = new SimpleDateFormat(format); + return sf.format(date); + } + + + /** + * + * @param day + * @param format + * @return long + * @throws ParseException + */ + public static long stringToLong(String day, String format) throws ParseException { + SimpleDateFormat dateFormat = new SimpleDateFormat(format); + long Date = dateFormat.parse(day).getTime(); + return Date; + } + + /** + * @param day + * @param format + * @return Date + * @throws ParseException + */ + public static Date stringToDate(String day, String format) { + try { + SimpleDateFormat dateFormat = new SimpleDateFormat(format); + Date Date = dateFormat.parse(day); + return Date; + } catch (ParseException e) { + return new Date(); + } + } + + + /** + * long型时间戳转为String型 + * + * @param day 秒 + * @return 格式化后的日期 + * @throws ParseException + */ + public static String longToString(long day, String format) throws ParseException { + if (("" + day).length() <= 10){ + day=day*1000; + } + SimpleDateFormat dateFormat = new SimpleDateFormat(format); + String Date = dateFormat.format(day); + return Date; + } + + /** + * + * @param day 秒 + * @param minusDay 需要减掉的天数 + * @return 秒 + */ + public static int getMinusDate(int day, int minusDay) { + Calendar cal = Calendar.getInstance(); + cal.setTime(new Date(day * 1000)); + cal.set(Calendar.DATE, cal.get(Calendar.DATE) - minusDay); + cal.set(Calendar.HOUR_OF_DAY, 0); + cal.set(Calendar.MINUTE, 0); + cal.set(Calendar.SECOND, 0); + cal.set(Calendar.MILLISECOND, 0); + return (int) cal.getTimeInMillis() / 1000; + } + + /** + * + * @return long + */ + public static long getMillByNow() { + Calendar cal = Calendar.getInstance(); + cal.setTime(new Date()); + return cal.getTimeInMillis(); + } + + public static int getWeeksBetweenTwoDates(long startDay, long endDay) { + int week = getWeekOfYear(endDay) - getWeekOfYear(startDay) + 1; + if(week<1){ + week = getWeekOfYear(endDay) + getMaxWeekOfYear(startDay) - getWeekOfYear(startDay) + 1; + } + return week; + } + + public static int getMaxWeekOfYear(long startDay) { + Calendar cal = Calendar.getInstance(); + cal.setTime(new Date(startDay * 1000)); + return cal.getMaximum(Calendar.WEEK_OF_YEAR); + } + + public static int getMonthsBetweenTwoDates(long startDay, long endDay) { + int month = DateUtil.getMonth(endDay) - DateUtil.getMonth(startDay) + 1; + if(month<1){ + month = getMonth(endDay) + 12 - getMonth(startDay) +1; + } + return month; + } + + public static Date parseDate(String dateStr, String pattern){ + SimpleDateFormat sdf = new SimpleDateFormat(); + sdf.applyPattern(pattern); + try { + return sdf.parse(dateStr); + } catch (ParseException e) { + return null; + } + } + + /** + * + * @param time Long 时间 + * @return long + */ + public static long getMinuteStart(long time) { + long firstDay = 0L; + Calendar cal = Calendar.getInstance(); + if (("" + time).length() > 10) { + cal.setTime(new Date(time)); + } else { + cal.setTime(new Date(time * 1000)); + } + cal.set(Calendar.SECOND, 0); + cal.set(Calendar.MILLISECOND, 0); + firstDay = cal.getTimeInMillis() / 1000; + return firstDay; + } + + /** + * @param time Long + * @return long + */ + public static long getHourStart(long time) { + long firstDay = 0L; + Calendar cal = Calendar.getInstance(); + if (("" + time).length() > 10) { + cal.setTime(new Date(time)); + } else { + cal.setTime(new Date(time * 1000)); + } + cal.set(Calendar.SECOND, 0); + cal.set(Calendar.MILLISECOND, 0); + cal.set(Calendar.MINUTE, 0); + firstDay = cal.getTimeInMillis() / 1000; + return firstDay; + } + + /** + * @param time + * @return Date + */ + public static Date getDateByLong(long time){ + Date date = new Date(); + date.setTime(time); + return date; + } + + + public static Date parseDate(String dateStr, String pattern, Locale locale){ + SimpleDateFormat df = new SimpleDateFormat( + pattern, locale); + + df.setTimeZone(new SimpleTimeZone(0, "GMT")); + try { + return df.parse(dateStr); + } catch (ParseException e) { + return null; + } + } + + public static String getDate(Date date, String format, Locale locale) { + SimpleDateFormat df = new SimpleDateFormat( + format, locale); + df.setTimeZone(new SimpleTimeZone(0, "GMT")); + return df.format(date); + } + + public static java.sql.Timestamp columnToTimestamp(Object column) { + if (column == null) { + return null; + } else if(column instanceof String) { + return new java.sql.Timestamp(stringToDate((String)column).getTime()); + } else if (column instanceof Integer) { + Integer rawData = (Integer) column; + return new java.sql.Timestamp(rawData.longValue()); + } else if (column instanceof Long) { + Long rawData = (Long) column; + return new java.sql.Timestamp(rawData.longValue()); + } else if (column instanceof java.sql.Date) { + return (java.sql.Timestamp) column; + } else if(column instanceof Timestamp) { + return (Timestamp) column; + } else if(column instanceof Date) { + Date d = (Date)column; + return new java.sql.Timestamp(d.getTime()); + } + + throw new IllegalArgumentException("Can't convert " + column.getClass().getName() + " to Date"); + } + + public static String dateToString(Date date) { + return dateFormatter.format(date); + } + + public static String timestampToString(Date date) { + return datetimeFormatter.format(date); + } + +} diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java new file mode 100644 index 000000000..b54c3479f --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.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 com.dtstack.flink.sql.util; + +import com.dtstack.flink.sql.enums.ColumnType; + +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.List; + +/** + * Reason: + * Date: 2018/6/22 + * Company: www.dtstack.com + * @author xuchao + */ + +public class DtStringUtil { + + /** + * 根据指定分隔符分割字符串---忽略在引号里面的分隔符 + * @param str + * @param delimiter + * @return + */ + public static List splitIgnoreQuota(String str, char delimiter){ + List tokensList = new ArrayList<>(); + boolean inQuotes = false; + boolean inSingleQuotes = false; + StringBuilder b = new StringBuilder(); + for (char c : str.toCharArray()) { + if(c == delimiter){ + if (inQuotes) { + b.append(c); + } else if(inSingleQuotes){ + b.append(c); + }else { + tokensList.add(b.toString()); + b = new StringBuilder(); + } + }else if(c == '\"'){ + inQuotes = !inQuotes; + b.append(c); + }else if(c == '\''){ + inSingleQuotes = !inSingleQuotes; + b.append(c); + }else{ + b.append(c); + } + } + + tokensList.add(b.toString()); + + return tokensList; + } + + /*** + * 根据指定分隔符分割字符串---忽略在引号 和 括号 里面的分隔符 + * @param str + * @param delimter + * @return + */ + public static String[] splitIgnoreQuotaBrackets(String str, String delimter){ + String splitPatternStr = delimter + "(?![^()]*+\\))(?![^{}]*+})(?![^\\[\\]]*+\\])(?=(?:[^\"]|\"[^\"]*\")*$)"; + return str.split(splitPatternStr); + } + + public static String replaceIgnoreQuota(String str, String oriStr, String replaceStr){ + String splitPatternStr = oriStr + "(?=(?:[^\"]*\"[^\"]*\")*[^\"]*$)(?=(?:[^']*'[^']*')*[^']*$)"; + return str.replaceAll(splitPatternStr, replaceStr); + } + + + public static String col2string(Object column, String type) { + String rowData = column.toString(); + ColumnType columnType = ColumnType.valueOf(type.toUpperCase()); + Object result = null; + switch (columnType) { + case TINYINT: + result = Byte.valueOf(rowData); + break; + case SMALLINT: + result = Short.valueOf(rowData); + break; + case INT: + result = Integer.valueOf(rowData); + break; + case BIGINT: + result = Long.valueOf(rowData); + break; + case FLOAT: + result = Float.valueOf(rowData); + break; + case DOUBLE: + result = Double.valueOf(rowData); + break; + case DECIMAL: + result = new BigDecimal(rowData); + break; + case STRING: + case VARCHAR: + case CHAR: + result = rowData; + break; + case BOOLEAN: + result = Boolean.valueOf(rowData); + break; + case DATE: + result = DateUtil.dateToString((java.util.Date)column); + break; + case TIMESTAMP: + result = DateUtil.timestampToString((java.util.Date)column); + break; + default: + throw new IllegalArgumentException(); + } + return result.toString(); + } + + + +} diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java new file mode 100644 index 000000000..b8940ffaf --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java @@ -0,0 +1,286 @@ +/* + * 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 com.dtstack.flink.sql.util; + + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.runtime.state.filesystem.FsStateBackend; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.environment.CheckpointConfig; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.java.BatchTableEnvironment; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.TableFunction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.List; +import java.util.Properties; + +/** + * Reason: + * Date: 2017/2/21 + * Company: www.dtstack.com + * @author xuchao + */ + +public class FlinkUtil { + + private static final Logger logger = LoggerFactory.getLogger(FlinkUtil.class); + + /** + * 开启checkpoint + * @param env + * @throws IOException + */ + public static void openCheckpoint(StreamExecutionEnvironment env, Properties properties) throws IOException { + + if(properties == null){ + return; + } + + //设置了时间间隔才表明开启了checkpoint + if(properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_INTERVAL_KEY) == null){ + return; + }else{ + Long interval = Long.valueOf(properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_INTERVAL_KEY)); + //start checkpoint every ${interval} + env.enableCheckpointing(interval); + } + + String checkMode = properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_MODE_KEY); + if(checkMode != null){ + if(checkMode.equalsIgnoreCase("EXACTLY_ONCE")){ + env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE); + }else if(checkMode.equalsIgnoreCase("AT_LEAST_ONCE")){ + env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.AT_LEAST_ONCE); + }else{ + throw new RuntimeException("not support of FLINK_CHECKPOINT_MODE_KEY :" + checkMode); + } + } + + String checkpointTimeoutStr = properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_TIMEOUT_KEY); + if(checkpointTimeoutStr != null){ + Long checkpointTimeout = Long.valueOf(checkpointTimeoutStr); + //checkpoints have to complete within one min,or are discard + env.getCheckpointConfig().setCheckpointTimeout(checkpointTimeout); + } + + String maxConcurrCheckpointsStr = properties.getProperty(ConfigConstrant.FLINK_MAXCONCURRENTCHECKPOINTS_KEY); + if(maxConcurrCheckpointsStr != null){ + Integer maxConcurrCheckpoints = Integer.valueOf(maxConcurrCheckpointsStr); + //allow only one checkpoint to be int porgress at the same time + env.getCheckpointConfig().setMaxConcurrentCheckpoints(maxConcurrCheckpoints); + } + + String cleanupModeStr = properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_CLEANUPMODE_KEY); + if(cleanupModeStr != null){//设置在cancle job情况下checkpoint是否被保存 + if("true".equalsIgnoreCase(cleanupModeStr)){ + env.getCheckpointConfig().enableExternalizedCheckpoints( + CheckpointConfig.ExternalizedCheckpointCleanup.DELETE_ON_CANCELLATION); + }else if("false".equalsIgnoreCase(cleanupModeStr)){ + env.getCheckpointConfig().enableExternalizedCheckpoints( + CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); + }else{ + throw new RuntimeException("not support value of cleanup mode :" + cleanupModeStr); + } + } + + String backendPath = properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_DATAURI_KEY); + if(backendPath != null){ + //set checkpoint save path on file system, 根据实际的需求设定文件路径,hdfs://, file:// + env.setStateBackend(new FsStateBackend(backendPath)); + } + + } + + /** + * #ProcessingTime(默认),IngestionTime,EventTime + * @param env + * @param properties + */ + public static void setStreamTimeCharacteristic(StreamExecutionEnvironment env, Properties properties){ + if(!properties.containsKey(ConfigConstrant.FLINK_TIME_CHARACTERISTIC_KEY)){ + //走默认值 + return; + } + + String characteristicStr = properties.getProperty(ConfigConstrant.FLINK_TIME_CHARACTERISTIC_KEY); + Boolean flag = false; + for(TimeCharacteristic tmp : TimeCharacteristic.values()){ + if(characteristicStr.equalsIgnoreCase(tmp.toString())){ + env.setStreamTimeCharacteristic(tmp); + flag = true; + } + } + + if(!flag){ + throw new RuntimeException("illegal property :" + ConfigConstrant.FLINK_TIME_CHARACTERISTIC_KEY); + } + } + + + + /** + * FIXME 暂时不支持 UDF 实现类--有参构造方法 + * TABLE|SCALA + * 注册UDF到table env + */ + public static void registerUDF(String type, String classPath, String funcName, TableEnvironment tableEnv, + ClassLoader classLoader){ + if("SCALA".equalsIgnoreCase(type)){ + registerScalaUDF(classPath, funcName, tableEnv, classLoader); + }else if("TABLE".equalsIgnoreCase(type)){ + registerTableUDF(classPath, funcName, tableEnv, classLoader); + }else{ + throw new RuntimeException("not support of UDF which is not in (TABLE, SCALA)"); + } + + } + + /** + * 注册自定义方法到env上 + * @param classPath + * @param funcName + * @param tableEnv + */ + public static void registerScalaUDF(String classPath, String funcName, TableEnvironment tableEnv, + ClassLoader classLoader){ + try{ + ScalarFunction udfFunc = Class.forName(classPath, false, classLoader) + .asSubclass(ScalarFunction.class).newInstance(); + tableEnv.registerFunction(funcName, udfFunc); + logger.info("register scala function:{} success.", funcName); + }catch (Exception e){ + logger.error("", e); + throw new RuntimeException("register UDF exception:" + e.getMessage()); + } + } + + /** + * 注册自定义TABLEFFUNC方法到env上 + * TODO 对User-Defined Aggregate Functions的支持 + * @param classPath + * @param funcName + * @param tableEnv + */ + public static void registerTableUDF(String classPath, String funcName, TableEnvironment tableEnv, + ClassLoader classLoader){ + try { + TableFunction udfFunc = Class.forName(classPath, false, classLoader) + .asSubclass(TableFunction.class).newInstance(); + + if(tableEnv instanceof StreamTableEnvironment){ + ((StreamTableEnvironment)tableEnv).registerFunction(funcName, udfFunc); + }else if(tableEnv instanceof BatchTableEnvironment){ + ((BatchTableEnvironment)tableEnv).registerFunction(funcName, udfFunc); + }else{ + throw new RuntimeException("no support tableEnvironment class for " + tableEnv.getClass().getName()); + } + + logger.info("register table function:{} success.", funcName); + }catch (Exception e){ + logger.error("", e); + throw new RuntimeException("register Table UDF exception:" + e.getMessage()); + } + } + + + /** + * + * FIXME 仅针对sql执行方式,暂时未找到区分设置source,transform,sink 并行度的方式 + * 设置job运行的并行度 + * @param properties + */ + public static int getEnvParallelism(Properties properties){ + String parallelismStr = properties.getProperty(ConfigConstrant.SQL_ENV_PARALLELISM); + return StringUtils.isNotBlank(parallelismStr)?Integer.parseInt(parallelismStr):1; + } + + + /** + * 最大并发度 + * @param properties + * @return + */ + public static int getMaxEnvParallelism(Properties properties){ + String parallelismStr = properties.getProperty(ConfigConstrant.SQL_MAX_ENV_PARALLELISM); + return StringUtils.isNotBlank(parallelismStr)?Integer.parseInt(parallelismStr):0; + } + + /** + * + * @param properties + * @return + */ + public static long getBufferTimeoutMillis(Properties properties){ + String mills = properties.getProperty(ConfigConstrant.SQL_BUFFER_TIMEOUT_MILLIS); + return StringUtils.isNotBlank(mills)?Long.parseLong(mills):0L; + } + + public static URLClassLoader loadExtraJar(List jarURLList, URLClassLoader classLoader) throws NoSuchMethodException, IllegalAccessException, InvocationTargetException { + + int size = 0; + for(URL url : jarURLList){ + if(url.toString().endsWith(".jar")){ + size++; + } + } + + URL[] urlArray = new URL[size]; + int i=0; + for(URL url : jarURLList){ + if(url.toString().endsWith(".jar")){ + urlArray[i] = url; + urlClassLoaderAddUrl(classLoader, url); + i++; + } + } + + return classLoader; + } + + private static void urlClassLoaderAddUrl(URLClassLoader classLoader, URL url) throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { + Method method = classLoader.getClass().getDeclaredMethod("addURL", URL.class); + method.setAccessible(true); + method.invoke(classLoader, url); + } + + + public static TypeInformation[] transformTypes(Class[] fieldTypes){ + TypeInformation[] types = new TypeInformation[fieldTypes.length]; + for(int i=0; i ObjectToMap(Object obj) throws Exception{ + return objectMapper.readValue(objectMapper.writeValueAsBytes(obj), Map.class); + } + + public static T jsonStrToObject(String jsonStr, Class clazz) throws JsonParseException, JsonMappingException, JsonGenerationException, IOException{ + return objectMapper.readValue(jsonStr, clazz); + } + + public static Properties stringToProperties(String str) throws IOException{ + Properties properties = new Properties(); + properties.load(new ByteArrayInputStream(str.getBytes("UTF-8"))); + return properties; + } + + public static URL getRemoteJarFilePath(String pluginType, String tableType,String remoteSqlRootDir) throws MalformedURLException { + String dirName = pluginType + tableType.toLowerCase(); + String jarName = String.format("%s-%s.jar", pluginType, tableType.toLowerCase()); + return new URL("file:" + remoteSqlRootDir + SP + dirName + SP + jarName); + } + + public static String upperCaseFirstChar(String str){ + return str.substring(0, 1).toUpperCase() + str.substring(1); + } + + public static void addPluginJar(String pluginDir, DtClassLoader classLoader) throws MalformedURLException { + File dirFile = new File(pluginDir); + if(!dirFile.exists() || !dirFile.isDirectory()){ + throw new RuntimeException("plugin path:" + pluginDir + "is not exist."); + } + + File[] files = dirFile.listFiles(tmpFile -> tmpFile.isFile() && tmpFile.getName().endsWith(JAR_SUFFIX)); + if(files == null || files.length == 0){ + throw new RuntimeException("plugin path:" + pluginDir + " is null."); + } + + for(File file : files){ + URL pluginJarURL = file.toURI().toURL(); + classLoader.addURL(pluginJarURL); + } + } + +} diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java new file mode 100644 index 000000000..cbd050c27 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java @@ -0,0 +1,65 @@ +/* + * 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 com.dtstack.flink.sql.watermarker; + +import com.dtstack.flink.sql.util.MathUtil; +import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 自定义watermark---用于eventtime + * Date: 2017/12/28 + * Company: www.dtstack.com + * @author xuchao + */ + +public class CustomerWaterMarkerForLong extends BoundedOutOfOrdernessTimestampExtractor { + + private static final Logger logger = LoggerFactory.getLogger(CustomerWaterMarkerForLong.class); + + private static final long serialVersionUID = 1L; + + private int pos; + + private long lastTime = 0; + + public CustomerWaterMarkerForLong(Time maxOutOfOrderness, int pos) { + super(maxOutOfOrderness); + this.pos = pos; + } + + @Override + public long extractTimestamp(Row row) { + + try{ + Long eveTime = MathUtil.getLongVal(row.getField(pos)); + lastTime = eveTime; + return eveTime; + }catch (Exception e){ + logger.error("", e); + } + + return lastTime; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java new file mode 100644 index 000000000..47b988c24 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java @@ -0,0 +1,66 @@ +/* + * 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 com.dtstack.flink.sql.watermarker; + +import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Timestamp; +import java.text.ParseException; +import java.text.SimpleDateFormat; + +/** + * 自定义watermark---用于eventtime + * Date: 2017/12/28 + * Company: www.dtstack.com + * @author xuchao + */ + +public class CustomerWaterMarkerForTimeStamp extends BoundedOutOfOrdernessTimestampExtractor { + + private static final Logger logger = LoggerFactory.getLogger(CustomerWaterMarkerForTimeStamp.class); + + private static final long serialVersionUID = 1L; + + private int pos; + + private long lastTime = 0; + + public CustomerWaterMarkerForTimeStamp(Time maxOutOfOrderness, int pos) { + super(maxOutOfOrderness); + this.pos = pos; + } + + @Override + public long extractTimestamp(Row row) { + try { + Timestamp time = (Timestamp) row.getField(pos); + lastTime = time.getTime(); + return time.getTime(); + } catch (RuntimeException e) { + logger.error("", e); + } + return lastTime; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java new file mode 100644 index 000000000..c922d12d1 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.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 com.dtstack.flink.sql.watermarker; + +import com.dtstack.flink.sql.table.SourceTableInfo; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.guava18.com.google.common.base.Strings; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +/** + * 指定流的watermarker + * Date: 2018/6/29 + * Company: www.dtstack.com + * @author xuchao + */ + +public class WaterMarkerAssigner { + + public boolean checkNeedAssignWaterMarker(SourceTableInfo tableInfo){ + if(Strings.isNullOrEmpty(tableInfo.getEventTimeField())){ + return false; + } + + return true; + } + + public DataStream assignWaterMarker(DataStream dataStream, RowTypeInfo typeInfo, String eventTimeFieldName, int maxOutOfOrderness){ + + String[] fieldNames = typeInfo.getFieldNames(); + TypeInformation[] fieldTypes = typeInfo.getFieldTypes(); + String fields = StringUtils.join(fieldNames, ","); + + if(Strings.isNullOrEmpty(eventTimeFieldName)){ + return dataStream; + } + + int pos = -1; + for(int i=0; i paramList = Lists.newArrayList(); + paramList.add("-sql"); +// String sql = "CREATE TABLE MyTable(channel STRING, pv INT, xctime bigint, timeLeng as CHARACTER_LENGTH(channel), WATERMARK FOR xctime AS withOffset(xctime, 1000)) WITH ( type='kafka09', bootstrapServers='172.16.8.198:9092', offsetReset='latest',topic='nbTest1');" + +// "CREATE TABLE MyResult(channel STRING, pv INT) WITH ( type='mysql', url='jdbc:mysql://172.16.8.104:3306/test?charset=utf8',userName='dtstack',password='abc123', tableName='pv');" + +// "create table sideTable(channel String, xccount int, PRIMARY KEY (channel),PERIOD FOR SYSTEM_TIME) WITH ( type='mysql', url='jdbc:mysql://172.16.8.104:3306/test?charset=utf8',userName='dtstack',password='abc123', tableName='sidetest');" + +// "insert into MyResult select a.channel,b.xccount from MyTable a join sideTable b on a.channel=b.channel where b.channel = 'xc' and a.pv=10"; + + //String insetSql = "insert into MyResult select a.channel,b.xccount from MyTable a join sideTable b on a.channel=b.channel where b.channel = 'xc' and a.pv=10;"; + //String insetSql = "insert into MyResult select a.channel,b.xccount from (select * from MyTable) a left join sideTable b on a.channel=b.channel where a.channel = 'xc1' and a.pv=10;"; + //String insetSql = "insert into MyResult select * from sideTable"; + //sql = sql + insetSql; + //String sql = "create scala function xchashcode with com.xc.udf.MyHashCode;CREATE TABLE MyTable ( channel STRING, pv INT, xctime bigint, timeLeng AS CHARACTER_LENGTH(channel),WATERMARK FOR xctime AS withOffset( xctime , 10000 )) WITH (type = 'kafka09',bootstrapServers = '172.16.8.198:9092',zookeeperConnect = '172.16.8.198:2181/kafka', offsetReset = 'latest',topic = 'nbTest1',parallelism = '1');CREATE TABLE MyResult ( channel VARCHAR, pv bigint) WITH (type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 'pv',parallelism = '1');insert into MyResult select channel, count(channel) from MyTable GROUP BY channel ;"; + //String sql = "create scala function xchashcode with com.xc.udf.MyHashCode;CREATE TABLE MyTable ( channel STRING, pv INT, xctime bigint, timeLeng AS CHARACTER_LENGTH(channel),WATERMARK FOR xctime AS withOffset( xctime , 10000 )) WITH (type = 'kafka09',bootstrapServers = '172.16.8.198:9092',zookeeperConnect = '172.16.8.198:2181/kafka', offsetReset = 'latest',topic = 'nbTest1',parallelism = '1');CREATE TABLE MyResult ( channel VARCHAR, pv bigint) WITH (type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 'pv',parallelism = '1');insert into MyResult select channel, count(channel) from MyTable GROUP BY TUMBLE(rowtime, INTERVAL '3' SECOND), channel ;"; +// String sql = "CREATE TABLE MyTable(channel STRING, pv INT, xctime bigint, timeLeng as CHARACTER_LENGTH(channel), WATERMARK FOR xctime AS withOffset(xctime, 100)) WITH ( type='kafka09', bootstrapServers='172.16.8.198:9092', offsetReset='latest',topic='nbTest1');\n" + +// "CREATE TABLE MyResult(cf:channel STRING, cf:pv BIGINT) WITH ( type='hbase', zookeeperQuorum='rdos1:2181', zookeeperParent = '/hbase', tableName='tb1', rowkey='cf:channel');\n" + +// "insert into MyResult select channel from MyTable group by channel"; + //String sql ="CREATE TABLE student_1 ( id_1 varchar, name_1 varchar, sex_1 varchar) WITH (type = 'kafka09',bootstrapServers = '172.16.8.198:9092',zookeeperQuorum = '172.16.8.198:2181/kafka', offsetReset = 'latest',topic = 'test',parallelism = '1');CREATE TABLE sum_1 ( id_1 VARCHAR, sex_1 VARCHAR, name_1 VARCHAR, mark_1 INT) WITH (type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 'sum_1',parallelism = '1');CREATE TABLE score_1 ( id_1 VARCHAR, name_1 VARCHAR, mark_1 INT,PRIMARY KEY (id_1) , PERIOD FOR SYSTEM_TIME ) WITH (type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 'score_1',cache = 'LRU',cacheSize = '10000',cacheTTLMs = '60000',parallelism = '1');insert into sum_1 select a.id_1, a.sex_1, a.name_1, b.mark_1 from student_1 a inner join score_1 b on a.id=b.id_1;"; + //String sql = "CREATE TABLE MyTable ( channel STRING, pv INT, xctime bigint) WITH (type = 'kafka09',bootstrapServers = '172.16.8.198:9092',zookeeperQuorum = '172.16.8.198:2181/kafka', offsetReset = 'latest',topic = 'nbTest1',parallelism = '1');CREATE TABLE MyResult ( aa INT, bb INT) WITH (type = 'elasticsearch',address = '172.16.10.47:9500',cluster='es_47_menghan',estype = 'type1',index = 'xc_es_test',id = '0,1',parallelism = '1');insert into MyResult select pv,pv from MyTable;"; + + String sql = "CREATE TABLE bal_cur_batch_s30_pb_tranflow_input (trf_flowno varchar, trf_batno varchar, trf_cstno varchar, trf_bsncode varchar, trf_transtype varchar, trf_payacc varchar, trf_paysubacc varchar, trf_payname varchar, trf_rcvacc varchar, trf_rcvsubacc varchar, trf_rcvname varchar, trf_rcvbank varchar, trf_comitrno varchar, trf_crytype varchar, trf_subtime varchar, trf_tranamet varchar, trf_fee1 varchar, trf_fee2 varchar, trf_fee3 varchar, trf_userrem varchar, trf_hostflwno varchar, trf_hostsendtime varchar, trf_hosterror varchar, trf_lastupdtime varchar, trf_stt varchar, trf_schl_flowno varchar, trf_logontype varchar, trf_reccity varchar, trf_recprovice varchar, trf_channel varchar, trf_hosterrormsg varchar, trf_ext1 varchar, trf_ext2 varchar, trf_security varchar, cast(current_timestamp as varchar) as currTime) WITH (type = 'kafka09',bootstrapServers = 'kudu1:9092',zookeeperQuorum = '172.16.8.107:2181/kafka', offsetReset = 'latest',topic = 'tranflow_input',parallelism = '1');CREATE TABLE resultbank ( run_date VARCHAR, run_time VARCHAR, trf_flowno VARCHAR, trf_payname VARCHAR, trf_payacc VARCHAR, trf_tranamet VARCHAR, trf_subtime VARCHAR, trf_rcvbank VARCHAR,PRIMARY KEY (run_date,run_time,trf_flowno) ) WITH (type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/bank_test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 'resultbank',parallelism = '1');CREATE TABLE bal_cur_batch_rds_report_lrlct_trans_cur_input ( run_date VARCHAR, run_time VARCHAR, trf_flowno VARCHAR, trf_payname VARCHAR, trf_payacc VARCHAR, trf_subtime VARCHAR, trf_rcvbank VARCHAR,PRIMARY KEY (run_date,trf_flowno) , PERIOD FOR SYSTEM_TIME ) WITH (type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/bank_test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 'rds_report_lrlc_trans_cur',cache = 'LRU',cacheSize = '10000',cacheTTLMs = '60000',parallelism = '1');CREATE TABLE bal_cur_batch_t03_deposit_acct_base_info_cur_input ( data_dt VARCHAR, card_id VARCHAR, sub_acct_id VARCHAR, acct_org varchar, acct_stat_cd VARCHAR, core_cust_id VARCHAR, cust_rm VARCHAR, cust_scale_cd VARCHAR, item_id VARCHAR,PRIMARY KEY (card_id) , PERIOD FOR SYSTEM_TIME ) WITH (type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/bank_test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 't03_deposit_acct_base_info',cache = 'LRU',cacheSize = '10000',cacheTTLMs = '60000',parallelism = '1');CREATE TABLE bal_cur_batch_t04_org_cur_cur_input ( org_id VARCHAR, org_nm VARCHAR, org_short_nm VARCHAR, up_lvl_org_id VARCHAR, org_lvl VARCHAR, org_sort VARCHAR, org_cur VARCHAR,PRIMARY KEY (org_id) , PERIOD FOR SYSTEM_TIME ) WITH (type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/bank_test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 't04_org_cur',cache = 'LRU',cacheSize = '10000',cacheTTLMs = '60000',parallelism = '1');CREATE TABLE max_deposit_acct_base ( max_data_dt varchar, PRIMARY KEY (max_data_dt) , PERIOD FOR SYSTEM_TIME ) WITH (type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/bank_test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 'max_deposit_acct_base',cache = 'LRU',cacheSize = '10000',cacheTTLMs = '60000',parallelism = '1');\n" + + "INSERT \n" + + "INTO resultbank select d.run_date,\n" + + " d.run_time,\n" + + " d.trf_flowno,\n" + + " d.trf_payname,\n" + + " d.trf_payacc,\n" + + " d.trf_tranamet,\n" + + " d.trf_rcvbank,\n" + + " d.org_nm \n" + + " from\n" + + " (\n" + + " select\n" + + " cast(current_timestamp as varchar) as run_date,\n" + + " SUBSTRING(cast(current_timestamp as varchar) from 1 for 16) as run_time,\n" + + " b.trf_flowno,\n" + + " b.trf_payname,\n" + + " b.trf_payacc,\n" + + " b.trf_tranamet,\n" + + " b.trf_rcvbank,\n" + + " b.data_dt,\n" + + " t3.org_nm \n" + + " from\n" + + " ( select\n" + + " a.trf_flowno,\n" + + " a.currTime,\n" + + " a.trf_payname,\n" + + " a.trf_tranamet,\n" + + " a.trf_rcvbank,\n" + + " a.trf_payacc,\n" + + " a.trf_subtime,\n" + + " a.trf_bsncode,\n" + + " t2.acct_org,\n" + + " t2.data_dt,\n" + + " current_timestamp as nowtime\n" + + " from\n" + + " (select\n" + + " t1.trf_flowno,\n" + + " t1.currTime,\n" + + " t1.trf_payname,\n" + + " t1.trf_tranamet,\n" + + " t1.trf_rcvbank,\n" + + " t1.trf_subtime,\n" + + " t1.trf_payacc,\n" + + " t1.trf_bsncode\n" + + " from\n" + + " bal_cur_batch_s30_pb_tranflow_input t1 \n" + + " join\n" + + " bal_cur_batch_rds_report_lrlct_trans_cur_input x \n" + + " on t1.trf_flowno = x.trf_flowno \n" + + " and x.run_date = t1.currTime ) as a \n" + + " join\n" + + " bal_cur_batch_t03_deposit_acct_base_info_cur_input t2 \n" + + " on a.trf_payacc = t2.card_id \n" + + " ) as b \n" + + " join\n" + + " bal_cur_batch_t04_org_cur_cur_input t3 \n" + + " on b.acct_org = t3.org_id\n" + + " where \n" + + " b.trf_bsncode in('002002', '002003', '002011')\n" + + " and b.trf_flowno is null \n" + + " and substring(b.trf_subtime from 1 for 8) = DATE_FORMAT(b.nowtime, '%Y%m%d')\n" + + " and cast(b.trf_tranamet as decimal) >= 100000000\n" + + " ) as d\n" + + " join max_deposit_acct_base maxdep\n" + + " on d.data_dt = maxdep.max_data_dt\n"; + test(sql); + } + + @Test + public void testRunHbaseSideTable() throws Exception { + String sql = "CREATE TABLE MyTable ( name string, channel STRING, pv INT, xctime bigint, CHARACTER_LENGTH(channel) AS timeLeng) " + + "WITH (type = 'kafka09',bootstrapServers = '172.16.8.198:9092',zookeeperQuorum = '172.16.8.198:2181/kafka', " + + "offsetReset = 'latest',topic = 'nbTest1',parallelism = '1');" + + "CREATE TABLE MyResult ( channel VARCHAR, pv VARCHAR) WITH (type = 'mysql'," + + "url = 'jdbc:mysql://172.16.8.104:3306/test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 'pv2'," + + "parallelism = '1');CREATE TABLE workerinfo ( cast(logtime as TIMESTAMP) AS rtime, cast(logtime) AS rtime) " + + "WITH (type = 'hbase',zookeeperQuorum = 'rdos1:2181',tableName = 'workerinfo',rowKey = 'ce,de'," + + "parallelism = '1',zookeeperParent = '/hbase');CREATE TABLE sideTable " + + "( cf:name String as name, cf:info String as info,PRIMARY KEY (name) , PERIOD FOR SYSTEM_TIME ) WITH " + + "(type = 'hbase',zookeeperQuorum = 'rdos1:2181',zookeeperParent = '/hbase',tableName = 'workerinfo',cache = 'LRU'," + + "cacheSize = '10000',cacheTTLMs = '60000',parallelism = '1');" + + "insert \n" + + "into\n" + + " MyResult\n" + + " select d.channel,d.info\n" + + " from\n" + + " (\n" + + " select\n" + + " *\n" + + " from\n" + + " MyTable a \n" + + " join\n" + + " sideTable b \n" + + " on a.channel=b.name \n" + + " where\n" + + " a.channel = 'xc2' \n" + + " and a.pv=10\n" + + " ) as d\n"; + + test(sql); + } + + public void test(String sql) throws Exception { + List paramList = Lists.newArrayList(); + paramList.add("-sql"); + String exeSql = URLEncoder.encode(sql, Charsets.UTF_8.name()); + paramList.add(exeSql); + paramList.add("-name"); + paramList.add("xc"); + paramList.add("-localSqlPluginPath"); + paramList.add("D:\\gitspace\\flink-sql-plugin\\plugins"); + paramList.add("-mode"); + paramList.add("local"); + paramList.add("-addjar"); + paramList.add(URLEncoder.encode("[\"D:\\\\gitspace\\\\rdos-execution-engine\\\\..\\\\tmp140\\\\flink14Test-1.0-SNAPSHOT.jar\"]", Charsets.UTF_8.name())); + paramList.add("-remoteSqlPluginPath"); + paramList.add("/opt/dtstack/flinkplugin"); + paramList.add("-confProp"); + String conf = "{\"time.characteristic\":\"EventTime\",\"sql.checkpoint.interval\":10000}"; + String confEncode = URLEncoder.encode(conf, Charsets.UTF_8.name()); + paramList.add(confEncode); + + String[] params = new String[paramList.size()]; + paramList.toArray(params); + Main.main(params); + } +} diff --git a/core/src/test/java/com/dtstack/flink/sql/side/TestSideSqlParser.java b/core/src/test/java/com/dtstack/flink/sql/side/TestSideSqlParser.java new file mode 100644 index 000000000..01e33a3b6 --- /dev/null +++ b/core/src/test/java/com/dtstack/flink/sql/side/TestSideSqlParser.java @@ -0,0 +1,59 @@ +/* + * 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 com.dtstack.flink.sql.side; + +import org.apache.calcite.sql.parser.SqlParseException; +import org.apache.flink.calcite.shaded.com.google.common.collect.Sets; +import org.junit.Test; + +import java.util.Set; + +/** + * Reason: + * Date: 2018/7/24 + * Company: www.dtstack.com + * @author xuchao + */ + +public class TestSideSqlParser { + + @Test + public void testSideSqlParser() throws SqlParseException { + String sql = "select j1.id,j2.name,j1.info \n" + + " from\n" + + " (\n" + + " \tselect a.id,a.name,b.id \n" + + " \t\tfrom tab1 a join tab2 b\n" + + " \t\ton a.id = b.id and a.proctime between b.proctime - interval '4' second and b.proctime + interval '4' second \n" + + " ) j1\n" + + " join tab3 j2\n" + + " on j1.id = j2.id \n" + + " where j1.info like 'xc2'"; + + Set sideTbList = Sets.newHashSet("TAB3", "TAB4"); + + + SideSQLParser sideSQLParser = new SideSQLParser(); + sideSQLParser.getExeQueue(sql, sideTbList); + } + + +} diff --git a/elasticsearch/elasticsearch-sink/pom.xml b/elasticsearch/elasticsearch-sink/pom.xml new file mode 100644 index 000000000..600158e84 --- /dev/null +++ b/elasticsearch/elasticsearch-sink/pom.xml @@ -0,0 +1,112 @@ + + + + sql.elasticsearch + com.dtstack.flinkx + 1.0-SNAPSHOT + + 4.0.0 + + sql.sink.elasticsearch + elasticsearch-sink + + + + junit + junit + 3.8.1 + test + + + + com.dtstack.flinkx + sql.core + 1.0-SNAPSHOT + provided + + + + org.apache.flink + flink-connector-elasticsearch5_2.11 + 1.4.0 + + + + ch.qos.logback + logback-core + 1.1.7 + + + + ch.qos.logback + logback-classic + 1.1.7 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java b/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java new file mode 100644 index 000000000..7ad2720b4 --- /dev/null +++ b/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java @@ -0,0 +1,111 @@ +/* + * 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 com.dtstack.flink.sql.sink.elasticsearch; + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.types.Row; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.Requests; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2017/7/19 + * Company: www.dtstack.com + * @author xuchao + */ + +public class CustomerSinkFunc implements ElasticsearchSinkFunction { + + private final Logger logger = LoggerFactory.getLogger(CustomerSinkFunc.class); + + private String index; + + private String type; + + private List idFieldIndexList; + + private List fieldNames; + + private List fieldTypes; + + /** 默认分隔符为'_' */ + private char sp = '_'; + + public CustomerSinkFunc(String index, String type, List fieldNames, List fieldTypes, List idFieldIndexes){ + this.index = index; + this.type = type; + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + this.idFieldIndexList = idFieldIndexes; + } + + @Override + public void process(Tuple2 tuple2, RuntimeContext ctx, RequestIndexer indexer) { + try{ + Tuple2 tupleTrans = tuple2; + Boolean retract = tupleTrans.getField(0); + Row element = tupleTrans.getField(1); + if(!retract){ + return; + } + + + indexer.add(createIndexRequest(element)); + }catch (Throwable e){ + logger.error("", e); + } + } + + private IndexRequest createIndexRequest(Row element) { + + List idFieldList = new ArrayList<>(); + for(int index : idFieldIndexList){ + if(index >= element.getArity()){ + continue; + } + + idFieldList.add(element.getField(index).toString()); + } + + Map dataMap = EsUtil.rowToJsonMap(element,fieldNames,fieldTypes); + int length = Math.min(element.getArity(), fieldNames.size()); + for(int i=0; i, IStreamSinkGener { + + private final Logger logger = LoggerFactory.getLogger(ElasticsearchSink.class); + + private String clusterName; + + private int bulkFlushMaxActions = 1; + + private List esAddressList; + + private String index = ""; + + private String type = ""; + + private List idIndexList; + + protected String[] fieldNames; + + protected String[] columnTypes; + + private TypeInformation[] fieldTypes; + + private int parallelism = -1; + + + @Override + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + + @Override + public TupleTypeInfo> getOutputType() { + return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), getRecordType()); + } + + @Override + public TypeInformation getRecordType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + + private RichSinkFunction createEsSinkFunction(){ + + + Map userConfig = new HashMap<>(); + userConfig.put("cluster.name", clusterName); + // This instructs the sink to emit after every element, otherwise they would be buffered + userConfig.put(org.apache.flink.streaming.connectors.elasticsearch5.ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "" + bulkFlushMaxActions); + List transports = new ArrayList<>(); + + for(String address : esAddressList){ + String[] infoArray = address.split(":"); + int port = 9300; + String host = infoArray[0]; + if(infoArray.length > 1){ + port = Integer.valueOf(infoArray[1].trim()); + } + + try { + transports.add(new InetSocketAddress(InetAddress.getByName(host), port)); + }catch (Exception e){ + logger.error("", e); + throw new RuntimeException(e); + } + } + + CustomerSinkFunc customerSinkFunc = new CustomerSinkFunc(index, type, Arrays.asList(fieldNames), Arrays.asList(columnTypes), idIndexList); + + return new org.apache.flink.streaming.connectors.elasticsearch5.ElasticsearchSink(userConfig, transports, customerSinkFunc); + } + + @Override + public void emitDataStream(DataStream> dataStream) { + RichSinkFunction richSinkFunction = createEsSinkFunction(); + DataStreamSink streamSink = dataStream.addSink(richSinkFunction); + if(parallelism > 0){ + streamSink.setParallelism(parallelism); + } + } + + public void setParallelism(int parallelism) { + this.parallelism = parallelism; + } + + public void setBulkFlushMaxActions(int bulkFlushMaxActions) { + this.bulkFlushMaxActions = bulkFlushMaxActions; + } + + @Override + public ElasticsearchSink genStreamSink(TargetTableInfo targetTableInfo) { + ElasticsearchTableInfo elasticsearchTableInfo = (ElasticsearchTableInfo) targetTableInfo; + clusterName = elasticsearchTableInfo.getClusterName(); + String address = elasticsearchTableInfo.getAddress(); + String[] addr = address.split(","); + esAddressList = Arrays.asList(addr); + index = elasticsearchTableInfo.getIndex(); + type = elasticsearchTableInfo.getEsType(); + String id = elasticsearchTableInfo.getId(); + String[] idField = id.split(","); + idIndexList = new ArrayList<>(); + + for(int i = 0; i < idField.length; ++i) { + idIndexList.add(Integer.valueOf(idField[i])); + } + + columnTypes = elasticsearchTableInfo.getFieldTypes(); + + return this; + } +} diff --git a/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/EsUtil.java b/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/EsUtil.java new file mode 100644 index 000000000..26d7827da --- /dev/null +++ b/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/EsUtil.java @@ -0,0 +1,65 @@ +/* + * 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 com.dtstack.flink.sql.sink.elasticsearch; + +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Utilities for ElasticSearch + * + * Company: www.dtstack.com + * @author huyifan.zju@163.com + */ +public class EsUtil { + + public static Map rowToJsonMap(Row row, List fields, List types) { + Preconditions.checkArgument(row.getArity() == fields.size()); + Map jsonMap = new HashMap<>(); + int i = 0; + for(; i < fields.size(); ++i) { + String field = fields.get(i); + String[] parts = field.split("\\."); + Map currMap = jsonMap; + for(int j = 0; j < parts.length - 1; ++j) { + String key = parts[j]; + if(currMap.get(key) == null) { + currMap.put(key, new HashMap()); + } + currMap = (Map) currMap.get(key); + } + String key = parts[parts.length - 1]; + Object col = row.getField(i); + if(col != null) { + Object value = DtStringUtil.col2string(col, types.get(i)); + currMap.put(key, value); + } + + } + + return jsonMap; + } + + +} diff --git a/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.java b/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.java new file mode 100644 index 000000000..dd9a65e3e --- /dev/null +++ b/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.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 com.dtstack.flink.sql.sink.elasticsearch.table; + + +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import java.util.Map; + +/** + * @date 2018/09/12 + * @author sishu.yss + * @Company: www.dtstack.com + */ +public class ElasticsearchSinkParser extends AbsTableParser { + + private static final String KEY_ES_ADDRESS = "address"; + + private static final String KEY_ES_CLUSTER = "cluster"; + + private static final String KEY_ES_INDEX = "index"; + + private static final String KEY_ES_TYPE = "estype"; + + private static final String KEY_ES_ID_FIELD_INDEX_LIST = "id"; + + @Override + protected boolean fieldNameNeedsUpperCase() { + return false; + } + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + ElasticsearchTableInfo elasticsearchTableInfo = new ElasticsearchTableInfo(); + elasticsearchTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, elasticsearchTableInfo); + elasticsearchTableInfo.setAddress((String) props.get(KEY_ES_ADDRESS.toLowerCase())); + elasticsearchTableInfo.setClusterName((String) props.get(KEY_ES_CLUSTER.toLowerCase())); + elasticsearchTableInfo.setId((String) props.get(KEY_ES_ID_FIELD_INDEX_LIST.toLowerCase())); + elasticsearchTableInfo.setIndex((String) props.get(KEY_ES_INDEX.toLowerCase())); + elasticsearchTableInfo.setEsType((String) props.get(KEY_ES_TYPE.toLowerCase())); + return elasticsearchTableInfo; + } +} diff --git a/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java b/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java new file mode 100644 index 000000000..19f158c3a --- /dev/null +++ b/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java @@ -0,0 +1,106 @@ +/* + * 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 com.dtstack.flink.sql.sink.elasticsearch.table; + + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * @date 2018/09/12 + * @author sishu.yss + * @Company: www.dtstack.com + */ +public class ElasticsearchTableInfo extends TargetTableInfo { + + private static final String CURR_TYPE = "elasticsearch"; + + private String address; + + private String index; + + private String id; + + private String clusterName; + + private String esType; + + public String getEsType() { + return esType; + } + + public void setEsType(String esType) { + this.esType = esType; + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public String getAddress() { + return address; + } + + public void setAddress(String address) { + this.address = address; + } + + public String getIndex() { + return index; + } + + public void setIndex(String index) { + this.index = index; + } + + @Override + public String getType() { + //return super.getType().toLowerCase() + TARGET_SUFFIX; + return super.getType().toLowerCase(); + } + + public String getClusterName() { + return clusterName; + } + + public void setClusterName(String clusterName) { + this.clusterName = clusterName; + } + + public ElasticsearchTableInfo() { + setType(CURR_TYPE); + } + + @Override + public boolean check() { + Preconditions.checkNotNull(address, "elasticsearch type of address is required"); + Preconditions.checkNotNull(index, "elasticsearch type of index is required"); + Preconditions.checkNotNull(esType, "elasticsearch type of type is required"); + Preconditions.checkNotNull(id, "elasticsearch type of id is required"); + Preconditions.checkNotNull(clusterName, "elasticsearch type of clusterName is required"); + return true; + } + +} diff --git a/elasticsearch/pom.xml b/elasticsearch/pom.xml new file mode 100644 index 000000000..03a1c1e3f --- /dev/null +++ b/elasticsearch/pom.xml @@ -0,0 +1,19 @@ + + + + flink.sql + com.dtstack.flinkx + 1.0-SNAPSHOT + + 4.0.0 + + sql.elasticsearch + pom + + elasticsearch-sink + + + + \ No newline at end of file diff --git a/hbase/hbase-side/doc/readme.txt b/hbase/hbase-side/doc/readme.txt new file mode 100644 index 000000000..06d628f51 --- /dev/null +++ b/hbase/hbase-side/doc/readme.txt @@ -0,0 +1,57 @@ +1: example: +CREATE TABLE sideTable( + cf:name String as name, + cf:info int as info, + PRIMARY KEY(md5(name) + 'test'), + PERIOD FOR SYSTEM_TIME + )WITH( + type ='hbase', + zookeeperQuorum ='rdos1:2181', + zookeeperParent ='/hbase', + tableName ='workerinfo', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='true' + ); + +2: 格式: + CREATE TABLE tableName( + columnFamily:columnName type as alias, + ... + PRIMARY KEY(keyInfo), + PERIOD FOR SYSTEM_TIME + )WITH( + type ='hbase', + zookeeperQuorum ='ip:port', + zookeeperParent ='/hbase', + tableName ='tableNamae', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' + ); + + +tableName ==> 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同) +columnFamily:columnName ==> hbase中的列族名称和列名称 +alias ===> hbase 中的列对应到flink中注册的列名称 +PERIOD FOR SYSTEM_TIME ==> 关键字表明该定义的表为维表信息 +PRIMARY KEY(keyInfo) ==> 维表主键定义;hbase 维表为rowkey的构造方式; + 可选择的构造包括 md5(alias + alias), '常量',也包括上述方式的自由组合 +type ==> 表明维表的类型 +zookeeperQuorum ==> hbase 的zk地址;格式ip:port[;ip:port] +zookeeperParent ==> hbase 的zk parent路径 +tableName ==> hbase 的表名称 +cache ==> 维表缓存策略(NONE/LRU) +partitionedJoin ==> 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量) + +NONE: 不做内存缓存 + +LRU: +cacheSize ==> 缓存的条目数量 +cacheTTLMs ==> 缓存的过期时间(ms) + + diff --git a/hbase/hbase-side/pom.xml b/hbase/hbase-side/pom.xml new file mode 100644 index 000000000..bf68e70b4 --- /dev/null +++ b/hbase/hbase-side/pom.xml @@ -0,0 +1,100 @@ + + + + sql.hbase + com.dtstack.flinkx + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.hbase + hbase-side + + + com.dtstack.flinkx + sql.core + 1.0-SNAPSHOT + provided + + + + org.apache.hbase + hbase-client + 1.3.1 + + + + org.hbase + asynchbase + 1.8.2 + + + + + jar + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java new file mode 100644 index 000000000..cec7a0400 --- /dev/null +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java @@ -0,0 +1,246 @@ +/* + * 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 com.dtstack.flink.sql.side.hbase; + +import com.dtstack.flink.sql.enums.ECacheContentType; +import com.dtstack.flink.sql.side.AsyncReqRow; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.cache.CacheObj; +import com.dtstack.flink.sql.side.hbase.rowkeydealer.AbsRowKeyModeDealer; +import com.dtstack.flink.sql.side.hbase.rowkeydealer.PreRowKeyModeDealerDealer; +import com.dtstack.flink.sql.side.hbase.rowkeydealer.RowKeyEqualModeDealer; +import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo; +import com.google.common.collect.Maps; +import com.stumbleupon.async.Deferred; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.types.Row; +import org.hbase.async.HBaseClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import com.dtstack.flink.sql.threadFactory.DTThreadFactory; +import java.sql.Timestamp; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +/** + * Date: 2018/8/21 + * Company: www.dtstack.com + * @author xuchao + */ + +public class HbaseAsyncReqRow extends AsyncReqRow { + + private static final long serialVersionUID = 2098635104857937717L; + + private static final Logger LOG = LoggerFactory.getLogger(HbaseAsyncReqRow.class); + + private static final int HBASE_WORKER_POOL_SIZE = 10; + + private RowKeyBuilder rowKeyBuilder; + + private transient HBaseClient hBaseClient; + + private transient AbsRowKeyModeDealer rowKeyMode; + + private String tableName; + + private String[] colNames; + + private Map colRefType; + + public HbaseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + rowKeyBuilder = new RowKeyBuilder(); + if(sideTableInfo.getPrimaryKeys().size() < 1){ + throw new RuntimeException("Primary key dimension table must be filled"); + } + + HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo; + rowKeyBuilder.init(sideTableInfo.getPrimaryKeys().get(0)); + + colRefType = Maps.newHashMap(); + for(int i=0; i sqlNodeList = Lists.newArrayList(); + if(conditionNode.getKind() == SqlKind.AND){ + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); + }else{ + sqlNodeList.add(conditionNode); + } + + for(SqlNode sqlNode : sqlNodeList){ + dealOneEqualCon(sqlNode, sideTableName); + } + + tableName = hbaseSideTableInfo.getTableName(); + colNames = hbaseSideTableInfo.getColumnRealNames(); + } + + @Override + public void open(Configuration parameters) throws Exception { + HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo; + ExecutorService executorService =new ThreadPoolExecutor(HBASE_WORKER_POOL_SIZE, HBASE_WORKER_POOL_SIZE, + 0L, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue<>(),new DTThreadFactory("hbase-aysnc")); + hBaseClient = new HBaseClient(hbaseSideTableInfo.getHost(), hbaseSideTableInfo.getParent(), executorService); + try { + Deferred deferred = hBaseClient.ensureTableExists(tableName) + .addCallbacks(arg -> new CheckResult(true, ""), arg -> new CheckResult(false, arg.toString())); + + CheckResult result = (CheckResult) deferred.join(); + if(!result.isConnect()){ + throw new RuntimeException(result.getExceptionMsg()); + } + + } catch (Exception e) { + throw new RuntimeException("create hbase connection fail:", e); + } + + if(hbaseSideTableInfo.isPreRowKey()){ + rowKeyMode = new PreRowKeyModeDealerDealer(colRefType, colNames, hBaseClient, + openCache(), joinType, outFieldInfoList, inFieldIndex, sideFieldIndex); + }else{ + rowKeyMode = new RowKeyEqualModeDealer(colRefType, colNames, hBaseClient, + openCache(), joinType, outFieldInfoList, inFieldIndex, sideFieldIndex); + } + } + + @Override + public void asyncInvoke(Row input, ResultFuture resultFuture) throws Exception { + Map refData = Maps.newHashMap(); + for (int i = 0; i < equalValIndex.size(); i++) { + Integer conValIndex = equalValIndex.get(i); + Object equalObj = input.getField(conValIndex); + if(equalObj == null){ + resultFuture.complete(null); + } + + refData.put(equalFieldList.get(i), equalObj); + } + + String rowKeyStr = rowKeyBuilder.getRowKey(refData); + + //get from cache + if(openCache()){ + CacheObj val = getFromCache(rowKeyStr); + if(val != null){ + if(ECacheContentType.MissVal == val.getType()){ + dealMissKey(input, resultFuture); + return; + }else if(ECacheContentType.SingleLine == val.getType()){ + Row row = fillData(input, val); + resultFuture.complete(Collections.singleton(row)); + }else if(ECacheContentType.MultiLine == val.getType()){ + for(Object one : (List)val.getContent()){ + Row row = fillData(input, one); + resultFuture.complete(Collections.singleton(row)); + } + } + return; + } + } + + rowKeyMode.asyncGetData(tableName, rowKeyStr, input, resultFuture, sideCache); + } + + @Override + protected Row fillData(Row input, Object sideInput){ + + List sideInputList = (List) sideInput; + Row row = new Row(outFieldInfoList.size()); + for(Map.Entry entry : inFieldIndex.entrySet()){ + Object obj = input.getField(entry.getValue()); + if(obj instanceof Timestamp){ + obj = ((Timestamp)obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for(Map.Entry entry : sideFieldIndex.entrySet()){ + if(sideInputList == null){ + row.setField(entry.getKey(), null); + }else{ + row.setField(entry.getKey(), sideInputList.get(entry.getValue())); + } + } + + return row; + } + + @Override + public void close() throws Exception { + super.close(); + hBaseClient.shutdown(); + } + + + class CheckResult{ + + private boolean connect; + + private String exceptionMsg; + + CheckResult(boolean connect, String msg){ + this.connect = connect; + this.exceptionMsg = msg; + } + + public boolean isConnect() { + return connect; + } + + public void setConnect(boolean connect) { + this.connect = connect; + } + + public String getExceptionMsg() { + return exceptionMsg; + } + + public void setExceptionMsg(String exceptionMsg) { + this.exceptionMsg = exceptionMsg; + } + } +} diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/Md5ReplaceOperator.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/Md5ReplaceOperator.java new file mode 100644 index 000000000..af5e24dd6 --- /dev/null +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/Md5ReplaceOperator.java @@ -0,0 +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 com.dtstack.flink.sql.side.hbase; + +import com.dtstack.flink.sql.side.hbase.enums.EReplaceOpType; +import com.dtstack.flink.sql.util.MD5Utils; + +/** + * Reason: + * Date: 2018/8/23 + * Company: www.dtstack.com + * @author xuchao + */ + +public class Md5ReplaceOperator extends ReplaceOperator{ + + public Md5ReplaceOperator(EReplaceOpType opType) { + super(opType); + } + + @Override + String doFunc(String replaceStr) { + return MD5Utils.getMD5String(replaceStr); + } +} diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceInfo.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceInfo.java new file mode 100644 index 000000000..92d8f6d45 --- /dev/null +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceInfo.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 com.dtstack.flink.sql.side.hbase; + +import com.dtstack.flink.sql.side.hbase.enums.EReplaceType; + +import java.io.Serializable; + +/** + * Reason: + * Date: 2018/8/23 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class ReplaceInfo implements Serializable { + + private static final long serialVersionUID = 2058635242957737717L; + + private EReplaceType type; + + private String param; + + public ReplaceInfo(EReplaceType type){ + this.type = type; + } + + public EReplaceType getType() { + return type; + } + + public void setType(EReplaceType type) { + this.type = type; + } + + public String getParam() { + return param; + } + + public void setParam(String param) { + this.param = param; + } +} diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceOperator.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceOperator.java new file mode 100644 index 000000000..ec33a8413 --- /dev/null +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceOperator.java @@ -0,0 +1,57 @@ +/* + * 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 com.dtstack.flink.sql.side.hbase; + +import com.dtstack.flink.sql.side.hbase.enums.EReplaceOpType; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/8/23 + * Company: www.dtstack.com + * @author xuchao + */ + +public abstract class ReplaceOperator { + + private EReplaceOpType opType; + + public ReplaceOperator(EReplaceOpType opType){ + this.opType = opType; + } + + public String doOperator(Map refData){ + String replaceStr = replaceStr(refData); + return doFunc(replaceStr); + } + + public String replaceStr(Map refData){ + return ""; + } + + /** + * 根据函数提供自定义的处理 + * @param replaceStr + * @return + */ + abstract String doFunc(String replaceStr); +} diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java new file mode 100644 index 000000000..f64af3ff9 --- /dev/null +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.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 com.dtstack.flink.sql.side.hbase; + +import com.dtstack.flink.sql.side.hbase.enums.EReplaceType; +import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.collect.Lists; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * rowkey 生成规则 + * Date: 2018/8/23 + * Company: www.dtstack.com + * @author xuchao + */ + +public class RowKeyBuilder implements Serializable{ + + private static final long serialVersionUID = 2058635242857937717L; + + private static Pattern Md5Operator = Pattern.compile("(?i)^md5\\(\\s*(.*)\\s*\\)$"); + + private List> operatorChain = Lists.newArrayList(); + + public void init(String rowKeyTempl){ + + String[] strArr = splitIgnoreQuotaBrackets(rowKeyTempl, "\\+"); + + for(String infoAlias : strArr){ + infoAlias = infoAlias.trim(); + Matcher matcher = Md5Operator.matcher(infoAlias); + String fieldCols = null; + if(matcher.find()){ + fieldCols = matcher.group(1); + }else{ + fieldCols = infoAlias; + } + + String[] fieldArr = fieldCols.split("\\+"); + List fieldList = Lists.newArrayList(); + for(String oneField : fieldArr){ + ReplaceInfo replaceInfo = getReplaceInfo(oneField); + fieldList.add(replaceInfo); + } + + operatorChain.add(fieldList); + } + } + + /** + * + * @param refData + * @return + */ + public String getRowKey(Map refData){ + + StringBuilder sb = new StringBuilder(""); + for(List fieldList : operatorChain){ + sb.append(buildStr(fieldList, refData)); + } + + return sb.toString(); + } + + + + private String buildStr(List fieldList, Map refData){ + StringBuffer sb = new StringBuffer(""); + for(ReplaceInfo replaceInfo : fieldList){ + + if(replaceInfo.getType() == EReplaceType.CONSTANT){ + sb.append(replaceInfo.getParam()); + continue; + } + + String replaceName = replaceInfo.getParam(); + if(!refData.containsKey(replaceName)){ + throw new RuntimeException(String.format("build rowKey with field %s which value not found.", replaceName)); + } + + sb.append(refData.get(replaceName)); + } + + return sb.toString(); + } + + public static String[] splitIgnoreQuotaBrackets(String str, String delimiter){ + String splitPatternStr = delimiter + "(?![^()]*+\\))(?![^{}]*+})(?![^\\[\\]]*+\\])"; + return str.split(splitPatternStr); + } + + /** + * + * @param field + * @return + */ + public ReplaceInfo getReplaceInfo(String field){ + + field = field.trim(); + if(field.length() <= 2){ + throw new RuntimeException(field + " \n" + + "Format defined exceptions"); + } + + //判断是不是常量==>''包裹的标识 + if(field.startsWith("'") && field.endsWith("'")){ + ReplaceInfo replaceInfo = new ReplaceInfo(EReplaceType.CONSTANT); + field = field.substring(1, field.length() - 1); + replaceInfo.setParam(field); + return replaceInfo; + } + + ReplaceInfo replaceInfo = new ReplaceInfo(EReplaceType.PARAM); + replaceInfo.setParam(field); + return replaceInfo; + } + +} diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceOpType.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceOpType.java new file mode 100644 index 000000000..834edf9ae --- /dev/null +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceOpType.java @@ -0,0 +1,32 @@ +/* + * 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 com.dtstack.flink.sql.side.hbase.enums; + +/** + * Reason: + * Date: 2018/8/23 + * Company: www.dtstack.com + * @author xuchao + */ + +public enum EReplaceOpType { + NO_FUNC, MD5_FUNC; +} diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceType.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceType.java new file mode 100644 index 000000000..3768d7c5e --- /dev/null +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceType.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 com.dtstack.flink.sql.side.hbase.enums; + +/** + * Reason: + * Date: 2018/8/23 + * Company: www.dtstack.com + * + * @author xuchao + */ +public enum EReplaceType { + PARAM, + CONSTANT; +} diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbsRowKeyModeDealer.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbsRowKeyModeDealer.java new file mode 100644 index 000000000..5fe61af4f --- /dev/null +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbsRowKeyModeDealer.java @@ -0,0 +1,110 @@ +/* + * 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 com.dtstack.flink.sql.side.hbase.rowkeydealer; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.cache.AbsSideCache; +import org.apache.calcite.sql.JoinType; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.types.Row; +import org.hbase.async.HBaseClient; + +import java.sql.Timestamp; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2018/9/10 + * Company: www.dtstack.com + * @author xuchao + */ + +public abstract class AbsRowKeyModeDealer { + + protected Map colRefType; + + protected String[] colNames; + + protected HBaseClient hBaseClient; + + protected boolean openCache; + + protected JoinType joinType; + + protected List outFieldInfoList; + + //key:Returns the value of the position, returns the index values ​​in the input data + protected Map inFieldIndex = Maps.newHashMap(); + + protected Map sideFieldIndex = Maps.newHashMap(); + + public AbsRowKeyModeDealer(Map colRefType, String[] colNames, HBaseClient hBaseClient, + boolean openCache, JoinType joinType, List outFieldInfoList, + Map inFieldIndex, Map sideFieldIndex){ + this.colRefType = colRefType; + this.colNames = colNames; + this.hBaseClient = hBaseClient; + this.openCache = openCache; + this.joinType = joinType; + this.outFieldInfoList = outFieldInfoList; + this.inFieldIndex = inFieldIndex; + this.sideFieldIndex = sideFieldIndex; + } + + protected void dealMissKey(Row input, ResultFuture resultFuture){ + if(joinType == JoinType.LEFT){ + //保留left 表数据 + Row row = fillData(input, null); + resultFuture.complete(Collections.singleton(row)); + }else{ + resultFuture.complete(null); + } + } + + protected Row fillData(Row input, Object sideInput){ + + List sideInputList = (List) sideInput; + Row row = new Row(outFieldInfoList.size()); + for(Map.Entry entry : inFieldIndex.entrySet()){ + Object obj = input.getField(entry.getValue()); + if(obj instanceof Timestamp){ + obj = ((Timestamp)obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for(Map.Entry entry : sideFieldIndex.entrySet()){ + if(sideInputList == null){ + row.setField(entry.getKey(), null); + }else{ + row.setField(entry.getKey(), sideInputList.get(entry.getValue())); + } + } + + return row; + } + + public abstract void asyncGetData(String tableName, String rowKeyStr, Row input, ResultFuture resultFuture, + AbsSideCache sideCache); +} diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java new file mode 100644 index 000000000..a5e536ff8 --- /dev/null +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java @@ -0,0 +1,146 @@ +/* + * 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 com.dtstack.flink.sql.side.hbase.rowkeydealer; + +import com.dtstack.flink.sql.enums.ECacheContentType; +import com.dtstack.flink.sql.side.hbase.utils.HbaseUtils; +import com.dtstack.flink.sql.side.CacheMissVal; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.cache.AbsSideCache; +import com.dtstack.flink.sql.side.cache.CacheObj; +import com.google.common.collect.Maps; +import org.apache.calcite.sql.JoinType; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.types.Row; +import org.hbase.async.BinaryPrefixComparator; +import org.hbase.async.Bytes; +import org.hbase.async.CompareFilter; +import org.hbase.async.HBaseClient; +import org.hbase.async.KeyValue; +import org.hbase.async.RowFilter; +import org.hbase.async.ScanFilter; +import org.hbase.async.Scanner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2018/9/10 + * Company: www.dtstack.com + * @author xuchao + */ + +public class PreRowKeyModeDealerDealer extends AbsRowKeyModeDealer { + + private static final Logger LOG = LoggerFactory.getLogger(PreRowKeyModeDealerDealer.class); + + public PreRowKeyModeDealerDealer(Map colRefType, String[] colNames, HBaseClient hBaseClient, + boolean openCache, JoinType joinType, List outFieldInfoList, + Map inFieldIndex, Map sideFieldIndex) { + super(colRefType, colNames, hBaseClient, openCache, joinType, outFieldInfoList, inFieldIndex, sideFieldIndex); + } + + @Override + public void asyncGetData(String tableName, String rowKeyStr, Row input, ResultFuture resultFuture, + AbsSideCache sideCache) { + Scanner prefixScanner = hBaseClient.newScanner(tableName); + ScanFilter scanFilter = new RowFilter(CompareFilter.CompareOp.EQUAL, new BinaryPrefixComparator(Bytes.UTF8(rowKeyStr))); + prefixScanner.setFilter(scanFilter); + prefixScanner.setMaxVersions(1); + + prefixScanner.nextRows().addCallbacks(arg -> { + dealOneRow(arg, rowKeyStr, input, resultFuture, sideCache); + return ""; + }, arg2 -> dealFail(arg2, input, resultFuture)); + } + + + private String dealOneRow(ArrayList> args, String rowKeyStr, Row input, ResultFuture resultFuture, AbsSideCache sideCache) { + if(args == null || args.size() == 0){ + dealMissKey(input, resultFuture); + if (openCache) { + sideCache.putCache(rowKeyStr, CacheMissVal.getMissKeyObj()); + } + } + + List cacheContent = Lists.newArrayList(); + + for(List oneRow : args){ + try { + Map sideMap = Maps.newHashMap(); + for (KeyValue keyValue : oneRow) { + String cf = new String(keyValue.family()); + String col = new String(keyValue.qualifier()); + String mapKey = cf + ":" + col; + + //The table format defined using different data type conversion byte + String colType = colRefType.get(mapKey.toUpperCase()); + Object val = HbaseUtils.convertByte(keyValue.value(), colType); + sideMap.put(mapKey.toUpperCase(), val); + } + + if (oneRow.size() > 0) { + //The order of the fields defined in the data conversion table + List sideVal = Lists.newArrayList(); + for (String key : colNames) { + Object val = sideMap.get(key); + if (val == null) { + System.out.println("can't get data with column " + key); + LOG.error("can't get data with column " + key); + } + + sideVal.add(val); + } + + Row row = fillData(input, sideVal); + if (openCache) { + cacheContent.add(sideVal); + } + + resultFuture.complete(Collections.singleton(row)); + } + } catch (Exception e) { + resultFuture.complete(null); + LOG.error("record:" + input); + LOG.error("get side record exception:", e); + } + } + + if(openCache){ + sideCache.putCache(rowKeyStr, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); + } + + return ""; + } + + private String dealFail(Object arg2, Row input, ResultFuture resultFuture){ + LOG.error("record:" + input); + LOG.error("get side record exception:" + arg2); + resultFuture.complete(null); + return ""; + } +} diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java new file mode 100644 index 000000000..5437ac8c1 --- /dev/null +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.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 com.dtstack.flink.sql.side.hbase.rowkeydealer; + +import com.dtstack.flink.sql.enums.ECacheContentType; +import com.dtstack.flink.sql.side.hbase.utils.HbaseUtils; +import com.dtstack.flink.sql.side.CacheMissVal; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.cache.AbsSideCache; +import com.dtstack.flink.sql.side.cache.CacheObj; +import com.google.common.collect.Maps; +import org.apache.calcite.sql.JoinType; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.types.Row; +import org.hbase.async.GetRequest; +import org.hbase.async.HBaseClient; +import org.hbase.async.KeyValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2018/9/10 + * Company: www.dtstack.com + * @author xuchao + */ + +public class RowKeyEqualModeDealer extends AbsRowKeyModeDealer { + + private static final Logger LOG = LoggerFactory.getLogger(RowKeyEqualModeDealer.class); + + public RowKeyEqualModeDealer(Map colRefType, String[] colNames, HBaseClient hBaseClient, + boolean openCache, JoinType joinType, List outFieldInfoList, + Map inFieldIndex, Map sideFieldIndex) { + super(colRefType, colNames, hBaseClient, openCache, joinType, outFieldInfoList, inFieldIndex, sideFieldIndex); + } + + + @Override + public void asyncGetData(String tableName, String rowKeyStr, Row input, ResultFuture resultFuture, + AbsSideCache sideCache){ + //TODO 是否有查询多个col family 和多个col的方法 + GetRequest getRequest = new GetRequest(tableName, rowKeyStr); + hBaseClient.get(getRequest).addCallbacks(arg -> { + + try{ + Map sideMap = Maps.newHashMap(); + for(KeyValue keyValue : arg){ + String cf = new String(keyValue.family()); + String col = new String(keyValue.qualifier()); + String mapKey = cf + ":" + col; + //The table format defined using different data type conversion byte + String colType = colRefType.get(mapKey.toUpperCase()); + Object val = HbaseUtils.convertByte(keyValue.value(), colType); + sideMap.put(mapKey.toUpperCase(), val); + } + + if(arg.size() > 0){ + //The order of the fields defined in the data conversion table + List sideVal = Lists.newArrayList(); + for(String key : colNames){ + Object val = sideMap.get(key); + if(val == null){ + System.out.println("can't get data with column " + key); + LOG.error("can't get data with column " + key); + } + + sideVal.add(val); + } + + Row row = fillData(input, sideVal); + if(openCache){ + sideCache.putCache(rowKeyStr, CacheObj.buildCacheObj(ECacheContentType.SingleLine, row)); + } + + resultFuture.complete(Collections.singleton(row)); + }else{ + + dealMissKey(input, resultFuture); + + if(openCache){ + sideCache.putCache(rowKeyStr, CacheMissVal.getMissKeyObj()); + } + } + }catch (Exception e){ + resultFuture.complete(null); + LOG.error("record:" + input); + LOG.error("get side record exception:", e); + } + + return ""; + }, arg2 -> { + LOG.error("record:" + input); + LOG.error("get side record exception:" + arg2); + resultFuture.complete(null); + return ""; + }); + } +} diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java new file mode 100644 index 000000000..71c1c7bad --- /dev/null +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java @@ -0,0 +1,117 @@ +/* + * 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 com.dtstack.flink.sql.side.hbase.table; + +import com.dtstack.flink.sql.table.AbsSideTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.ClassUtil; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; + +/** + * 定义的hbase 字段信息必须包含别名-->sql里面不允许出现“:” + * Date: 2018/8/21 + * Company: www.dtstack.com + * @author xuchao + */ + +public class HbaseSideParser extends AbsSideTableParser { + + private final static String FIELD_KEY = "fieldKey"; + + private final static Pattern FIELD_PATTERN = Pattern.compile("(?i)(.*)\\s+AS\\s+(\\w+)$"); + + private final static String SIDE_SIGN_KEY = "sideSignKey"; + + private final static Pattern SIDE_TABLE_SIGN = Pattern.compile("(?i)^PERIOD\\s+FOR\\s+SYSTEM_TIME$"); + + public static final String HBASE_ZOOKEEPER_QUORUM = "zookeeperQuorum"; + + public static final String ZOOKEEPER_PARENT = "zookeeperParent"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String PRE_ROW_KEY = "preRowKey"; + + + static { + keyPatternMap.put(SIDE_SIGN_KEY, SIDE_TABLE_SIGN); + keyHandlerMap.put(SIDE_SIGN_KEY, HbaseSideParser::dealSideSign); + + keyPatternMap.put(FIELD_KEY, FIELD_PATTERN); + keyHandlerMap.put(FIELD_KEY, HbaseSideParser::dealField); + } + + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + HbaseSideTableInfo hbaseTableInfo = new HbaseSideTableInfo(); + hbaseTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, hbaseTableInfo); + hbaseTableInfo.setTableName((String) props.get(TABLE_NAME_KEY.toLowerCase())); + hbaseTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase()))); + hbaseTableInfo.setHost((String) props.get(HBASE_ZOOKEEPER_QUORUM.toLowerCase())); + hbaseTableInfo.setParent((String)props.get(ZOOKEEPER_PARENT.toLowerCase())); + hbaseTableInfo.setPreRowKey(MathUtil.getBoolean(props.get(PRE_ROW_KEY.toLowerCase()), false)); + return hbaseTableInfo; + } + + private static void dealSideSign(Matcher matcher, TableInfo tableInfo){ + //FIXME 暂时不适用该标识--仅仅只是作为一个标识适用 + } + + /** + * hbase 维表的字段定义需要特殊处理 + * @param matcher + * @param tableInfo + */ + private static void dealField(Matcher matcher, TableInfo tableInfo){ + + HbaseSideTableInfo sideTableInfo = (HbaseSideTableInfo) tableInfo; + String filedDefineStr = matcher.group(1); + String aliasStr = matcher.group(2); + + String[] filedInfoArr = filedDefineStr.split("\\s+"); + if(filedInfoArr.length < 2){ + throw new RuntimeException(String.format("table [%s] field [%s] format error.", tableInfo.getName(), matcher.group(0))); + } + + //兼容可能在fieldName中出现空格的情况 + String[] filedNameArr = new String[filedInfoArr.length - 1]; + System.arraycopy(filedInfoArr, 0, filedNameArr, 0, filedInfoArr.length - 1); + String fieldName = String.join(" ", filedNameArr); + String fieldType = filedInfoArr[filedInfoArr.length - 1 ].trim(); + Class fieldClass = ClassUtil.stringConvertClass(filedInfoArr[1].trim()); + + sideTableInfo.addColumnRealName(fieldName); + sideTableInfo.addField(aliasStr); + sideTableInfo.addFieldClass(fieldClass); + sideTableInfo.addFieldType(fieldType); + sideTableInfo.putAliasNameRef(aliasStr, fieldName); + } + + +} diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideTableInfo.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideTableInfo.java new file mode 100644 index 000000000..6b8715e07 --- /dev/null +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideTableInfo.java @@ -0,0 +1,156 @@ +/* + * 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 com.dtstack.flink.sql.side.hbase.table; + +import com.dtstack.flink.sql.side.SideTableInfo; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; + +/** + * hbase 维表填写的属性 + * Date: 2018/8/21 + * Company: www.dtstack.com + * @author xuchao + */ +public class HbaseSideTableInfo extends SideTableInfo { + + private static final String CURR_TYPE = "hbase"; + + private String host; + + private String port; + + private String parent; + + private String[] rowkey; + + /**是否根据rowkey前缀查询*/ + private boolean preRowKey = false; + + private Map columnNameFamily; + + private String tableName; + + private String[] columnRealNames; + + private List columnRealNameList = Lists.newArrayList(); + + private Map aliasNameRef = Maps.newHashMap(); + + public HbaseSideTableInfo(){ + setType(CURR_TYPE); + } + + + @Override + public boolean check() { + return false; + } + + public String getHost() { + return host; + } + + public void setHost(String host) { + this.host = host; + } + + public String getPort() { + return port; + } + + public void setPort(String port) { + this.port = port; + } + + public String getParent() { + return parent; + } + + public void setParent(String parent) { + this.parent = parent; + } + + public String[] getRowkey() { + return rowkey; + } + + public void setRowkey(String[] rowkey) { + this.rowkey = rowkey; + } + + public Map getColumnNameFamily() { + return columnNameFamily; + } + + public void setColumnNameFamily(Map columnNameFamily) { + this.columnNameFamily = columnNameFamily; + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public void putAliasNameRef(String aliasName, String hbaseField){ + aliasNameRef.put(aliasName, hbaseField); + } + + public Map getAliasNameRef() { + return aliasNameRef; + } + + public String getHbaseField(String fieldAlias){ + return aliasNameRef.get(fieldAlias); + } + + public String[] getColumnRealNames() { + return columnRealNames; + } + + public void setColumnRealNames(String[] columnRealNames) { + this.columnRealNames = columnRealNames; + } + + public void addColumnRealName(String realName){ + this.columnRealNameList.add(realName); + } + + public boolean isPreRowKey() { + return preRowKey; + } + + public void setPreRowKey(boolean preRowKey) { + this.preRowKey = preRowKey; + } + + @Override + public void finish(){ + super.finish(); + this.columnRealNames = columnRealNameList.toArray(new String[columnRealNameList.size()]); + } +} diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseUtils.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseUtils.java new file mode 100644 index 000000000..39df3c913 --- /dev/null +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseUtils.java @@ -0,0 +1,74 @@ +/* + * 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 com.dtstack.flink.sql.side.hbase.utils; + +import com.dtstack.flink.sql.util.ByteUtils; +import org.apache.hadoop.hbase.util.Bytes; + +/** + * Reason: + * Date: 2018/9/10 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class HbaseUtils { + + public static Object convertByte(byte[] hbaseData, String type){ + if(type == null){ + return new String(hbaseData); + } + + switch (type.toLowerCase()) { + case "boolean": + return Bytes.toBoolean(hbaseData); + + case "int": + return Bytes.toInt(hbaseData); + + case "bigint": + return Bytes.toLong(hbaseData); + + case "tinyint": + case "byte": + return ByteUtils.bytes2Byte(hbaseData); + + case "short": + case "smallint": + return Bytes.toShort(hbaseData); + + case "char": + case "varchar": + case "string": + return Bytes.toString(hbaseData); + + case "float": + return Bytes.toFloat(hbaseData); + + case "double": + return Bytes.toDouble(hbaseData); + + } + + throw new RuntimeException("not support type of " + type); + } +} diff --git a/hbase/hbase-sink/pom.xml b/hbase/hbase-sink/pom.xml new file mode 100644 index 000000000..193128a61 --- /dev/null +++ b/hbase/hbase-sink/pom.xml @@ -0,0 +1,99 @@ + + + + sql.hbase + com.dtstack.flinkx + 1.0-SNAPSHOT + + 4.0.0 + + sql.sink.hbase + hbase-sink + + + + junit + junit + 3.8.1 + test + + + + com.dtstack.flinkx + sql.core + 1.0-SNAPSHOT + provided + + + + org.apache.hbase + hbase-client + 1.3.1 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + org.apache.hadoop:hadoop-common + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java new file mode 100644 index 000000000..a75f00da7 --- /dev/null +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java @@ -0,0 +1,209 @@ +/* + * 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 com.dtstack.flink.sql.sink.hbase; + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.ConnectionFactory; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; + +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.List; + +/** + * author: jingzhen@dtstack.com + * date: 2017-6-29 + */ +public class HbaseOutputFormat extends RichOutputFormat { + + private String host; + private String zkParent; + private String[] rowkey; + private String tableName; + private String[] columnNames; + private String[] columnTypes; + + private String[] families; + private String[] qualifiers; + + private transient org.apache.hadoop.conf.Configuration conf; + private transient Connection conn; + private transient Table table; + + public final SimpleDateFormat ROWKEY_DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss"); + public final SimpleDateFormat FIELD_DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + + @Override + public void configure(Configuration parameters) { + conf = HBaseConfiguration.create(); + conf.set("hbase.zookeeper.quorum", host); + if(zkParent != null && !"".equals(zkParent)){ + conf.set("zookeeper.znode.parent", zkParent); + } + } + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + conn = ConnectionFactory.createConnection(conf); + table = conn.getTable(TableName.valueOf(tableName)); + } + + @Override + public void writeRecord(Tuple2 tuple2) throws IOException { + + Tuple2 tupleTrans = tuple2; + Boolean retract = tupleTrans.getField(0); + if(!retract){ + //FIXME 暂时不处理hbase删除操作--->hbase要求有key,所有认为都是可以执行update查找 + return; + } + + Row record = tupleTrans.getField(1); + + List list = new ArrayList<>(); + for(int i = 0; i < rowkey.length; ++i) { + String colName = rowkey[i]; + int j = 0; + for(; j < columnNames.length; ++j) { + if(columnNames[j].equals(colName)) { + break; + } + } + if(j != columnNames.length && record.getField(i) != null) { + Object field = record.getField(j); + if(field == null ) { + list.add("null"); + } else if (field instanceof java.util.Date){ + java.util.Date d = (java.util.Date)field; + list.add(ROWKEY_DATE_FORMAT.format(d)); + } else { + list.add(field.toString()); + } + } + } + + String key = StringUtils.join(list, "-"); + Put put = new Put(key.getBytes()); + for(int i = 0; i < record.getArity(); ++i) { + Object field = record.getField(i); + byte[] val = null; + if (field != null) { + val = field.toString().getBytes(); + } + byte[] cf = families[i].getBytes(); + byte[] qualifier = qualifiers[i].getBytes(); + put.addColumn(cf, qualifier, val); + + } + + table.put(put); + + } + + @Override + public void close() throws IOException { + if(conn != null) { + conn.close(); + conn = null; + } + } + + private HbaseOutputFormat() {} + + public static HbaseOutputFormatBuilder buildHbaseOutputFormat() { + return new HbaseOutputFormatBuilder(); + } + + public static class HbaseOutputFormatBuilder { + + private HbaseOutputFormat format; + + private HbaseOutputFormatBuilder() { + format = new HbaseOutputFormat(); + } + + public HbaseOutputFormatBuilder setHost(String host) { + format.host = host; + return this; + } + + public HbaseOutputFormatBuilder setZkParent(String parent){ + format.zkParent = parent; + return this; + } + + + public HbaseOutputFormatBuilder setTable(String tableName) { + format.tableName = tableName; + return this; + } + + public HbaseOutputFormatBuilder setRowkey(String[] rowkey) { + format.rowkey = rowkey; + return this; + } + + public HbaseOutputFormatBuilder setColumnNames(String[] columnNames) { + format.columnNames = columnNames; + return this; + } + + public HbaseOutputFormatBuilder setColumnTypes(String[] columnTypes) { + format.columnTypes = columnTypes; + return this; + } + + public HbaseOutputFormat finish() { + Preconditions.checkNotNull(format.host, "zookeeperQuorum should be specified"); + Preconditions.checkNotNull(format.tableName, "tableName should be specified"); + Preconditions.checkNotNull(format.columnNames, "columnNames should be specified"); + Preconditions.checkArgument(format.columnNames.length != 0, "columnNames length should not be zero"); + + String[] families = new String[format.columnNames.length]; + String[] qualifiers = new String[format.columnNames.length]; + + for(int i = 0; i < format.columnNames.length; ++i) { + String col = format.columnNames[i]; + String[] part = col.split(":"); + families[i] = part[0]; + qualifiers[i] = part[1]; + } + + format.families = families; + format.qualifiers = qualifiers; + + return format; + } + + } + + +} diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseSink.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseSink.java new file mode 100644 index 000000000..e4a6b9422 --- /dev/null +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseSink.java @@ -0,0 +1,103 @@ +/* + * 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 com.dtstack.flink.sql.sink.hbase; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.hbase.table.HbaseTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.RetractStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + + +public class HbaseSink implements RetractStreamTableSink, IStreamSinkGener { + + protected String[] fieldNames; + TypeInformation[] fieldTypes; + protected String zookeeperQuorum; + protected String port; + protected String parent; + protected String tableName; + protected String[] rowkey; + + public HbaseSink() { + // TO DO NOTHING + } + + @Override + public HbaseSink genStreamSink(TargetTableInfo targetTableInfo) { + HbaseTableInfo hbaseTableInfo = (HbaseTableInfo) targetTableInfo; + this.zookeeperQuorum = hbaseTableInfo.getHost(); + this.port = hbaseTableInfo.getPort(); + this.parent = hbaseTableInfo.getParent(); + this.tableName = hbaseTableInfo.getTableName(); + this.rowkey = hbaseTableInfo.getRowkey(); + return this; + } + + @Override + public void emitDataStream(DataStream> dataStream) { + HbaseOutputFormat.HbaseOutputFormatBuilder builder = HbaseOutputFormat.buildHbaseOutputFormat(); + builder.setHost(this.zookeeperQuorum).setZkParent(this.parent).setTable(this.tableName); + + builder.setRowkey(rowkey); + builder.setColumnNames(fieldNames); + + HbaseOutputFormat outputFormat = builder.finish(); + RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(outputFormat); + dataStream.addSink(richSinkFunction); + } + + @Override + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + + @Override + public TupleTypeInfo> getOutputType() { + return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), getRecordType()); + } + + @Override + public TypeInformation getRecordType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } +} diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseUtil.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseUtil.java new file mode 100644 index 000000000..8f1166574 --- /dev/null +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseUtil.java @@ -0,0 +1,67 @@ +/* + * 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 com.dtstack.flink.sql.sink.hbase; + +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.hadoop.io.ByteWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.Text; + +/** + * Created by softfly on 17/6/30. + */ +public class HbaseUtil { + + private HbaseUtil() {} + + public static TypeInformation columnTypeToTypeInformation(String type) { + type = type.toUpperCase(); + + switch(type) { + case "TINYINT": + return BasicTypeInfo.getInfoFor(ByteWritable.class); + case "SMALLINT": + return BasicTypeInfo.SHORT_TYPE_INFO; + case "INT": + return BasicTypeInfo.getInfoFor(IntWritable.class); + case "BIGINT": + return BasicTypeInfo.LONG_TYPE_INFO; + case "FLOAT": + return BasicTypeInfo.FLOAT_TYPE_INFO; + case "DOUBLE": + return BasicTypeInfo.DOUBLE_TYPE_INFO; + case "TIMESTAMP": + case "DATE": + return BasicTypeInfo.DATE_TYPE_INFO; + case "STRING": + case "VARCHAR": + case "CHAR": + return BasicTypeInfo.getInfoFor(Text.class); + case "BOOLEAN": + return BasicTypeInfo.BOOLEAN_TYPE_INFO; + default: + throw new IllegalArgumentException("Unsupported type"); + } + + } + +} diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java new file mode 100644 index 000000000..686b05019 --- /dev/null +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java @@ -0,0 +1,63 @@ +/* + * 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 com.dtstack.flink.sql.sink.hbase.table; + + +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; + +public class HbaseSinkParser extends AbsTableParser { + + + public static final String HBASE_ZOOKEEPER_QUORUM = "zookeeperQuorum"; + + public static final String ZOOKEEPER_PARENT = "zookeeperParent"; + + public static final String HBASE_COLUMN_FAMILY = "columnFamily"; + + public static final String HBASE_ROWKEY = "rowkey"; + + public static final String TABLE_NAME_KEY = "tableName"; + + @Override + protected boolean fieldNameNeedsUpperCase() { + return false; + } + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + HbaseTableInfo hbaseTableInfo = new HbaseTableInfo(); + hbaseTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, hbaseTableInfo); + hbaseTableInfo.setTableName((String) props.get(TABLE_NAME_KEY.toLowerCase())); + hbaseTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase()))); + hbaseTableInfo.setHost((String) props.get(HBASE_ZOOKEEPER_QUORUM.toLowerCase())); + hbaseTableInfo.setParent((String)props.get(ZOOKEEPER_PARENT.toLowerCase())); + String rk = (String) props.get(HBASE_ROWKEY.toLowerCase()); + hbaseTableInfo.setRowkey(rk.split(",")); + return hbaseTableInfo; + } +} diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java new file mode 100644 index 000000000..3f827f102 --- /dev/null +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java @@ -0,0 +1,138 @@ +/* + * 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 com.dtstack.flink.sql.sink.hbase.table; + + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +import java.util.Map; + +public class HbaseTableInfo extends TargetTableInfo { + + private static final String CURR_TYPE = "hbase"; + + private String host; + + private String port; + + private String parent; + + private String[] rowkey; + + private Map columnNameFamily; + + private String[] columnNames; + + private String[] inputColumnTypes; + + private String[] columnTypes; + + private String tableName; + + public HbaseTableInfo(){ + setType(CURR_TYPE); + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String getHost() { + return host; + } + + public void setHost(String host) { + this.host = host; + } + + public String getPort() { + return port; + } + + public void setPort(String port) { + this.port = port; + } + + public String getParent() { + return parent; + } + + public void setParent(String parent) { + this.parent = parent; + } + + public String[] getRowkey() { + return rowkey; + } + + public void setRowkey(String[] rowkey) { + this.rowkey = rowkey; + } + + public Map getColumnNameFamily() { + return columnNameFamily; + } + + public void setColumnNameFamily(Map columnNameFamily) { + this.columnNameFamily = columnNameFamily; + } + + public String[] getColumnNames() { + return columnNames; + } + + public void setColumnNames(String[] columnNames) { + this.columnNames = columnNames; + } + + public String[] getInputColumnTypes() { + return inputColumnTypes; + } + + public void setInputColumnTypes(String[] inputColumnTypes) { + this.inputColumnTypes = inputColumnTypes; + } + + public String[] getColumnTypes() { + return columnTypes; + } + + public void setColumnTypes(String[] columnTypes) { + this.columnTypes = columnTypes; + } + + @Override + public boolean check() { + Preconditions.checkNotNull(host, "hbase field of zookeeperQuorum is required"); + return true; + } + + @Override + public String getType() { + return super.getType().toLowerCase(); + } + +} diff --git a/hbase/pom.xml b/hbase/pom.xml new file mode 100644 index 000000000..774b001ab --- /dev/null +++ b/hbase/pom.xml @@ -0,0 +1,20 @@ + + + + flink.sql + com.dtstack.flinkx + 1.0-SNAPSHOT + + 4.0.0 + + sql.hbase + pom + + hbase-sink + hbase-side + + + + \ No newline at end of file diff --git a/kafka09/kafka09-source/pom.xml b/kafka09/kafka09-source/pom.xml new file mode 100644 index 000000000..371fe7cb9 --- /dev/null +++ b/kafka09/kafka09-source/pom.xml @@ -0,0 +1,107 @@ + + + sql.kafka09 + com.dtstack.flinkx + 1.0-SNAPSHOT + + + 4.0.0 + + sql.source.kafka09 + jar + + kafka09-source + http://maven.apache.org + + + UTF-8 + 1.4.0 + + + + + junit + junit + 3.8.1 + test + + + + com.dtstack.flinkx + sql.core + 1.0-SNAPSHOT + provided + + + + org.apache.flink + flink-connector-kafka-0.9_2.11 + ${flink.version} + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + org.slf4j + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/CustomerJsonDeserialization.java new file mode 100644 index 000000000..f3f04430f --- /dev/null +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/CustomerJsonDeserialization.java @@ -0,0 +1,114 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka09; + + +import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Iterator; + +/** + * 自定义的json字符串解析 + * Date: 2017/5/28 + * Company: www.dtstack.com + * @author xuchao + */ + +public class CustomerJsonDeserialization extends AbstractDeserializationSchema { + + private static final Logger logger = LoggerFactory.getLogger(CustomerJsonDeserialization.class); + + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Type information describing the result type. */ + private final TypeInformation typeInfo; + + /** Field names to parse. Indices match fieldTypes indices. */ + private final String[] fieldNames; + + /** Types to parse fields as. Indices match fieldNames indices. */ + private final TypeInformation[] fieldTypes; + + /** Flag indicating whether to fail on a missing field. */ + private boolean failOnMissingField; + + public CustomerJsonDeserialization(TypeInformation typeInfo){ + this.typeInfo = typeInfo; + + this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); + + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + } + + @Override + public Row deserialize(byte[] message) throws IOException { + try { + JsonNode root = objectMapper.readTree(message); + Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + JsonNode node = getIgnoreCase(root, fieldNames[i]); + + if (node == null) { + if (failOnMissingField) { + throw new IllegalStateException("Failed to find field with name '" + + fieldNames[i] + "'."); + } else { + row.setField(i, null); + } + } else { + // Read the value as specified type + Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + row.setField(i, value); + } + } + + return row; + } catch (Throwable t) { + throw new IOException("Failed to deserialize JSON object.", t); + } + } + + public void setFailOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; + } + + public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { + + Iterator iter = jsonNode.fieldNames(); + while (iter.hasNext()) { + String key1 = iter.next(); + if (key1.equalsIgnoreCase(key)) { + return jsonNode.get(key1); + } + } + + return null; + + } +} diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/Kafka09Source.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/Kafka09Source.java new file mode 100644 index 000000000..643b025c7 --- /dev/null +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/Kafka09Source.java @@ -0,0 +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 com.dtstack.flink.sql.source.kafka09; + +import com.dtstack.flink.sql.source.IStreamSourceGener; +import com.dtstack.flink.sql.source.kafka09.table.Kafka09SourceTableInfo; +import com.dtstack.flink.sql.table.SourceTableInfo; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.types.Row; + +import java.util.Properties; + +/** + * If eventtime field is specified, the default time field rowtime + * Date: 2017/2/20 + * Company: www.dtstack.com + * @author xuchao + */ + +public class Kafka09Source implements IStreamSourceGener { + + /** + * Get kafka data source, you need to provide the data field names, data types + * If you do not specify auto.offset.reset, the default use groupoffset + * @param sourceTableInfo + * @return + */ + @SuppressWarnings("rawtypes") + @Override + public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { + + Kafka09SourceTableInfo kafka09SourceTableInfo = (Kafka09SourceTableInfo) sourceTableInfo; + String topicName = kafka09SourceTableInfo.getTopic(); + + Properties props = new Properties(); + props.setProperty("bootstrap.servers", kafka09SourceTableInfo.getBootstrapServers()); + props.setProperty("auto.offset.reset", kafka09SourceTableInfo.getOffsetReset()); + //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) + + TypeInformation[] types = new TypeInformation[kafka09SourceTableInfo.getFields().length]; + for(int i = 0; i< kafka09SourceTableInfo.getFieldClasses().length; i++){ + types[i] = TypeInformation.of(kafka09SourceTableInfo.getFieldClasses()[i]); + } + + TypeInformation typeInformation = new RowTypeInfo(types, kafka09SourceTableInfo.getFields()); + FlinkKafkaConsumer09 kafkaSrc = new FlinkKafkaConsumer09(topicName, + new CustomerJsonDeserialization(typeInformation), props); + + //earliest,latest + if("earliest".equalsIgnoreCase(kafka09SourceTableInfo.getOffsetReset())){ + kafkaSrc.setStartFromEarliest(); + }else{ + kafkaSrc.setStartFromLatest(); + } + + String fields = StringUtils.join(kafka09SourceTableInfo.getFields(), ","); + return tableEnv.fromDataStream(env.addSource(kafkaSrc, typeInformation), fields); + } +} diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/table/Kafka09SourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/table/Kafka09SourceParser.java new file mode 100644 index 000000000..51fccd660 --- /dev/null +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/table/Kafka09SourceParser.java @@ -0,0 +1,51 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka09.table; + +import com.dtstack.flink.sql.table.AbsSourceParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/7/4 + * Company: www.dtstack.com + * @author xuchao + */ + +public class Kafka09SourceParser extends AbsSourceParser { + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + + Kafka09SourceTableInfo kafka09SourceTableInfo = new Kafka09SourceTableInfo(); + kafka09SourceTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka09SourceTableInfo); + + kafka09SourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(Kafka09SourceTableInfo.PARALLELISM_KEY.toLowerCase()))); + kafka09SourceTableInfo.setBootstrapServers(MathUtil.getString(props.get(Kafka09SourceTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase()))); + kafka09SourceTableInfo.setGroupId(MathUtil.getString(props.get(Kafka09SourceTableInfo.GROUPID_KEY.toLowerCase()))); + kafka09SourceTableInfo.setTopic(MathUtil.getString(props.get(Kafka09SourceTableInfo.TOPIC_KEY.toLowerCase()))); + return kafka09SourceTableInfo; + } +} diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/table/Kafka09SourceTableInfo.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/table/Kafka09SourceTableInfo.java new file mode 100644 index 000000000..94691adca --- /dev/null +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/table/Kafka09SourceTableInfo.java @@ -0,0 +1,120 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka09.table; + +import com.dtstack.flink.sql.table.SourceTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * Reason: + * Date: 2018/6/22 + * Company: www.dtstack.com + * @author xuchao + */ + +public class Kafka09SourceTableInfo extends SourceTableInfo { + + //version + private static final String CURR_TYPE = "kafka09"; + + public static final String BOOTSTRAPSERVERS_KEY = "bootstrapServers"; + + public static final String TOPIC_KEY = "topic"; + + public static final String GROUPID_KEY = "groupId"; + + private String bootstrapServers; + + private String topic; + + private String groupId; + + //latest, earliest + private String offsetReset = "latest"; + + private String offset; + + public Kafka09SourceTableInfo(){ + super.setType(CURR_TYPE); + } + + + public String getBootstrapServers() { + return bootstrapServers; + } + + public void setBootstrapServers(String bootstrapServers) { + this.bootstrapServers = bootstrapServers; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public String getGroupId() { + return groupId; + } + + public void setGroupId(String groupId) { + this.groupId = groupId; + } + + public String getOffsetReset() { + return offsetReset; + } + + public void setOffsetReset(String offsetReset) { + if(offsetReset == null){ + return; + } + + this.offsetReset = offsetReset; + } + + public String getOffset() { + return offset; + } + + public void setOffset(String offset) { + this.offset = offset; + } + + @Override + public boolean check() { + Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(topic, "kafka of topic is required"); + Preconditions.checkNotNull(groupId, "kafka of groupId is required"); + Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") + || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + + return false; + } + + @Override + public String getType() { +// return super.getType() + SOURCE_SUFFIX; + return super.getType(); + } +} diff --git a/kafka09/kafka09-source/src/test/java/com/dtstack/flinkx/AppTest.java b/kafka09/kafka09-source/src/test/java/com/dtstack/flinkx/AppTest.java new file mode 100644 index 000000000..33a0233ac --- /dev/null +++ b/kafka09/kafka09-source/src/test/java/com/dtstack/flinkx/AppTest.java @@ -0,0 +1,58 @@ +/* + * 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 com.dtstack.flinkx; + +import junit.framework.Test; +import junit.framework.TestCase; +import junit.framework.TestSuite; + +/** + * Unit test for simple App. + */ +public class AppTest + extends TestCase +{ + /** + * Create the test case + * + * @param testName name of the test case + */ + public AppTest( String testName ) + { + super( testName ); + } + + /** + * @return the suite of tests being tested + */ + public static Test suite() + { + return new TestSuite( AppTest.class ); + } + + /** + * Rigourous Test :-) + */ + public void testApp() + { + assertTrue( true ); + } +} diff --git a/kafka09/pom.xml b/kafka09/pom.xml new file mode 100644 index 000000000..c9a2134e8 --- /dev/null +++ b/kafka09/pom.xml @@ -0,0 +1,21 @@ + + + + flink.sql + com.dtstack.flinkx + 1.0-SNAPSHOT + + + 4.0.0 + + sql.kafka09 + pom + + + kafka09-source + + + + \ No newline at end of file diff --git a/launcher/job/sideSql.txt b/launcher/job/sideSql.txt new file mode 100644 index 000000000..35a4be1e7 --- /dev/null +++ b/launcher/job/sideSql.txt @@ -0,0 +1,72 @@ +CREATE TABLE MyTable( + name string, + channel STRING, + pv INT, + xctime bigint, + CHARACTER_LENGTH(channel) AS timeLeng + )WITH( + type ='kafka09', + bootstrapServers ='172.16.8.198:9092', + zookeeperQuorum ='172.16.8.198:2181/kafka', + offsetReset ='latest', + topic ='nbTest1', + parallelism ='1' + ); + +CREATE TABLE MyResult( + channel VARCHAR, + pv VARCHAR + )WITH( + type ='mysql', + url ='jdbc:mysql://172.16.8.104:3306/test?charset=utf8', + userName ='dtstack', + password ='abc123', + tableName ='pv2', + parallelism ='1' + ); + +CREATE TABLE workerinfo( + cast(logtime as TIMESTAMP)AS rtime, + cast(logtime)AS rtime + )WITH( + type ='hbase', + zookeeperQuorum ='rdos1:2181', + tableName ='workerinfo', + rowKey ='ce,de', + parallelism ='1', + zookeeperParent ='/hbase' + ); + +CREATE TABLE sideTable( + cf:name String as name, + cf:info String as info, + PRIMARY KEY(name), + PERIOD FOR SYSTEM_TIME + )WITH( + type ='hbase', + zookeeperQuorum ='rdos1:2181', + zookeeperParent ='/hbase', + tableName ='workerinfo', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1' + ); + +insert +into + MyResult + select + d.channel, + d.info + from + ( select + a.*,b.info + from + MyTable a + join + sideTable b + on a.channel=b.name + where + a.channel = 'xc2' + and a.pv=10 ) as d diff --git a/launcher/pom.xml b/launcher/pom.xml new file mode 100644 index 000000000..78b2f6e6a --- /dev/null +++ b/launcher/pom.xml @@ -0,0 +1,34 @@ + + + + flink.sql + com.dtstack.flinkx + 1.0-SNAPSHOT + + 4.0.0 + + sql.launcher + + + + com.dtstack.flinkx + sql.core + 1.0-SNAPSHOT + + + + org.apache.flink + flink-yarn_2.11 + ${flink.version} + + + + log4j + log4j + 1.2.17 + + + + \ No newline at end of file diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java new file mode 100644 index 000000000..960398906 --- /dev/null +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -0,0 +1,210 @@ +/** + * 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 com.dtstack.flink.sql.launcher; + +import org.apache.commons.lang.StringUtils; +import org.apache.flink.client.deployment.StandaloneClusterDescriptor; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.StandaloneClusterClient; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.yarn.AbstractYarnClusterDescriptor; +import org.apache.flink.yarn.YarnClusterClient; +import org.apache.flink.yarn.YarnClusterDescriptor; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +import java.io.File; +import java.io.FilenameFilter; +import java.lang.reflect.Field; +import java.util.EnumSet; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +import static com.dtstack.flink.sql.launcher.LauncherOptions.*; + +/** + * The Factory of ClusterClient + * + * Company: www.dtstack.com + * @author huyifan.zju@163.com + */ +public class ClusterClientFactory { + + public static ClusterClient createClusterClient(Properties props) { + String clientType = props.getProperty(OPTION_MODE); + if(clientType.equals(ClusterMode.MODE_STANDALONE)) { + return createStandaloneClient(props); + } else if(clientType.equals(ClusterMode.MODE_YARN)) { + return createYarnClient(props); + } + throw new IllegalArgumentException("Unsupported cluster client type: "); + } + + public static StandaloneClusterClient createStandaloneClient(Properties props) { + String flinkConfDir = props.getProperty(LauncherOptions.OPTION_FLINK_CONF_DIR); + Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); + StandaloneClusterDescriptor descriptor = new StandaloneClusterDescriptor(config); + StandaloneClusterClient clusterClient = descriptor.retrieve(null); + clusterClient.setDetached(true); + return clusterClient; + } + + public static YarnClusterClient createYarnClient(Properties props) { + String flinkConfDir = props.getProperty(LauncherOptions.OPTION_FLINK_CONF_DIR); + Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); + String yarnConfDir = props.getProperty(LauncherOptions.OPTION_YARN_CONF_DIR); + org.apache.hadoop.conf.Configuration yarnConf = new YarnConfiguration(); + if(StringUtils.isNotBlank(yarnConfDir)) { + try { + + config.setString(ConfigConstants.PATH_HADOOP_CONFIG, yarnConfDir); + FileSystem.initialize(config); + + File dir = new File(yarnConfDir); + if(dir.exists() && dir.isDirectory()) { + File[] xmlFileList = new File(yarnConfDir).listFiles(new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + if(name.endsWith(".xml")){ + return true; + } + return false; + } + }); + if(xmlFileList != null) { + for(File xmlFile : xmlFileList) { + yarnConf.addResource(xmlFile.toURI().toURL()); + } + } + + YarnClient yarnClient = YarnClient.createYarnClient(); + yarnClient.init(yarnConf); + yarnClient.start(); + String applicationId = null; + + Set set = new HashSet<>(); + set.add("Apache Flink"); + EnumSet enumSet = EnumSet.noneOf(YarnApplicationState.class); + enumSet.add(YarnApplicationState.RUNNING); + List reportList = yarnClient.getApplications(set, enumSet); + + int maxMemory = -1; + int maxCores = -1; + for(ApplicationReport report : reportList) { + if(!report.getName().startsWith("Flink session")){ + continue; + } + + if(!report.getYarnApplicationState().equals(YarnApplicationState.RUNNING)) { + continue; + } + + int thisMemory = report.getApplicationResourceUsageReport().getNeededResources().getMemory(); + int thisCores = report.getApplicationResourceUsageReport().getNeededResources().getVirtualCores(); + if(thisMemory > maxMemory || thisMemory == maxMemory && thisCores > maxCores) { + maxMemory = thisMemory; + maxCores = thisCores; + applicationId = report.getApplicationId().toString(); + } + + } + + if(org.apache.commons.lang3.StringUtils.isEmpty(applicationId)) { + throw new RuntimeException("No flink session found on yarn cluster."); + } + + yarnClient.stop(); + + AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor(config, "."); + Field confField = AbstractYarnClusterDescriptor.class.getDeclaredField("conf"); + confField.setAccessible(true); + haYarnConf(yarnConf); + confField.set(clusterDescriptor, yarnConf); + + YarnClusterClient clusterClient = clusterDescriptor.retrieve(applicationId); + clusterClient.setDetached(true); + return clusterClient; + } + } catch(Exception e) { + throw new RuntimeException(e); + } + } + + + + throw new UnsupportedOperationException("Haven't been developed yet!"); + } + + /** + * 处理yarn HA的配置项 + */ + private static org.apache.hadoop.conf.Configuration haYarnConf(org.apache.hadoop.conf.Configuration yarnConf) { + Iterator> iterator = yarnConf.iterator(); + while(iterator.hasNext()) { + Map.Entry entry = iterator.next(); + String key = entry.getKey(); + String value = entry.getValue(); + if(key.startsWith("yarn.resourcemanager.hostname.")) { + String rm = key.substring("yarn.resourcemanager.hostname.".length()); + String addressKey = "yarn.resourcemanager.address." + rm; + if(yarnConf.get(addressKey) == null) { + yarnConf.set(addressKey, value + ":" + YarnConfiguration.DEFAULT_RM_PORT); + } + } + } + return yarnConf; + } + + private static org.apache.hadoop.conf.Configuration getYarnConf(String yarnConfDir) { + org.apache.hadoop.conf.Configuration yarnConf = new YarnConfiguration(); + try { + + File dir = new File(yarnConfDir); + if(dir.exists() && dir.isDirectory()) { + File[] xmlFileList = new File(yarnConfDir).listFiles(new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + if(name.endsWith(".xml")){ + return true; + } + return false; + } + }); + if(xmlFileList != null) { + for(File xmlFile : xmlFileList) { + yarnConf.addResource(xmlFile.toURI().toURL()); + } + } + } + } catch(Exception e) { + throw new RuntimeException(e); + } + return yarnConf; + } + +} diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterMode.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterMode.java new file mode 100644 index 000000000..590aba48e --- /dev/null +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterMode.java @@ -0,0 +1,35 @@ +/** + * 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 com.dtstack.flink.sql.launcher; + +/** + * This class defines three running mode of FlinkX + * + * Company: www.dtstack.com + * @author huyifan.zju@163.com + */ +public class ClusterMode { + + public static final String MODE_LOCAL = "local"; + + public static final String MODE_STANDALONE = "standalone"; + + public static final String MODE_YARN = "yarn"; + +} diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java new file mode 100644 index 000000000..786a44de1 --- /dev/null +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -0,0 +1,64 @@ +/* + * 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 com.dtstack.flink.sql.launcher; + +import avro.shaded.com.google.common.collect.Lists; +import com.dtstack.flink.sql.Main; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.PackagedProgram; + +import java.io.File; +import java.util.List; + +import static com.dtstack.flink.sql.launcher.ClusterMode.MODE_LOCAL; +import static com.dtstack.flink.sql.launcher.LauncherOptions.OPTION_LOCAL_SQL_PLUGIN_PATH; +import static com.dtstack.flink.sql.launcher.LauncherOptions.OPTION_MODE; + +public class LauncherMain { + + private static final String CORE_JAR = "core.jar"; + + private static String SP = File.separator; + + + private static String getLocalCoreJarPath(String localSqlRootJar){ + return localSqlRootJar + SP + CORE_JAR; + } + + public static void main(String[] args) throws Exception { + LauncherOptionParser optionParser = new LauncherOptionParser(args); + String mode = (String) optionParser.getVal(OPTION_MODE); + List argList = optionParser.getProgramExeArgList(); + + if(mode.equals(MODE_LOCAL)) { + String[] localArgs = argList.toArray(new String[argList.size()]); + Main.main(localArgs); + } else { + ClusterClient clusterClient = ClusterClientFactory.createClusterClient(optionParser.getProperties()); + String pluginRoot = (String) optionParser.getVal(OPTION_LOCAL_SQL_PLUGIN_PATH); + File jarFile = new File(getLocalCoreJarPath(pluginRoot)); + String[] remoteArgs = argList.toArray(new String[argList.size()]); + PackagedProgram program = new PackagedProgram(jarFile, Lists.newArrayList(), remoteArgs); + clusterClient.run(program, 1); + clusterClient.shutdown(); + } + } +} diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java new file mode 100644 index 000000000..1a6a1043e --- /dev/null +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java @@ -0,0 +1,150 @@ +/** + * 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 com.dtstack.flink.sql.launcher; + +import avro.shaded.com.google.common.collect.Lists; +import org.apache.commons.cli.BasicParser; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Options; +import org.apache.commons.lang.StringUtils; +import org.apache.flink.hadoop.shaded.com.google.common.base.Charsets; +import org.apache.flink.hadoop.shaded.com.google.common.base.Preconditions; + +import java.io.File; +import java.io.FileInputStream; +import java.net.URLEncoder; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static com.dtstack.flink.sql.launcher.LauncherOptions.*; +import static com.dtstack.flink.sql.launcher.ClusterMode.*; + + +/** + * The Parser of Launcher commandline options + * + * Company: www.dtstack.com + * @author huyifan.zju@163.com + */ +public class LauncherOptionParser { + + private Options options = new Options(); + + private BasicParser parser = new BasicParser(); + + private Properties properties = new Properties(); + + public LauncherOptionParser(String[] args) { + options.addOption(LauncherOptions.OPTION_MODE, true, "Running mode"); + options.addOption(OPTION_SQL, true, "Job sql file"); + options.addOption(OPTION_NAME, true, "Job name"); + options.addOption(OPTION_FLINK_CONF_DIR, true, "Flink configuration directory"); + options.addOption(OPTION_LOCAL_SQL_PLUGIN_PATH, true, "sql local plugin root"); + options.addOption(OPTION_REMOTE_SQL_PLUGIN_PATH, true, "sql remote plugin root"); + options.addOption(OPTION_ADDJAR, true, "sql ext jar,eg udf jar"); + options.addOption(OPTION_CONF_PROP, true, "sql ref prop,eg specify event time"); + options.addOption(OPTION_YARN_CONF_DIR, true, "Yarn and hadoop configuration directory"); + + try { + CommandLine cl = parser.parse(options, args); + String mode = cl.getOptionValue(OPTION_MODE, MODE_LOCAL); + //check mode + properties.put(OPTION_MODE, mode); + + String job = Preconditions.checkNotNull(cl.getOptionValue(OPTION_SQL), + "Must specify job file using option '" + OPTION_SQL + "'"); + File file = new File(job); + FileInputStream in = new FileInputStream(file); + byte[] filecontent = new byte[(int) file.length()]; + in.read(filecontent); + String content = new String(filecontent, "UTF-8"); + String sql = URLEncoder.encode(content, Charsets.UTF_8.name()); + properties.put(OPTION_SQL, sql); + + String localPlugin = Preconditions.checkNotNull(cl.getOptionValue(OPTION_LOCAL_SQL_PLUGIN_PATH)); + properties.put(OPTION_LOCAL_SQL_PLUGIN_PATH, localPlugin); + + String remotePlugin = Preconditions.checkNotNull(cl.getOptionValue(OPTION_REMOTE_SQL_PLUGIN_PATH)); + properties.put(OPTION_REMOTE_SQL_PLUGIN_PATH, remotePlugin); + + String name = Preconditions.checkNotNull(cl.getOptionValue(OPTION_NAME)); + properties.put(OPTION_NAME, name); + + String addJar = cl.getOptionValue(OPTION_ADDJAR); + if(StringUtils.isNotBlank(addJar)){ + properties.put(OPTION_ADDJAR, addJar); + } + + String confProp = cl.getOptionValue(OPTION_CONF_PROP); + if(StringUtils.isNotBlank(confProp)){ + properties.put(OPTION_CONF_PROP, confProp); + } + + String flinkConfDir = cl.getOptionValue(OPTION_FLINK_CONF_DIR); + if(StringUtils.isNotBlank(flinkConfDir)) { + properties.put(OPTION_FLINK_CONF_DIR, flinkConfDir); + } + + String yarnConfDir = cl.getOptionValue(OPTION_YARN_CONF_DIR); + if(StringUtils.isNotBlank(yarnConfDir)) { + properties.put(OPTION_YARN_CONF_DIR, yarnConfDir); + } + + } catch (Exception e) { + throw new RuntimeException(e); + } + + } + + public Properties getProperties(){ + return properties; + } + + public Object getVal(String key){ + return properties.get(key); + } + + public List getAllArgList(){ + List args = Lists.newArrayList(); + for(Map.Entry one : properties.entrySet()){ + args.add("-" + one.getKey().toString()); + args.add(one.getValue().toString()); + } + + return args; + } + + public List getProgramExeArgList(){ + List args = Lists.newArrayList(); + for(Map.Entry one : properties.entrySet()){ + String key = one.getKey().toString(); + if(OPTION_FLINK_CONF_DIR.equalsIgnoreCase(key) + || OPTION_YARN_CONF_DIR.equalsIgnoreCase(key)){ + continue; + } + + args.add("-" + key); + args.add(one.getValue().toString()); + } + + return args; + } + +} diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java new file mode 100644 index 000000000..2169bb698 --- /dev/null +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java @@ -0,0 +1,48 @@ +/** + * 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 com.dtstack.flink.sql.launcher; + +/** + * This class define commandline options for the Launcher program + * + * Company: www.dtstack.com + * @author huyifan.zju@163.com + */ +public class LauncherOptions { + + public static final String OPTION_MODE = "mode"; + + public static final String OPTION_NAME = "name"; + + public static final String OPTION_SQL = "sql"; + + public static final String OPTION_FLINK_CONF_DIR = "flinkconf"; + + public static final String OPTION_YARN_CONF_DIR = "yarnconf"; + + public static final String OPTION_LOCAL_SQL_PLUGIN_PATH = "localSqlPluginPath"; + + public static final String OPTION_REMOTE_SQL_PLUGIN_PATH = "remoteSqlPluginPath"; + + public static final String OPTION_ADDJAR = "addjar"; + + public static final String OPTION_CONF_PROP = "confProp"; + + +} diff --git a/mysql/mysql-side/pom.xml b/mysql/mysql-side/pom.xml new file mode 100644 index 000000000..b8e2d92fc --- /dev/null +++ b/mysql/mysql-side/pom.xml @@ -0,0 +1,108 @@ + + + + sql.mysql + com.dtstack.flinkx + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.mysql + mysql-side + + + com.dtstack.flinkx + sql.core + 1.0-SNAPSHOT + provided + + + + + + io.vertx + vertx-jdbc-client + 3.5.2 + + + + io.vertx + vertx-core + 3.5.2 + + + + mysql + mysql-connector-java + 5.1.46 + + + + jar + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java new file mode 100644 index 000000000..712b5673b --- /dev/null +++ b/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -0,0 +1,340 @@ +/* + * 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 com.dtstack.flink.sql.side.mysql; + +import com.dtstack.flink.sql.enums.ECacheContentType; +import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; +import com.dtstack.flink.sql.side.AsyncReqRow; +import com.dtstack.flink.sql.side.CacheMissVal; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.cache.CacheObj; +import io.vertx.core.Vertx; +import io.vertx.core.VertxOptions; +import io.vertx.core.json.JsonArray; +import io.vertx.core.json.JsonObject; +import io.vertx.ext.jdbc.JDBCClient; +import io.vertx.ext.sql.SQLClient; +import io.vertx.ext.sql.SQLConnection; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Timestamp; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Mysql dim table + * Date: 2018/7/27 + * Company: www.dtstack.com + * @author xuchao + */ + +public class MysqlAsyncReqRow extends AsyncReqRow { + + private static final long serialVersionUID = 2098635244857937717L; + + private static final Logger LOG = LoggerFactory.getLogger(MysqlAsyncReqRow.class); + + private transient SQLClient mySQLClient; + + private final static String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; + + //TODO 这三个参数应该如何设置? + private final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 10; + + private final static int DEFAULT_VERTX_WORKER_POOL_SIZE = 20; + + private final static int DEFAULT_MAX_DB_CONN_POOL_SIZE = 20; + + + public MysqlAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void parseSelectFields(JoinInfo joinInfo){ + String sideTableName = joinInfo.getSideTableName(); + String nonSideTableName = joinInfo.getNonSideTable(); + List fields = Lists.newArrayList(); + + int sideIndex = 0; + for( int i=0; i sqlNodeList = Lists.newArrayList(); + if(conditionNode.getKind() == SqlKind.AND){ + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); + }else{ + sqlNodeList.add(conditionNode); + } + + for(SqlNode sqlNode : sqlNodeList){ + dealOneEqualCon(sqlNode, sideTableName); + } + + sqlCondition = "select ${selectField} from ${tableName} where "; + for(int i=0; i resultFuture) throws Exception { + + JsonArray inputParams = new JsonArray(); + for(Integer conValIndex : equalValIndex){ + Object equalObj = input.getField(conValIndex); + if(equalObj == null){ + resultFuture.complete(null); + } + + inputParams.add(equalObj); + } + + String key = buildCacheKey(inputParams); + if(openCache()){ + CacheObj val = getFromCache(key); + if(val != null){ + + if(ECacheContentType.MissVal == val.getType()){ + dealMissKey(input, resultFuture); + return; + }else if(ECacheContentType.MultiLine == val.getType()){ + + for(Object jsonArray : (List)val.getContent()){ + Row row = fillData(input, jsonArray); + resultFuture.complete(Collections.singleton(row)); + } + + }else{ + throw new RuntimeException("not support cache obj type " + val.getType()); + } + return; + } + } + + mySQLClient.getConnection(conn -> { + if (conn.failed()) { + //处理失败情况 + resultFuture.completeExceptionally(conn.cause()); + return; + } + + final SQLConnection connection = conn.result(); + connection.queryWithParams(sqlCondition, inputParams, rs -> { + if (rs.failed()) { + LOG.error("Cannot retrieve the data from the database"); + LOG.error("", rs.cause()); + resultFuture.complete(null); + return; + } + + List cacheContent = Lists.newArrayList(); + + int resultSize = rs.result().getResults().size(); + if(resultSize > 0){ + for (JsonArray line : rs.result().getResults()) { + Row row = fillData(input, line); + if(openCache()){ + cacheContent.add(line); + } + resultFuture.complete(Collections.singleton(row)); + } + + if(openCache()){ + putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); + } + }else{ + dealMissKey(input, resultFuture); + if(openCache()){ + putCache(key, CacheMissVal.getMissKeyObj()); + } + } + + // and close the connection + connection.close(done -> { + if (done.failed()) { + throw new RuntimeException(done.cause()); + } + }); + }); + }); + } + + @Override + public Row fillData(Row input, Object line){ + JsonArray jsonArray = (JsonArray) line; + Row row = new Row(outFieldInfoList.size()); + for(Map.Entry entry : inFieldIndex.entrySet()){ + Object obj = input.getField(entry.getValue()); + if(obj instanceof Timestamp){ + obj = ((Timestamp)obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for(Map.Entry entry : sideFieldIndex.entrySet()){ + if(jsonArray == null){ + row.setField(entry.getKey(), null); + }else{ + row.setField(entry.getKey(), jsonArray.getValue(entry.getValue())); + } + } + + return row; + } + + @Override + public void close() throws Exception { + super.close(); + mySQLClient.close(); + } + + public String buildCacheKey(JsonArray jsonArray){ + StringBuilder sb = new StringBuilder(); + for(Object ele : jsonArray.getList()){ + sb.append(ele.toString()) + .append("_"); + } + + return sb.toString(); + } + +} diff --git a/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java b/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java new file mode 100644 index 000000000..13a4bef39 --- /dev/null +++ b/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java @@ -0,0 +1,67 @@ +/* + * 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 com.dtstack.flink.sql.side.mysql.table; + +import com.dtstack.flink.sql.table.AbsSideTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/7/25 + * Company: www.dtstack.com + * @author xuchao + */ + +public class MysqlSideParser extends AbsSideTableParser { + + private final static String SIDE_SIGN_KEY = "sideSignKey"; + + private final static Pattern SIDE_TABLE_SIGN = Pattern.compile("(?i)^PERIOD\\s+FOR\\s+SYSTEM_TIME$"); + + static { + keyPatternMap.put(SIDE_SIGN_KEY, SIDE_TABLE_SIGN); + keyHandlerMap.put(SIDE_SIGN_KEY, MysqlSideParser::dealSideSign); + } + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + MysqlSideTableInfo mysqlTableInfo = new MysqlSideTableInfo(); + mysqlTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, mysqlTableInfo); + + parseCacheProp(mysqlTableInfo, props); + mysqlTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(MysqlSideTableInfo.PARALLELISM_KEY.toLowerCase()))); + mysqlTableInfo.setUrl(MathUtil.getString(props.get(MysqlSideTableInfo.URL_KEY.toLowerCase()))); + mysqlTableInfo.setTableName(MathUtil.getString(props.get(MysqlSideTableInfo.TABLE_NAME_KEY.toLowerCase()))); + mysqlTableInfo.setUserName(MathUtil.getString(props.get(MysqlSideTableInfo.USER_NAME_KEY.toLowerCase()))); + mysqlTableInfo.setPassword(MathUtil.getString(props.get(MysqlSideTableInfo.PASSWORD_KEY.toLowerCase()))); + + return mysqlTableInfo; + } + + private static void dealSideSign(Matcher matcher, TableInfo tableInfo){ + } +} diff --git a/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java b/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java new file mode 100644 index 000000000..72ac0dc55 --- /dev/null +++ b/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + + +package com.dtstack.flink.sql.side.mysql.table; + +import com.dtstack.flink.sql.side.SideTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * Reason: + * Date: 2018/7/25 + * Company: www.dtstack.com + * @author xuchao + */ + +public class MysqlSideTableInfo extends SideTableInfo { + + private static final long serialVersionUID = -1L; + + private static final String CURR_TYPE = "mysql"; + + public static final String URL_KEY = "url"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String USER_NAME_KEY = "userName"; + + public static final String PASSWORD_KEY = "password"; + + public MysqlSideTableInfo(){ + setType(CURR_TYPE); + } + + @Override + public boolean check() { + Preconditions.checkNotNull(url, "mysql of URL is required"); + Preconditions.checkNotNull(tableName, "mysql of tableName is required"); + Preconditions.checkNotNull(userName, "mysql of userName is required"); + Preconditions.checkNotNull(password, "mysql of password is required"); + return true; + } + + private String url; + + private String tableName; + + private String userName; + + private String password; + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } +} diff --git a/mysql/mysql-sink/pom.xml b/mysql/mysql-sink/pom.xml new file mode 100644 index 000000000..7df86cfe6 --- /dev/null +++ b/mysql/mysql-sink/pom.xml @@ -0,0 +1,111 @@ + + + sql.mysql + com.dtstack.flinkx + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.mysql + jar + + mysql-sink + http://maven.apache.org + + + UTF-8 + 1.4.0 + + + + + junit + junit + 3.8.1 + test + + + + com.dtstack.flinkx + sql.core + 1.0-SNAPSHOT + provided + + + + org.apache.flink + flink-jdbc + ${flink.version} + + + + mysql + mysql-connector-java + 5.1.17 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java new file mode 100644 index 000000000..26ab2da73 --- /dev/null +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java @@ -0,0 +1,176 @@ +/* + * 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 com.dtstack.flink.sql.sink.mysql; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.table.sinks.RetractStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +import java.sql.Types; +import java.util.List; + +/** + * Date: 2017/2/27 + * Company: www.dtstack.com + * @author xuchao + */ + +public abstract class DBSink implements RetractStreamTableSink { + + protected String driverName; + + protected String dbURL; + + protected String userName; + + protected String password; + + protected int batchInterval = 1; + + protected int[] sqlTypes; + + protected String tableName; + + protected String sql; + + protected List primaryKeys; + + protected String[] fieldNames; + + private TypeInformation[] fieldTypes; + + private int parallelism = -1; + + public RichSinkFunction createJdbcSinkFunc(){ + + if(driverName == null || dbURL == null || userName == null + || password == null || sqlTypes == null || tableName == null){ + throw new RuntimeException("any of params in(driverName, dbURL, userName, password, type, tableName) " + + " must not be null. please check it!!!"); + } + + RetractJDBCOutputFormat.JDBCOutputFormatBuilder jdbcFormatBuild = RetractJDBCOutputFormat.buildJDBCOutputFormat(); + jdbcFormatBuild.setDBUrl(dbURL); + jdbcFormatBuild.setDrivername(driverName); + jdbcFormatBuild.setUsername(userName); + jdbcFormatBuild.setPassword(password); + jdbcFormatBuild.setInsertQuery(sql); + jdbcFormatBuild.setBatchInterval(batchInterval); + jdbcFormatBuild.setSqlTypes(sqlTypes); + RetractJDBCOutputFormat outputFormat = jdbcFormatBuild.finish(); + + OutputFormatSinkFunction outputFormatSinkFunc = new OutputFormatSinkFunction(outputFormat); + return outputFormatSinkFunc; + } + + /** + * By now specified class type conversion. + * FIXME Follow-up has added a new type of time needs to be modified + * @param fieldTypeArray + */ + protected void buildSqlTypes(List fieldTypeArray){ + + int[] tmpFieldsType = new int[fieldTypeArray.size()]; + for(int i=0; i> dataStream) { + RichSinkFunction richSinkFunction = createJdbcSinkFunc(); + DataStreamSink streamSink = dataStream.addSink(richSinkFunction); + if(parallelism > 0){ + streamSink.setParallelism(parallelism); + } + } + + @Override + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + + @Override + public TupleTypeInfo> getOutputType() { + return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), getRecordType()); + } + + @Override + public TypeInformation getRecordType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + + public void setParallelism(int parallelism){ + this.parallelism = parallelism; + } + + public void buildSql(String tableName, List fields){ + throw new RuntimeException("you need to overwrite this method in your own class."); + } +} diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java new file mode 100644 index 000000000..ba6eeb7f2 --- /dev/null +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java @@ -0,0 +1,98 @@ +/* + * 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 com.dtstack.flink.sql.sink.mysql; + + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.mysql.table.MysqlTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; + +import java.util.Arrays; +import java.util.List; + +/** + * Date: 2017/2/27 + * Company: www.dtstack.com + * @author xuchao + */ + +public class MysqlSink extends DBSink implements IStreamSinkGener { + + public MysqlSink(){ + } + + @Override + public void buildSql(String tableName, List fields){ + buildInsertSql(tableName, fields); + } + + private void buildInsertSql(String tableName, List fields){ + String sqlTmp = "replace into " + tableName + " (${fields}) values (${placeholder})"; + String fieldsStr = ""; + String placeholder = ""; + + for(String fieldName : fields){ + fieldsStr += ",`" + fieldName + "`"; + placeholder += ",?"; + } + + fieldsStr = fieldsStr.replaceFirst(",", ""); + placeholder = placeholder.replaceFirst(",", ""); + + sqlTmp = sqlTmp.replace("${fields}", fieldsStr).replace("${placeholder}", placeholder); + this.sql = sqlTmp; + } + + @Override + public MysqlSink genStreamSink(TargetTableInfo targetTableInfo) { + + MysqlTableInfo mysqlTableInfo = (MysqlTableInfo) targetTableInfo; + + String tmpDbURL = mysqlTableInfo.getUrl(); + String tmpUserName = mysqlTableInfo.getUserName(); + String tmpPassword = mysqlTableInfo.getPassword(); + String tmpTableName = mysqlTableInfo.getTableName(); + + Integer tmpSqlBatchSize = mysqlTableInfo.getBatchSize(); + if(tmpSqlBatchSize != null){ + setBatchInterval(tmpSqlBatchSize); + } + + Integer tmpSinkParallelism = mysqlTableInfo.getParallelism(); + if(tmpSinkParallelism != null){ + setParallelism(tmpSinkParallelism); + } + + List fields = Arrays.asList(mysqlTableInfo.getFields()); + List fieldTypeArray = Arrays.asList(mysqlTableInfo.getFieldClasses()); + + this.driverName = "com.mysql.jdbc.Driver"; + this.dbURL = tmpDbURL; + this.userName = tmpUserName; + this.password = tmpPassword; + this.tableName = tmpTableName; + this.primaryKeys = mysqlTableInfo.getPrimaryKeys(); + buildSql(tableName, fields); + buildSqlTypes(fieldTypeArray); + return this; + } + +} diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java new file mode 100644 index 000000000..3a6d76364 --- /dev/null +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java @@ -0,0 +1,354 @@ +/* + * 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. + */ + + + +/* + * 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 com.dtstack.flink.sql.sink.mysql; + +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.SQLException; + +/** + * OutputFormat to write tuples into a database. + * The OutputFormat has to be configured using the supplied OutputFormatBuilder. + * + * @see Tuple + * @see DriverManager + */ +public class RetractJDBCOutputFormat extends RichOutputFormat { + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(org.apache.flink.api.java.io.jdbc.JDBCOutputFormat.class); + + private String username; + private String password; + private String drivername; + private String dbURL; + private String insertQuery; + private int batchInterval = 5000; + + private Connection dbConn; + private PreparedStatement upload; + + private int batchCount = 0; + + public int[] typesArray; + + public RetractJDBCOutputFormat() { + } + + @Override + public void configure(Configuration parameters) { + } + + /** + * Connects to the target database and initializes the prepared statement. + * + * @param taskNumber The number of the parallel instance. + * @throws IOException Thrown, if the output could not be opened due to an + * I/O problem. + */ + @Override + public void open(int taskNumber, int numTasks) throws IOException { + try { + establishConnection(); + upload = dbConn.prepareStatement(insertQuery); + } catch (SQLException sqe) { + throw new IllegalArgumentException("open() failed.", sqe); + } catch (ClassNotFoundException cnfe) { + throw new IllegalArgumentException("JDBC driver class not found.", cnfe); + } + } + + private void establishConnection() throws SQLException, ClassNotFoundException { + Class.forName(drivername); + if (username == null) { + dbConn = DriverManager.getConnection(dbURL); + } else { + dbConn = DriverManager.getConnection(dbURL, username, password); + } + } + + /** + * Adds a record to the prepared statement. + *

+ * When this method is called, the output format is guaranteed to be opened. + *

+ * + * WARNING: this may fail when no column types specified (because a best effort approach is attempted in order to + * insert a null value but it's not guaranteed that the JDBC driver handles PreparedStatement.setObject(pos, null)) + * + * @param tuple2 The records to add to the output. + * @see PreparedStatement + * @throws IOException Thrown, if the records could not be added due to an I/O problem. + */ + @Override + public void writeRecord(Tuple2 tuple2) throws IOException { + + Tuple2 tupleTrans = tuple2; + Boolean retract = tupleTrans.getField(0); + Row row = tupleTrans.getField(1); + + + if (typesArray != null && typesArray.length > 0 && typesArray.length != row.getArity()) { + LOG.warn("Column SQL types array doesn't match arity of passed Row! Check the passed array..."); + } + try { + if(retract){ + insertWrite(row); + }else{ + //do nothing + } + } catch (SQLException | IllegalArgumentException e) { + throw new IllegalArgumentException("writeRecord() failed", e); + } + } + + + private void insertWrite(Row row) throws SQLException { + updatePreparedStmt(row, upload); + upload.addBatch(); + batchCount++; + if (batchCount >= batchInterval) { + upload.executeBatch(); + batchCount = 0; + } + } + + + private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLException { + if (typesArray == null ) { + // no types provided + for (int index = 0; index < row.getArity(); index++) { + LOG.warn("Unknown column type for column %s. Best effort approach to set its value: %s.", index + 1, row.getField(index)); + pstmt.setObject(index + 1, row.getField(index)); + } + } else { + // types provided + for (int index = 0; index < row.getArity(); index++) { + + if (row.getField(index) == null) { + pstmt.setNull(index + 1, typesArray[index]); + } else { + // casting values as suggested by http://docs.oracle.com/javase/1.5.0/docs/guide/jdbc/getstart/mapping.html + switch (typesArray[index]) { + case java.sql.Types.NULL: + pstmt.setNull(index + 1, typesArray[index]); + break; + case java.sql.Types.BOOLEAN: + case java.sql.Types.BIT: + pstmt.setBoolean(index + 1, (boolean) row.getField(index)); + break; + case java.sql.Types.CHAR: + case java.sql.Types.NCHAR: + case java.sql.Types.VARCHAR: + case java.sql.Types.LONGVARCHAR: + case java.sql.Types.LONGNVARCHAR: + pstmt.setString(index + 1, (String) row.getField(index)); + break; + case java.sql.Types.TINYINT: + pstmt.setByte(index + 1, (byte) row.getField(index)); + break; + case java.sql.Types.SMALLINT: + pstmt.setShort(index + 1, (short) row.getField(index)); + break; + case java.sql.Types.INTEGER: + pstmt.setInt(index + 1, (int) row.getField(index)); + break; + case java.sql.Types.BIGINT: + pstmt.setLong(index + 1, (long) row.getField(index)); + break; + case java.sql.Types.REAL: + pstmt.setFloat(index + 1, (float) row.getField(index)); + break; + case java.sql.Types.FLOAT: + case java.sql.Types.DOUBLE: + pstmt.setDouble(index + 1, (double) row.getField(index)); + break; + case java.sql.Types.DECIMAL: + case java.sql.Types.NUMERIC: + pstmt.setBigDecimal(index + 1, (java.math.BigDecimal) row.getField(index)); + break; + case java.sql.Types.DATE: + pstmt.setDate(index + 1, (java.sql.Date) row.getField(index)); + break; + case java.sql.Types.TIME: + pstmt.setTime(index + 1, (java.sql.Time) row.getField(index)); + break; + case java.sql.Types.TIMESTAMP: + pstmt.setTimestamp(index + 1, (java.sql.Timestamp) row.getField(index)); + break; + case java.sql.Types.BINARY: + case java.sql.Types.VARBINARY: + case java.sql.Types.LONGVARBINARY: + pstmt.setBytes(index + 1, (byte[]) row.getField(index)); + break; + default: + pstmt.setObject(index + 1, row.getField(index)); + LOG.warn("Unmanaged sql type (%s) for column %s. Best effort approach to set its value: %s.", + typesArray[index], index + 1, row.getField(index)); + // case java.sql.Types.SQLXML + // case java.sql.Types.ARRAY: + // case java.sql.Types.JAVA_OBJECT: + // case java.sql.Types.BLOB: + // case java.sql.Types.CLOB: + // case java.sql.Types.NCLOB: + // case java.sql.Types.DATALINK: + // case java.sql.Types.DISTINCT: + // case java.sql.Types.OTHER: + // case java.sql.Types.REF: + // case java.sql.Types.ROWID: + // case java.sql.Types.STRUC + } + } + } + } + } + + /** + * Executes prepared statement and closes all resources of this instance. + * + * @throws IOException Thrown, if the input could not be closed properly. + */ + @Override + public void close() throws IOException { + try { + if (upload != null) { + upload.executeBatch(); + upload.close(); + } + } catch (SQLException se) { + LOG.info("Inputformat couldn't be closed - " + se.getMessage()); + } finally { + upload = null; + batchCount = 0; + } + + try { + if (dbConn != null) { + dbConn.close(); + } + } catch (SQLException se) { + LOG.info("Inputformat couldn't be closed - " + se.getMessage()); + } finally { + dbConn = null; + } + } + + public static JDBCOutputFormatBuilder buildJDBCOutputFormat() { + return new JDBCOutputFormatBuilder(); + } + + public static class JDBCOutputFormatBuilder { + private final RetractJDBCOutputFormat format; + + protected JDBCOutputFormatBuilder() { + this.format = new RetractJDBCOutputFormat(); + } + + public JDBCOutputFormatBuilder setUsername(String username) { + format.username = username; + return this; + } + + public JDBCOutputFormatBuilder setPassword(String password) { + format.password = password; + return this; + } + + public JDBCOutputFormatBuilder setDrivername(String drivername) { + format.drivername = drivername; + return this; + } + + public JDBCOutputFormatBuilder setDBUrl(String dbURL) { + format.dbURL = dbURL; + return this; + } + + public JDBCOutputFormatBuilder setInsertQuery(String query) { + format.insertQuery = query; + return this; + } + + + public JDBCOutputFormatBuilder setBatchInterval(int batchInterval) { + format.batchInterval = batchInterval; + return this; + } + + public JDBCOutputFormatBuilder setSqlTypes(int[] typesArray) { + format.typesArray = typesArray; + return this; + } + + /** + * Finalizes the configuration and checks validity. + * + * @return Configured RetractJDBCOutputFormat + */ + public RetractJDBCOutputFormat finish() { + if (format.username == null) { + LOG.info("Username was not supplied separately."); + } + if (format.password == null) { + LOG.info("Password was not supplied separately."); + } + if (format.dbURL == null) { + throw new IllegalArgumentException("No dababase URL supplied."); + } + if (format.insertQuery == null) { + throw new IllegalArgumentException("No insertQuery suplied"); + } + if (format.drivername == null) { + throw new IllegalArgumentException("No driver supplied"); + } + return format; + } + } + +} diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java new file mode 100644 index 000000000..f1522226b --- /dev/null +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.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 com.dtstack.flink.sql.sink.mysql.table; + +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/7/4 + * Company: www.dtstack.com + * @author xuchao + */ + +public class MysqlSinkParser extends AbsTableParser { + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + MysqlTableInfo mysqlTableInfo = new MysqlTableInfo(); + mysqlTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, mysqlTableInfo); + + mysqlTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(MysqlTableInfo.PARALLELISM_KEY.toLowerCase()))); + mysqlTableInfo.setUrl(MathUtil.getString(props.get(MysqlTableInfo.URL_KEY.toLowerCase()))); + mysqlTableInfo.setTableName(MathUtil.getString(props.get(MysqlTableInfo.TABLE_NAME_KEY.toLowerCase()))); + mysqlTableInfo.setUserName(MathUtil.getString(props.get(MysqlTableInfo.USER_NAME_KEY.toLowerCase()))); + mysqlTableInfo.setPassword(MathUtil.getString(props.get(MysqlTableInfo.PASSWORD_KEY.toLowerCase()))); + mysqlTableInfo.setBatchSize(MathUtil.getIntegerVal(props.get(MysqlTableInfo.BATCH_SIZE_KEY.toLowerCase()))); + mysqlTableInfo.setBufferSize(MathUtil.getString(props.get(MysqlTableInfo.BUFFER_SIZE_KEY.toLowerCase()))); + mysqlTableInfo.setFlushIntervalMs(MathUtil.getString(props.get(MysqlTableInfo.FLUSH_INTERVALMS_KEY.toLowerCase()))); + + return mysqlTableInfo; + } +} diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlTableInfo.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlTableInfo.java new file mode 100644 index 000000000..0576c466e --- /dev/null +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlTableInfo.java @@ -0,0 +1,139 @@ +/* + * 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 com.dtstack.flink.sql.sink.mysql.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * Reason: + * Date: 2018/6/22 + * Company: www.dtstack.com + * @author xuchao + */ + +public class MysqlTableInfo extends TargetTableInfo { + + private static final String CURR_TYPE = "mysql"; + + public static final String URL_KEY = "url"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String USER_NAME_KEY = "userName"; + + public static final String PASSWORD_KEY = "password"; + + public static final String BATCH_SIZE_KEY = "batchSize"; + + public static final String BUFFER_SIZE_KEY = "bufferSize"; + + public static final String FLUSH_INTERVALMS_KEY = "flushIntervalMs"; + + public MysqlTableInfo(){ + setType(CURR_TYPE); + } + + private String url; + + private String tableName; + + private String userName; + + private String password; + + private Integer batchSize; + + private String bufferSize; + + private String flushIntervalMs; + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } + + public Integer getBatchSize() { + return batchSize; + } + + public void setBatchSize(Integer batchSize) { + this.batchSize = batchSize; + } + + public String getBufferSize() { + return bufferSize; + } + + public void setBufferSize(String bufferSize) { + this.bufferSize = bufferSize; + } + + public String getFlushIntervalMs() { + return flushIntervalMs; + } + + public void setFlushIntervalMs(String flushIntervalMs) { + this.flushIntervalMs = flushIntervalMs; + } + + @Override + public boolean check() { + Preconditions.checkNotNull(url, "mysql field of URL is required"); + Preconditions.checkNotNull(tableName, "mysql field of tableName is required"); + Preconditions.checkNotNull(userName, "mysql field of userName is required"); + Preconditions.checkNotNull(password, "mysql field of password is required"); + return true; + } + + @Override + public String getType() { + // return super.getType().toLowerCase() + TARGET_SUFFIX; + return super.getType().toLowerCase(); + } +} diff --git a/mysql/mysql-sink/src/test/java/com/dtstack/flinkx/AppTest.java b/mysql/mysql-sink/src/test/java/com/dtstack/flinkx/AppTest.java new file mode 100644 index 000000000..33a0233ac --- /dev/null +++ b/mysql/mysql-sink/src/test/java/com/dtstack/flinkx/AppTest.java @@ -0,0 +1,58 @@ +/* + * 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 com.dtstack.flinkx; + +import junit.framework.Test; +import junit.framework.TestCase; +import junit.framework.TestSuite; + +/** + * Unit test for simple App. + */ +public class AppTest + extends TestCase +{ + /** + * Create the test case + * + * @param testName name of the test case + */ + public AppTest( String testName ) + { + super( testName ); + } + + /** + * @return the suite of tests being tested + */ + public static Test suite() + { + return new TestSuite( AppTest.class ); + } + + /** + * Rigourous Test :-) + */ + public void testApp() + { + assertTrue( true ); + } +} diff --git a/mysql/pom.xml b/mysql/pom.xml new file mode 100644 index 000000000..fa1c2df63 --- /dev/null +++ b/mysql/pom.xml @@ -0,0 +1,20 @@ + + + + flink.sql + com.dtstack.flinkx + 1.0-SNAPSHOT + + 4.0.0 + sql.mysql + pom + + + mysql-sink + mysql-side + + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml new file mode 100644 index 000000000..84f93b0e9 --- /dev/null +++ b/pom.xml @@ -0,0 +1,38 @@ + + 4.0.0 + + com.dtstack.flinkx + flink.sql + 1.0-SNAPSHOT + + core + kafka09 + mysql + hbase + elasticsearch + launcher + + pom + + flink.sql + http://maven.apache.org + + + UTF-8 + 1.4.0 + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.8 + 1.8 + + + + + From 1bf593df241aec5f09e8eeab128e104764bd44bc Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 12 Sep 2018 18:03:10 +0800 Subject: [PATCH 004/470] upload license --- LICENSE.txt | 201 ++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 201 insertions(+) create mode 100644 LICENSE.txt diff --git a/LICENSE.txt b/LICENSE.txt new file mode 100644 index 000000000..9c8f3ea08 --- /dev/null +++ b/LICENSE.txt @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright {yyyy} {name of copyright owner} + + Licensed 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. \ No newline at end of file From d3babe544cb71162255459f1f0fc5388587db9ff Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 12 Sep 2018 20:03:53 +0800 Subject: [PATCH 005/470] =?UTF-8?q?=E7=BC=96=E8=BE=91readme.md?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 94 ++++++++++++++++++- .../main/java/com/dtstack/flink/sql/Main.java | 3 +- .../sql/launcher/LauncherOptionParser.java | 7 +- 3 files changed, 99 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index 490262eb7..359a5a870 100644 --- a/README.md +++ b/README.md @@ -1,2 +1,94 @@ # flinkStreamSQL -基于开源的flink,对其实时sql进行扩展;主要实现了流与维表的join。 +> * 基于开源的flink,对其实时sql进行扩展 +> > 自定义create table 语法(包括输入源表,输出表,维表) +> > 自定义create function 语法 +> > 实现了流与维表的join + + +## 1 快速起步 +### 1.1 运行模式 + + +* 单机模式:对应Flink集群的单机模式 +* standalone模式:对应Flink集群的分布式模式 +* yarn模式:对应Flink集群的yarn模式 + +### 1.2 执行环境 + +* Java: JDK8及以上 +* Flink集群: 1.4(单机模式不需要安装Flink集群) +* 操作系统:理论上不限 + +### 1.3 打包 + +进入项目根目录,使用maven打包: + +``` +mvn clean package -Dmaven.test.skip +``` + +打包结束后,项目根目录下会产生plugins目录,plugins目录下存放编译好的数据同步插件包 + + +### 1.4 启动 + +#### 1.4.1 命令行参数选项 + +* **model** + * 描述:执行模式,也就是flink集群的工作模式 + * local: 本地模式 + * standalone: 独立部署模式的flink集群 + * yarn: yarn模式的flink集群 + * 必选:否 + * 默认值:local + +* **name** + * 描述:flink 任务对应名称。 + * 必选:是 + * 默认值:无 + +* **sql** + * 描述:执行flink sql 的主体语句。 + * 必选:是 + * 默认值:无 + +* **localSqlPluginPath** + * 描述:本地插件根目录地址,也就是打包后产生的plugins目录。 + * 必选:是 + * 默认值:无 + +* **remoteSqlPluginPath** + * 描述:flink执行集群上的插件根目录地址(将打包好的插件存放到各个flink节点上,如果是yarn集群需要存放到所有的nodemanager上)。 + * 必选:否 + * 默认值:无 + +* **addjar** + * 描述:扩展jar路径,当前主要是UDF定义的jar; + * 必选:否 + * 默认值:无 + +* **confProp** + * 描述:一些参数设置 + * 必选:否 + * 默认值:无 + * 可选参数: + sql.env.parallelism: 默认并行度设置 + sql.max.env.parallelism: 最大并行度设置 + time.characteristic: 可选值[ProcessingTime|IngestionTime|EventTime] + sql.checkpoint.interval: 设置了该参数表明开启checkpoint(ms) + sql.checkpoint.mode: 可选值[EXACTLY_ONCE|AT_LEAST_ONCE] + sql.checkpoint.timeout: 生成checkpoint的超时时间(ms) + sql.max.concurrent.checkpoints: 最大并发生成checkpoint数 + sql.checkpoint.cleanup.mode: 默认是不会将checkpoint存储到外部存储,[true(任务cancel之后会删除外部存储)|false(外部存储需要手动删除)] + flinkCheckpointDataURI: 设置checkpoint的外部存储路径,根据实际的需求设定文件路径,hdfs://, file:// + + +* **flinkconf** + * 描述:flink配置文件所在的目录(单机模式下不需要),如/hadoop/flink-1.4.0/conf + * 必选:否 + * 默认值:无 + +* **yarnconf** + * 描述:Hadoop配置文件(包括hdfs和yarn)所在的目录(单机模式下不需要),如/hadoop/etc/hadoop + * 必选:否 + * 默认值:无 \ No newline at end of file diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 541e87909..04a7997a0 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -119,7 +119,6 @@ public static void main(String[] args) throws Exception { Preconditions.checkNotNull(sql, "it requires input parameters sql"); Preconditions.checkNotNull(name, "it requires input parameters name"); Preconditions.checkNotNull(localSqlPluginPath, "it requires input parameters localSqlPluginPath"); - Preconditions.checkNotNull(remoteSqlPluginPath, "it requires input parameters remoteSqlPluginPath"); sql = URLDecoder.decode(sql, Charsets.UTF_8.name()); SqlParser.setLocalSqlPluginRoot(localSqlPluginPath); @@ -260,7 +259,7 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en }else{ fields += ",PROCTIME.PROCTIME"; } - //tableEnv.registerDataStream(tableInfo.getName(), adaptStream, fields); + Table regTable = tableEnv.fromDataStream(adaptStream, fields); tableEnv.registerTable(tableInfo.getName(), regTable); registerTableCache.put(tableInfo.getName(), regTable); diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java index 1a6a1043e..50484dd02 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java @@ -81,8 +81,11 @@ public LauncherOptionParser(String[] args) { String localPlugin = Preconditions.checkNotNull(cl.getOptionValue(OPTION_LOCAL_SQL_PLUGIN_PATH)); properties.put(OPTION_LOCAL_SQL_PLUGIN_PATH, localPlugin); - String remotePlugin = Preconditions.checkNotNull(cl.getOptionValue(OPTION_REMOTE_SQL_PLUGIN_PATH)); - properties.put(OPTION_REMOTE_SQL_PLUGIN_PATH, remotePlugin); + String remotePlugin = cl.getOptionValue(OPTION_REMOTE_SQL_PLUGIN_PATH); + if(!mode.equalsIgnoreCase(ClusterMode.MODE_LOCAL)){ + Preconditions.checkNotNull(remotePlugin); + properties.put(OPTION_REMOTE_SQL_PLUGIN_PATH, remotePlugin); + } String name = Preconditions.checkNotNull(cl.getOptionValue(OPTION_NAME)); properties.put(OPTION_NAME, name); From 5f36eb135e1aed1c61d74ccd6d366539a69e3ab3 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 12 Sep 2018 21:34:16 +0800 Subject: [PATCH 006/470] =?UTF-8?q?=E7=BC=96=E8=BE=91readme.md?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 116 +++++++++++++++++++++++++++++++++----- docs/elasticsearchSink.md | 0 docs/hbaseSide.md | 65 +++++++++++++++++++++ docs/hbaseSink.md | 0 docs/kafka09source.md | 35 ++++++++++++ docs/mysqlSide.md | 0 docs/mysqlSink.md | 0 7 files changed, 203 insertions(+), 13 deletions(-) create mode 100644 docs/elasticsearchSink.md create mode 100644 docs/hbaseSide.md create mode 100644 docs/hbaseSink.md create mode 100644 docs/kafka09source.md create mode 100644 docs/mysqlSide.md create mode 100644 docs/mysqlSink.md diff --git a/README.md b/README.md index 359a5a870..5fd92371f 100644 --- a/README.md +++ b/README.md @@ -1,8 +1,8 @@ # flinkStreamSQL > * 基于开源的flink,对其实时sql进行扩展 -> > 自定义create table 语法(包括输入源表,输出表,维表) -> > 自定义create function 语法 -> > 实现了流与维表的join +> > * 自定义create table 语法(包括源表,输出表,维表) +> > * 自定义create function 语法 +> > * 实现了流与维表的join ## 1 快速起步 @@ -72,15 +72,15 @@ mvn clean package -Dmaven.test.skip * 必选:否 * 默认值:无 * 可选参数: - sql.env.parallelism: 默认并行度设置 - sql.max.env.parallelism: 最大并行度设置 - time.characteristic: 可选值[ProcessingTime|IngestionTime|EventTime] - sql.checkpoint.interval: 设置了该参数表明开启checkpoint(ms) - sql.checkpoint.mode: 可选值[EXACTLY_ONCE|AT_LEAST_ONCE] - sql.checkpoint.timeout: 生成checkpoint的超时时间(ms) - sql.max.concurrent.checkpoints: 最大并发生成checkpoint数 - sql.checkpoint.cleanup.mode: 默认是不会将checkpoint存储到外部存储,[true(任务cancel之后会删除外部存储)|false(外部存储需要手动删除)] - flinkCheckpointDataURI: 设置checkpoint的外部存储路径,根据实际的需求设定文件路径,hdfs://, file:// + * sql.env.parallelism: 默认并行度设置 + * sql.max.env.parallelism: 最大并行度设置 + * time.characteristic: 可选值[ProcessingTime|IngestionTime|EventTime] + * sql.checkpoint.interval: 设置了该参数表明开启checkpoint(ms) + * sql.checkpoint.mode: 可选值[EXACTLY_ONCE|AT_LEAST_ONCE] + * sql.checkpoint.timeout: 生成checkpoint的超时时间(ms) + * sql.max.concurrent.checkpoints: 最大并发生成checkpoint数 + * sql.checkpoint.cleanup.mode: 默认是不会将checkpoint存储到外部存储,[true(任务cancel之后会删除外部存储)|false(外部存储需要手动删除)] + * flinkCheckpointDataURI: 设置checkpoint的外部存储路径,根据实际的需求设定文件路径,hdfs://, file:// * **flinkconf** @@ -91,4 +91,94 @@ mvn clean package -Dmaven.test.skip * **yarnconf** * 描述:Hadoop配置文件(包括hdfs和yarn)所在的目录(单机模式下不需要),如/hadoop/etc/hadoop * 必选:否 - * 默认值:无 \ No newline at end of file + * 默认值:无 + +## 2 结构 +### 2.1 源表插件 +* [kafka09 源表插件](docs/kafka09Source.md) + +### 2.2 结果表插件 +* [elasticsearch 结果表插件](docs/elasticsearchSink.md) +* [hbase 结果表插件](docs/hbaseSink.md) +* [mysql 结果表插件](docs/mysqlSink.md) + +### 2.3 维表插件 +* [hbase 维表插件](docs/hbaseSide.md) +* [mysql 维表插件](docs/mysqlSide.md) + +## 3 样例 + +``` +CREATE TABLE MyTable( + name string, + channel STRING, + pv INT, + xctime bigint, + CHARACTER_LENGTH(channel) AS timeLeng + )WITH( + type ='kafka09', + bootstrapServers ='172.16.8.198:9092', + zookeeperQuorum ='172.16.8.198:2181/kafka', + offsetReset ='latest', + topic ='nbTest1', + parallelism ='1' + ); + +CREATE TABLE MyResult( + channel VARCHAR, + pv VARCHAR + )WITH( + type ='mysql', + url ='jdbc:mysql://172.16.8.104:3306/test?charset=utf8', + userName ='dtstack', + password ='abc123', + tableName ='pv2', + parallelism ='1' + ); + +CREATE TABLE workerinfo( + cast(logtime as TIMESTAMP)AS rtime, + cast(logtime)AS rtime + )WITH( + type ='hbase', + zookeeperQuorum ='rdos1:2181', + tableName ='workerinfo', + rowKey ='ce,de', + parallelism ='1', + zookeeperParent ='/hbase' + ); + +CREATE TABLE sideTable( + cf:name String as name, + cf:info String as info, + PRIMARY KEY(name), + PERIOD FOR SYSTEM_TIME + )WITH( + type ='hbase', + zookeeperQuorum ='rdos1:2181', + zookeeperParent ='/hbase', + tableName ='workerinfo', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1' + ); + +insert +into + MyResult + select + d.channel, + d.info + from + ( select + a.*,b.info + from + MyTable a + join + sideTable b + on a.channel=b.name + where + a.channel = 'xc2' + and a.pv=10 ) as d +``` \ No newline at end of file diff --git a/docs/elasticsearchSink.md b/docs/elasticsearchSink.md new file mode 100644 index 000000000..e69de29bb diff --git a/docs/hbaseSide.md b/docs/hbaseSide.md new file mode 100644 index 000000000..4e99bcfbb --- /dev/null +++ b/docs/hbaseSide.md @@ -0,0 +1,65 @@ + +## 1.格式: +``` + CREATE TABLE tableName( + columnFamily:columnName type as alias, + ... + PRIMARY KEY(keyInfo), + PERIOD FOR SYSTEM_TIME + )WITH( + type ='hbase', + zookeeperQuorum ='ip:port', + zookeeperParent ='/hbase', + tableName ='tableNamae', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' + ); +``` + +## 2.参数 + +* tableName ==> 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同) +* columnFamily:columnName ==> hbase中的列族名称和列名称 +* alias ===> hbase 中的列对应到flink中注册的列名称 +* PERIOD FOR SYSTEM_TIME ==> 关键字表明该定义的表为维表信息 +* PRIMARY KEY(keyInfo) ==> 维表主键定义;hbase 维表为rowkey的构造方式; + 可选择的构造包括 md5(alias + alias), '常量',也包括上述方式的自由组合 + +* type ==> 表明维表的类型[hbase|mysql] +* zookeeperQuorum ==> hbase 的zk地址;格式ip:port[;ip:port] +* zookeeperParent ==> hbase 的zk parent路径 +* tableName ==> hbase 的表名称 +* cache ==> 维表缓存策略(NONE/LRU) + + > * NONE: 不做内存缓存 + > * LRU: + > > cacheSize ==> 缓存的条目数量 + > > cacheTTLMs ==> 缓存的过期时间(ms) + +* partitionedJoin ==> 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量) + +## 3.样例 +``` +CREATE TABLE sideTable( + cf:name String as name, + cf:info int as info, + PRIMARY KEY(md5(name) + 'test'), + PERIOD FOR SYSTEM_TIME + )WITH( + type ='hbase', + zookeeperQuorum ='rdos1:2181', + zookeeperParent ='/hbase', + tableName ='workerinfo', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='true' + ); + +``` + + diff --git a/docs/hbaseSink.md b/docs/hbaseSink.md new file mode 100644 index 000000000..e69de29bb diff --git a/docs/kafka09source.md b/docs/kafka09source.md new file mode 100644 index 000000000..e5ad822cb --- /dev/null +++ b/docs/kafka09source.md @@ -0,0 +1,35 @@ +## 1.格式: +``` +CREATE TABLE tableName( + colName colType, + ... + function(channel) AS alias + )WITH( + type ='kafka09', + bootstrapServers ='ip:port,ip:port...', + zookeeperQuorum ='ip:port,ip:port/zkparent', + offsetReset ='latest', + topic ='nbTest1', + parallelism ='1' + ); +``` +## 2.参数: + * type ==> kafka09 + * bootstrapServers +## 3.样例: +``` +CREATE TABLE MyTable( + name string, + channel STRING, + pv INT, + xctime bigint, + CHARACTER_LENGTH(channel) AS timeLeng + )WITH( + type ='kafka09', + bootstrapServers ='172.16.8.198:9092', + zookeeperQuorum ='172.16.8.198:2181/kafka', + offsetReset ='latest', + topic ='nbTest1', + parallelism ='1' + ); +``` \ No newline at end of file diff --git a/docs/mysqlSide.md b/docs/mysqlSide.md new file mode 100644 index 000000000..e69de29bb diff --git a/docs/mysqlSink.md b/docs/mysqlSink.md new file mode 100644 index 000000000..e69de29bb From b5db1edec8b133b64c3677d0f8aaee40c57a579b Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 12 Sep 2018 22:00:19 +0800 Subject: [PATCH 007/470] Update README.md --- README.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 5fd92371f..fc4c0c073 100644 --- a/README.md +++ b/README.md @@ -25,6 +25,8 @@ ``` mvn clean package -Dmaven.test.skip + +所有插件包会生成到plugins下 ``` 打包结束后,项目根目录下会产生plugins目录,plugins目录下存放编译好的数据同步插件包 @@ -181,4 +183,4 @@ into where a.channel = 'xc2' and a.pv=10 ) as d -``` \ No newline at end of file +``` From da6b7cb12f4c5884f41a9bf01f97ba6a206dc295 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 13 Sep 2018 11:37:21 +0800 Subject: [PATCH 008/470] Update README.md --- README.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/README.md b/README.md index fc4c0c073..9bd80dabb 100644 --- a/README.md +++ b/README.md @@ -4,6 +4,13 @@ > > * 自定义create function 语法 > > * 实现了流与维表的join +# 后续开发计划 + * 增加全局缓存功能 + * sql增加临时表功能 + * 增加redis维表功能 + * 增加mongdb维表功能 + * 增加oracle维表功能 + * 增加SQlServer维表功能 ## 1 快速起步 ### 1.1 运行模式 From 3e611ca857027a3ce9df0ca25a00a0ae80816623 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 13 Sep 2018 11:45:50 +0800 Subject: [PATCH 009/470] =?UTF-8?q?=E7=BC=96=E8=BE=91readme.md?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 1 + .../flink/sql/table/AbsTableParser.java | 1 - .../sql/watermarker/WaterMarkerAssigner.java | 1 - docs/colType.md | 16 +++++ docs/hbaseSink.md | 45 ++++++++++++ docs/kafka09source.md | 22 ++++-- docs/mysqlSide.md | 69 +++++++++++++++++++ docs/mysqlSink.md | 42 +++++++++++ 8 files changed, 191 insertions(+), 6 deletions(-) create mode 100644 docs/colType.md diff --git a/README.md b/README.md index fc4c0c073..9e4272435 100644 --- a/README.md +++ b/README.md @@ -71,6 +71,7 @@ mvn clean package -Dmaven.test.skip * **confProp** * 描述:一些参数设置 + * 格式: json * 必选:否 * 默认值:无 * 可选参数: diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index 84ccc0779..108e776bf 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -102,7 +102,6 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ System.arraycopy(filedInfoArr, 0, filedNameArr, 0, filedInfoArr.length - 1); String fieldName = String.join(" ", filedNameArr); String fieldType = filedInfoArr[filedInfoArr.length - 1 ].trim(); - //Class fieldClass = ClassUtil.stringConvertClass(filedInfoArr[1].trim()); Class fieldClass = ClassUtil.stringConvertClass(fieldType); tableInfo.addField(fieldName); diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java index c922d12d1..2ef0d6a79 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java @@ -52,7 +52,6 @@ public DataStream assignWaterMarker(DataStream dataStream, RowTypeInfo type String[] fieldNames = typeInfo.getFieldNames(); TypeInformation[] fieldTypes = typeInfo.getFieldTypes(); - String fields = StringUtils.join(fieldNames, ","); if(Strings.isNullOrEmpty(eventTimeFieldName)){ return dataStream; diff --git a/docs/colType.md b/docs/colType.md new file mode 100644 index 000000000..9a9593e7a --- /dev/null +++ b/docs/colType.md @@ -0,0 +1,16 @@ +| 支持的类型 | java对应类型 | +| ------ | ----- | +| boolean | Boolean | +| int | Integer | +| bigint | Long | +| tinyint | Byte | +| byte | Byte | +| short | Short | +| smallint | Short| +| char | String| +| varchar | String | +| string | String| +| float | Float| +| double | Double| +| date | Date | +| timestamp | Timestamp | \ No newline at end of file diff --git a/docs/hbaseSink.md b/docs/hbaseSink.md index e69de29bb..9ae98edfe 100644 --- a/docs/hbaseSink.md +++ b/docs/hbaseSink.md @@ -0,0 +1,45 @@ +## 1.格式: +``` +CREATE TABLE MyResult( + colFamily:colName colType, + ... + )WITH( + type ='hbase', + zookeeperQuorum ='ip:port[,ip:port]', + tableName ='tableName', + rowKey ='colFamily:colName[,colFamily:colName]', + parallelism ='1', + zookeeperParent ='/hbase' + ) + + +``` + +## 2.参数: + * tableName ==> 在 sql 中使用的名称;即注册到flink-table-env上的名称 + * colFamily:colName ==> hbase中的列族名称和列名称 + * colType ==> 列类型 [colType支持的类型](colType.md) + + * type ==> 表明 输出表类型[mysql|hbase|elasticsearch] + * zookeeperQuorum ==> hbase zk地址,多个直接用逗号隔开 + * zookeeperParent ==> zkParent 路径 + * tableName ==> 关联的hbase表名称 + * rowKey ==> hbase的rowkey关联的列信息 + * parallelism ==> 并行度设置 + + +## 3.样例: +``` +CREATE TABLE MyResult( + cf:channel STRING, + cf:pv BIGINT + )WITH( + type ='hbase', + zookeeperQuorum ='rdos1:2181', + tableName ='workerinfo', + rowKey ='cf:channel', + parallelism ='1', + zookeeperParent ='/hbase' + ) + + ``` \ No newline at end of file diff --git a/docs/kafka09source.md b/docs/kafka09source.md index e5ad822cb..ca2607ca0 100644 --- a/docs/kafka09source.md +++ b/docs/kafka09source.md @@ -3,19 +3,33 @@ CREATE TABLE tableName( colName colType, ... - function(channel) AS alias + function(colNameX) AS aliasName, + WATERMARK FOR colName AS withOffset( colName , delayTime ) )WITH( type ='kafka09', bootstrapServers ='ip:port,ip:port...', zookeeperQuorum ='ip:port,ip:port/zkparent', offsetReset ='latest', - topic ='nbTest1', - parallelism ='1' + topic ='topicName', + parallelism ='parllNum' ); ``` ## 2.参数: + * tableName ==> 在 sql 中使用的名称;即注册到flink-table-env上的名称 + * colName ==> 列名称 + * colType ==> 列类型 [colType支持的类型](colType.md) + * function(colNameX) as aliasName ==> 支持在定义列信息的时候根据已有列类型生成新的列(函数可以使用系统函数和已经注册的UDF) + * WATERMARK FOR colName AS withOffset( colName , delayTime ) ==> 标识输入流生的watermake生成规则,根据指定的colName(当前支持列的类型为Long|Timestamp) + 和delayTime生成waterMark 同时会在注册表的使用附带上rowtime字段(如果未指定则默认添加proctime字段); + 注意:添加该标识的使用必须设置系统参数 time.characteristic:EventTime; delayTime: 数据最大延迟时间(ms) + * type ==> kafka09 - * bootstrapServers + * bootstrapServers ==> kafka bootstrap-server 地址信息(多个用逗号隔开) + * zookeeperQuorum ==> kafka zk地址信息(多个之间用逗号分隔) + * topic ==> 需要读取的 topic 名称 + * offsetReset ==> 读取的topic 的offset初始位置[latest|earliest] + * parallelism ==> 并行度设置 + ## 3.样例: ``` CREATE TABLE MyTable( diff --git a/docs/mysqlSide.md b/docs/mysqlSide.md index e69de29bb..552b9cb65 100644 --- a/docs/mysqlSide.md +++ b/docs/mysqlSide.md @@ -0,0 +1,69 @@ + +## 1.格式: +``` + CREATE TABLE tableName( + colName cloType, + ... + PRIMARY KEY(keyInfo), + PERIOD FOR SYSTEM_TIME + )WITH( + type='mysql', + url='jdbcUrl', + userName='dbUserName', + password='dbPwd', + tableName='tableName', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' + ); +``` + +## 2.参数 + + * tableName ==> 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同) + * colName ==> 列名称 + * colType ==> 列类型 [colType支持的类型](colType.md) + * PERIOD FOR SYSTEM_TIME ==> 关键字表明该定义的表为维表信息 + * PRIMARY KEY(keyInfo) ==> 维表主键定义;多个列之间用逗号隔开 + * url ==> 连接mysql数据库 jdbcUrl + * userName ==> mysql连接用户名 + * password ==> mysql连接密码 + * tableName ==> mysql表名称 + * type ==> 表明维表的类型[hbase|mysql] + + * tableName ==> mysql 的表名称 + * cache ==> 维表缓存策略(NONE/LRU) + + > * NONE: 不做内存缓存 + > * LRU: + > > cacheSize ==> 缓存的条目数量 + > > cacheTTLMs ==> 缓存的过期时间(ms) + + * partitionedJoin ==> 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量) + +## 3.样例 +``` +create table sideTable( + channel String, + xccount int, + PRIMARY KEY(channel), + PERIOD FOR SYSTEM_TIME + )WITH( + type='mysql', + url='jdbc:mysql://172.16.8.104:3306/test?charset=utf8', + userName='dtstack', + password='abc123', + tableName='sidetest', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' + ); + + +``` + + diff --git a/docs/mysqlSink.md b/docs/mysqlSink.md index e69de29bb..544c4d377 100644 --- a/docs/mysqlSink.md +++ b/docs/mysqlSink.md @@ -0,0 +1,42 @@ +## 1.格式: +``` +CREATE TABLE tableName( + colName colType, + ... + colNameX colType + )WITH( + type ='mysql', + url ='jdbcUrl', + userName ='userName', + password ='pwd', + tableName ='tableName', + parallelism ='parllNum' + ); + +``` + +## 2.参数: + * tableName ==> 在 sql 中使用的名称;即注册到flink-table-env上的名称 + * colName ==> 列名称 + * colType ==> 列类型 [colType支持的类型](colType.md) + * type ==> 表明 输出表类型[mysql|hbase|elasticsearch] + * url ==> 连接mysql数据库 jdbcUrl + * userName ==> mysql连接用户名 + * password ==> mysql连接密码 + * tableName ==> mysql表名称 + * parallelism ==> 并行度设置 + +## 3.样例: +``` +CREATE TABLE MyResult( + channel VARCHAR, + pv VARCHAR + )WITH( + type ='mysql', + url ='jdbc:mysql://172.16.8.104:3306/test?charset=utf8', + userName ='dtstack', + password ='abc123', + tableName ='pv2', + parallelism ='1' + ) + ``` \ No newline at end of file From b1df81d8c04e3dac6b3d2a6e6e3acc26ffe04418 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 13 Sep 2018 11:47:31 +0800 Subject: [PATCH 010/470] Update README.md --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 9bd80dabb..527bbd00b 100644 --- a/README.md +++ b/README.md @@ -9,8 +9,8 @@ * sql增加临时表功能 * 增加redis维表功能 * 增加mongdb维表功能 - * 增加oracle维表功能 - * 增加SQlServer维表功能 + * 增加oracle维表,结果表功能 + * 增加SQlServer维表,结果表功能 ## 1 快速起步 ### 1.1 运行模式 From b126ecfbeb424141dbf7cd6dd8cd2d4b26ec026d Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 13 Sep 2018 15:03:33 +0800 Subject: [PATCH 011/470] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 527bbd00b..b8750d3bd 100644 --- a/README.md +++ b/README.md @@ -8,7 +8,7 @@ * 增加全局缓存功能 * sql增加临时表功能 * 增加redis维表功能 - * 增加mongdb维表功能 + * 增加mongodb维表功能 * 增加oracle维表,结果表功能 * 增加SQlServer维表,结果表功能 From 1c12be39861d32e497bb416030b8babf6241b255 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 13 Sep 2018 16:18:38 +0800 Subject: [PATCH 012/470] Update README.md --- README.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/README.md b/README.md index b8750d3bd..3f0be08e4 100644 --- a/README.md +++ b/README.md @@ -191,3 +191,7 @@ into a.channel = 'xc2' and a.pv=10 ) as d ``` + +# 招聘 +1.大数据平台开发工程师,想了解岗位详细信息可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至sishu@dtstack.com。 + From d7df27f71d299f800679dfa3a60880f41fbf6dc9 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 13 Sep 2018 16:48:33 +0800 Subject: [PATCH 013/470] =?UTF-8?q?=E7=BC=96=E8=BE=91readme.md?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/elasticsearchSink.md | 52 +++++++++++++++++++++++++++++++++++++++ docs/hbaseSide.md | 39 ++++++++++++++++++----------- docs/hbaseSink.md | 32 +++++++++++++++--------- docs/kafka09source.md | 43 ++++++++++++++++++++------------ docs/mysqlSide.md | 43 ++++++++++++++++++++------------ docs/mysqlSink.md | 33 ++++++++++++++++--------- 6 files changed, 173 insertions(+), 69 deletions(-) diff --git a/docs/elasticsearchSink.md b/docs/elasticsearchSink.md index e69de29bb..59fc56b54 100644 --- a/docs/elasticsearchSink.md +++ b/docs/elasticsearchSink.md @@ -0,0 +1,52 @@ +## 1.格式: +``` +CREATE TABLE tableName( + colName colType, + bb INT + )WITH( + type ='elasticsearch', + address ='ip:port[,ip:port]', + cluster='clusterName', + estype ='esType', + index ='index', + id ='num[,num]', + parallelism ='1' + ) +``` +## 2.支持的版本 + ES5 + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +|tableName|在 sql 中使用的名称;即注册到flink-table-env上的名称| +|colName|列名称| +|colType|列类型 [colType支持的类型](colType.md)| + +## 4.参数: +|参数名称|含义|是否必填| +|----|---|---| +|type|表明 输出表类型[mysql|hbase|elasticsearch]|是| +|address | 连接ES Transport地址(tcp地址)|是| +|cluster | ES 集群名称 |是| +|index | 选择的ES上的index名称|是| +|estype | 选择ES上的type名称|是| +|id | 生成id的规则(当前是根据指定的字段pos获取字段信息,拼接生成id)|是| +|parallelism | 并行度设置|否| + +## 5.样例: +``` +CREATE TABLE MyResult( + aa INT, + bb INT + )WITH( + type ='elasticsearch', + address ='172.16.10.47:9500', + cluster='es_47_menghan', + estype ='type1', + index ='xc_es_test', + id ='0,1', + parallelism ='1' + ) + ``` \ No newline at end of file diff --git a/docs/hbaseSide.md b/docs/hbaseSide.md index 4e99bcfbb..0b2975d28 100644 --- a/docs/hbaseSide.md +++ b/docs/hbaseSide.md @@ -18,30 +18,39 @@ partitionedJoin='false' ); ``` - -## 2.参数 +## 2.支持版本 + hbase2.0 -* tableName ==> 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同) -* columnFamily:columnName ==> hbase中的列族名称和列名称 -* alias ===> hbase 中的列对应到flink中注册的列名称 -* PERIOD FOR SYSTEM_TIME ==> 关键字表明该定义的表为维表信息 -* PRIMARY KEY(keyInfo) ==> 维表主键定义;hbase 维表为rowkey的构造方式; - 可选择的构造包括 md5(alias + alias), '常量',也包括上述方式的自由组合 +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName | 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同)| +| columnFamily:columnName | hbase中的列族名称和列名称 | +| alias | hbase 中的列对应到flink中注册的列名称 | +| PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| +| PRIMARY KEY(keyInfo) | 维表主键定义;hbase 维表rowkey的构造方式;可选择的构造包括 md5(alias + alias), '常量',也包括上述方式的自由组合 | + +## 3.参数 -* type ==> 表明维表的类型[hbase|mysql] -* zookeeperQuorum ==> hbase 的zk地址;格式ip:port[;ip:port] -* zookeeperParent ==> hbase 的zk parent路径 -* tableName ==> hbase 的表名称 -* cache ==> 维表缓存策略(NONE/LRU) +|参数名称|含义|是否必填| +|----|---|---| +| type | 表明维表的类型[hbase|mysql]|是| +| zookeeperQuorum | hbase 的zk地址;格式ip:port[;ip:port]|是| +| zookeeperParent | hbase 的zk parent路径|是| +| tableName | hbase 的表名称|是| +| cache | 维表缓存策略(NONE/LRU)|否| +| partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否| +-------------- + > 缓存策略 > * NONE: 不做内存缓存 > * LRU: > > cacheSize ==> 缓存的条目数量 > > cacheTTLMs ==> 缓存的过期时间(ms) -* partitionedJoin ==> 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量) -## 3.样例 +## 4.样例 ``` CREATE TABLE sideTable( cf:name String as name, diff --git a/docs/hbaseSink.md b/docs/hbaseSink.md index 9ae98edfe..cc0f2f6d0 100644 --- a/docs/hbaseSink.md +++ b/docs/hbaseSink.md @@ -15,20 +15,30 @@ CREATE TABLE MyResult( ``` -## 2.参数: - * tableName ==> 在 sql 中使用的名称;即注册到flink-table-env上的名称 - * colFamily:colName ==> hbase中的列族名称和列名称 - * colType ==> 列类型 [colType支持的类型](colType.md) +## 2.支持版本 +hbase2.0 + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName | 在 sql 中使用的名称;即注册到flink-table-env上的名称 +| colFamily:colName | hbase中的列族名称和列名称 +| colType | 列类型 [colType支持的类型](colType.md) + +## 4.参数: - * type ==> 表明 输出表类型[mysql|hbase|elasticsearch] - * zookeeperQuorum ==> hbase zk地址,多个直接用逗号隔开 - * zookeeperParent ==> zkParent 路径 - * tableName ==> 关联的hbase表名称 - * rowKey ==> hbase的rowkey关联的列信息 - * parallelism ==> 并行度设置 +|参数名称|含义|是否必填| +|----|---|---| +|type | 表明 输出表类型[mysql|hbase|elasticsearch]|是| +|zookeeperQuorum | hbase zk地址,多个直接用逗号隔开|是| +|zookeeperParent | zkParent 路径|是| +|tableName | 关联的hbase表名称|是| +|rowKey | hbase的rowkey关联的列信息|是| +|parallelism | 并行度设置|否| -## 3.样例: +## 5.样例: ``` CREATE TABLE MyResult( cf:channel STRING, diff --git a/docs/kafka09source.md b/docs/kafka09source.md index ca2607ca0..223b2adfc 100644 --- a/docs/kafka09source.md +++ b/docs/kafka09source.md @@ -14,23 +14,34 @@ CREATE TABLE tableName( parallelism ='parllNum' ); ``` -## 2.参数: - * tableName ==> 在 sql 中使用的名称;即注册到flink-table-env上的名称 - * colName ==> 列名称 - * colType ==> 列类型 [colType支持的类型](colType.md) - * function(colNameX) as aliasName ==> 支持在定义列信息的时候根据已有列类型生成新的列(函数可以使用系统函数和已经注册的UDF) - * WATERMARK FOR colName AS withOffset( colName , delayTime ) ==> 标识输入流生的watermake生成规则,根据指定的colName(当前支持列的类型为Long|Timestamp) - 和delayTime生成waterMark 同时会在注册表的使用附带上rowtime字段(如果未指定则默认添加proctime字段); - 注意:添加该标识的使用必须设置系统参数 time.characteristic:EventTime; delayTime: 数据最大延迟时间(ms) + +## 2.支持的版本 + kafka09 + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName | 在 sql 中使用的名称;即注册到flink-table-env上的名称| +| colName | 列名称| +| colType | 列类型 [colType支持的类型](colType.md)| +| function(colNameX) as aliasName | 支持在定义列信息的时候根据已有列类型生成新的列(函数可以使用系统函数和已经注册的UDF)| +| WATERMARK FOR colName AS withOffset( colName , delayTime ) | 标识输入流生的watermake生成规则,根据指定的colName(当前支持列的类型为Long or Timestamp) +和delayTime生成waterMark 同时会在注册表的使用附带上rowtime字段(如果未指定则默认添加proctime字段); +注意:添加该标识的使用必须设置系统参数 time.characteristic:EventTime; delayTime: 数据最大延迟时间(ms)| + +## 4.参数: + +|参数名称|含义|是否必填|默认值| +|----|---|---|---| +|type | kafka09 | 是|| +|bootstrapServers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| +|zookeeperQuorum | kafka zk地址信息(多个之间用逗号分隔)|是|| +|topic | 需要读取的 topic 名称|是|| +|offsetReset | 读取的topic 的offset初始位置[latest\|earliest]|否|latest| +|parallelism | 并行度设置|否|1| - * type ==> kafka09 - * bootstrapServers ==> kafka bootstrap-server 地址信息(多个用逗号隔开) - * zookeeperQuorum ==> kafka zk地址信息(多个之间用逗号分隔) - * topic ==> 需要读取的 topic 名称 - * offsetReset ==> 读取的topic 的offset初始位置[latest|earliest] - * parallelism ==> 并行度设置 - -## 3.样例: +## 5.样例: ``` CREATE TABLE MyTable( name string, diff --git a/docs/mysqlSide.md b/docs/mysqlSide.md index 552b9cb65..10cf0745a 100644 --- a/docs/mysqlSide.md +++ b/docs/mysqlSide.md @@ -19,31 +19,42 @@ partitionedJoin='false' ); ``` - -## 2.参数 - * tableName ==> 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同) - * colName ==> 列名称 - * colType ==> 列类型 [colType支持的类型](colType.md) - * PERIOD FOR SYSTEM_TIME ==> 关键字表明该定义的表为维表信息 - * PRIMARY KEY(keyInfo) ==> 维表主键定义;多个列之间用逗号隔开 - * url ==> 连接mysql数据库 jdbcUrl - * userName ==> mysql连接用户名 - * password ==> mysql连接密码 - * tableName ==> mysql表名称 - * type ==> 表明维表的类型[hbase|mysql] +# 2.支持版本 + mysql-5.6.35 + +## 3.表结构定义 - * tableName ==> mysql 的表名称 - * cache ==> 维表缓存策略(NONE/LRU) + |参数名称|含义| + |----|---| + | tableName | 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同)| + | colName | 列名称| + | colType | 列类型 [colType支持的类型](colType.md)| + | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| + | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| + +## 4.参数 + + |参数名称|含义|是否必填|默认值| + |----|---|---|----| + | type | 表明维表的类型[hbase\|mysql] |是|| + | url | 连接mysql数据库 jdbcUrl |是|| + | userName | mysql连接用户名 |是|| + | password | mysql连接密码|是|| + | tableName | mysql表名称|是|| + | tableName | mysql 的表名称|是|| + | cache | 维表缓存策略(NONE/LRU)|否|NONE| + | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| + ---------- + > 缓存策略 > * NONE: 不做内存缓存 > * LRU: > > cacheSize ==> 缓存的条目数量 > > cacheTTLMs ==> 缓存的过期时间(ms) - * partitionedJoin ==> 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量) -## 3.样例 +## 5.样例 ``` create table sideTable( channel String, diff --git a/docs/mysqlSink.md b/docs/mysqlSink.md index 544c4d377..68fa10cf2 100644 --- a/docs/mysqlSink.md +++ b/docs/mysqlSink.md @@ -15,18 +15,29 @@ CREATE TABLE tableName( ``` -## 2.参数: - * tableName ==> 在 sql 中使用的名称;即注册到flink-table-env上的名称 - * colName ==> 列名称 - * colType ==> 列类型 [colType支持的类型](colType.md) - * type ==> 表明 输出表类型[mysql|hbase|elasticsearch] - * url ==> 连接mysql数据库 jdbcUrl - * userName ==> mysql连接用户名 - * password ==> mysql连接密码 - * tableName ==> mysql表名称 - * parallelism ==> 并行度设置 +## 2.支持版本 + mysql-5.6.35 + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName| 在 sql 中使用的名称;即注册到flink-table-env上的名称| +| colName | 列名称| +| colType | 列类型 [colType支持的类型](colType.md)| + +## 4.参数: + +|参数名称|含义|是否必填| +|----|---|---| +|type |表明 输出表类型[mysql|hbase|elasticsearch]|是| +|url | 连接mysql数据库 jdbcUrl |是| +|userName | mysql连接用户名 |是| +| password | mysql连接密码|是| +| tableName | mysql表名称|是| +| parallelism | 并行度设置|否| -## 3.样例: +## 5.样例: ``` CREATE TABLE MyResult( channel VARCHAR, From 4d65e3d4fc06e6d0de4d5832c2df2b307f3ffc31 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 13 Sep 2018 16:55:15 +0800 Subject: [PATCH 014/470] =?UTF-8?q?=E7=BC=96=E8=BE=91readme.md?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/elasticsearchSink.md | 18 +++++++++--------- docs/hbaseSide.md | 26 +++++++++++++------------- docs/hbaseSink.md | 16 ++++++++-------- docs/kafka09source.md | 4 +--- docs/mysqlSide.md | 10 +++++----- docs/mysqlSink.md | 16 ++++++++-------- 6 files changed, 44 insertions(+), 46 deletions(-) diff --git a/docs/elasticsearchSink.md b/docs/elasticsearchSink.md index 59fc56b54..9a406245a 100644 --- a/docs/elasticsearchSink.md +++ b/docs/elasticsearchSink.md @@ -25,15 +25,15 @@ CREATE TABLE tableName( |colType|列类型 [colType支持的类型](colType.md)| ## 4.参数: -|参数名称|含义|是否必填| -|----|---|---| -|type|表明 输出表类型[mysql|hbase|elasticsearch]|是| -|address | 连接ES Transport地址(tcp地址)|是| -|cluster | ES 集群名称 |是| -|index | 选择的ES上的index名称|是| -|estype | 选择ES上的type名称|是| -|id | 生成id的规则(当前是根据指定的字段pos获取字段信息,拼接生成id)|是| -|parallelism | 并行度设置|否| +|参数名称|含义|是否必填|默认值| +|----|---|---|----| +|type|表明 输出表类型[mysql\|hbase\|elasticsearch]|是|| +|address | 连接ES Transport地址(tcp地址)|是|| +|cluster | ES 集群名称 |是|| +|index | 选择的ES上的index名称|是|| +|estype | 选择ES上的type名称|是|| +|id | 生成id的规则(当前是根据指定的字段pos获取字段信息,拼接生成id)|是|| +|parallelism | 并行度设置|否|1| ## 5.样例: ``` diff --git a/docs/hbaseSide.md b/docs/hbaseSide.md index 0b2975d28..39c1a04d2 100644 --- a/docs/hbaseSide.md +++ b/docs/hbaseSide.md @@ -33,21 +33,21 @@ ## 3.参数 -|参数名称|含义|是否必填| -|----|---|---| -| type | 表明维表的类型[hbase|mysql]|是| -| zookeeperQuorum | hbase 的zk地址;格式ip:port[;ip:port]|是| -| zookeeperParent | hbase 的zk parent路径|是| -| tableName | hbase 的表名称|是| -| cache | 维表缓存策略(NONE/LRU)|否| -| partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否| +|参数名称|含义|是否必填|默认值| +|----|---|---|----| +| type | 表明维表的类型[hbase\|mysql]|是|| +| zookeeperQuorum | hbase 的zk地址;格式ip:port[;ip:port]|是|| +| zookeeperParent | hbase 的zk parent路径|是|| +| tableName | hbase 的表名称|是|| +| cache | 维表缓存策略(NONE/LRU)|否|NONE| +| partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| -------------- - > 缓存策略 - > * NONE: 不做内存缓存 - > * LRU: - > > cacheSize ==> 缓存的条目数量 - > > cacheTTLMs ==> 缓存的过期时间(ms) +> 缓存策略 +> * NONE: 不做内存缓存 +> * LRU: +> > cacheSize ==> 缓存的条目数量 +> > cacheTTLMs ==> 缓存的过期时间(ms) ## 4.样例 diff --git a/docs/hbaseSink.md b/docs/hbaseSink.md index cc0f2f6d0..ff2f83091 100644 --- a/docs/hbaseSink.md +++ b/docs/hbaseSink.md @@ -28,14 +28,14 @@ hbase2.0 ## 4.参数: -|参数名称|含义|是否必填| -|----|---|---| -|type | 表明 输出表类型[mysql|hbase|elasticsearch]|是| -|zookeeperQuorum | hbase zk地址,多个直接用逗号隔开|是| -|zookeeperParent | zkParent 路径|是| -|tableName | 关联的hbase表名称|是| -|rowKey | hbase的rowkey关联的列信息|是| -|parallelism | 并行度设置|否| +|参数名称|含义|是否必填|默认值| +|----|---|---|-----| +|type | 表明 输出表类型[mysql\|hbase\|elasticsearch]|是|| +|zookeeperQuorum | hbase zk地址,多个直接用逗号隔开|是|| +|zookeeperParent | zkParent 路径|是|| +|tableName | 关联的hbase表名称|是|| +|rowKey | hbase的rowkey关联的列信息|是|| +|parallelism | 并行度设置|否|1| ## 5.样例: diff --git a/docs/kafka09source.md b/docs/kafka09source.md index 223b2adfc..e809ed916 100644 --- a/docs/kafka09source.md +++ b/docs/kafka09source.md @@ -26,9 +26,7 @@ CREATE TABLE tableName( | colName | 列名称| | colType | 列类型 [colType支持的类型](colType.md)| | function(colNameX) as aliasName | 支持在定义列信息的时候根据已有列类型生成新的列(函数可以使用系统函数和已经注册的UDF)| -| WATERMARK FOR colName AS withOffset( colName , delayTime ) | 标识输入流生的watermake生成规则,根据指定的colName(当前支持列的类型为Long or Timestamp) -和delayTime生成waterMark 同时会在注册表的使用附带上rowtime字段(如果未指定则默认添加proctime字段); -注意:添加该标识的使用必须设置系统参数 time.characteristic:EventTime; delayTime: 数据最大延迟时间(ms)| +| WATERMARK FOR colName AS withOffset( colName , delayTime ) | 标识输入流生的watermake生成规则,根据指定的colName(当前支持列的类型为Long \| Timestamp) 和delayTime生成waterMark 同时会在注册表的使用附带上rowtime字段(如果未指定则默认添加proctime字段);注意:添加该标识的使用必须设置系统参数 time.characteristic:EventTime; delayTime: 数据最大延迟时间(ms)| ## 4.参数: diff --git a/docs/mysqlSide.md b/docs/mysqlSide.md index 10cf0745a..cf41d5851 100644 --- a/docs/mysqlSide.md +++ b/docs/mysqlSide.md @@ -47,11 +47,11 @@ | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| ---------- - > 缓存策略 - > * NONE: 不做内存缓存 - > * LRU: - > > cacheSize ==> 缓存的条目数量 - > > cacheTTLMs ==> 缓存的过期时间(ms) + > 缓存策略 + > * NONE: 不做内存缓存 + > * LRU: + > > cacheSize ==> 缓存的条目数量 + > > cacheTTLMs ==> 缓存的过期时间(ms) ## 5.样例 diff --git a/docs/mysqlSink.md b/docs/mysqlSink.md index 68fa10cf2..192c4944a 100644 --- a/docs/mysqlSink.md +++ b/docs/mysqlSink.md @@ -28,14 +28,14 @@ CREATE TABLE tableName( ## 4.参数: -|参数名称|含义|是否必填| -|----|---|---| -|type |表明 输出表类型[mysql|hbase|elasticsearch]|是| -|url | 连接mysql数据库 jdbcUrl |是| -|userName | mysql连接用户名 |是| -| password | mysql连接密码|是| -| tableName | mysql表名称|是| -| parallelism | 并行度设置|否| +|参数名称|含义|是否必填|默认值| +|----|----|----|----| +|type |表明 输出表类型[mysql\|hbase\|elasticsearch]|是|| +|url | 连接mysql数据库 jdbcUrl |是|| +|userName | mysql连接用户名 |是|| +| password | mysql连接密码|是|| +| tableName | mysql表名称|是|| +| parallelism | 并行度设置|否|1| ## 5.样例: ``` From b990a20e07a253cb9351ce338e6036b167b3f328 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 13 Sep 2018 16:58:28 +0800 Subject: [PATCH 015/470] =?UTF-8?q?=E7=BC=96=E8=BE=91readme.md?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/hbaseSide.md | 9 ++++----- docs/mysqlSide.md | 8 ++++---- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/docs/hbaseSide.md b/docs/hbaseSide.md index 39c1a04d2..60f0ef768 100644 --- a/docs/hbaseSide.md +++ b/docs/hbaseSide.md @@ -44,11 +44,10 @@ -------------- > 缓存策略 -> * NONE: 不做内存缓存 -> * LRU: -> > cacheSize ==> 缓存的条目数量 -> > cacheTTLMs ==> 缓存的过期时间(ms) - + * NONE: 不做内存缓存 + * LRU: + * cacheSize: 缓存的条目数量 + * cacheTTLMs:缓存的过期时间(ms) ## 4.样例 ``` diff --git a/docs/mysqlSide.md b/docs/mysqlSide.md index cf41d5851..f374a51c6 100644 --- a/docs/mysqlSide.md +++ b/docs/mysqlSide.md @@ -48,10 +48,10 @@ ---------- > 缓存策略 - > * NONE: 不做内存缓存 - > * LRU: - > > cacheSize ==> 缓存的条目数量 - > > cacheTTLMs ==> 缓存的过期时间(ms) + * NONE: 不做内存缓存 + * LRU: + * cacheSize: 缓存的条目数量 + * cacheTTLMs:缓存的过期时间(ms) ## 5.样例 From d2ccc4ec89051b39aca4368c001a9034a49e0025 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 13 Sep 2018 17:01:18 +0800 Subject: [PATCH 016/470] =?UTF-8?q?=E7=BC=96=E8=BE=91readme.md?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/{kafka09source.md => kafka09Source.md} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename docs/{kafka09source.md => kafka09Source.md} (100%) diff --git a/docs/kafka09source.md b/docs/kafka09Source.md similarity index 100% rename from docs/kafka09source.md rename to docs/kafka09Source.md From 703ddc6b361a55cf22149371958587dfedd9c8b1 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 14 Sep 2018 11:10:18 +0800 Subject: [PATCH 017/470] Update README.md --- README.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/README.md b/README.md index 06e590de8..5d76d9d18 100644 --- a/README.md +++ b/README.md @@ -3,6 +3,11 @@ > > * 自定义create table 语法(包括源表,输出表,维表) > > * 自定义create function 语法 > > * 实现了流与维表的join + +# 已支持 + * 源表:kafka 0.9,1.x版本 + * 维表:mysql,hbase + * 结果表:mysql,hbase,elasticsearch5.x # 后续开发计划 * 增加全局缓存功能 From 167ca858d0f91926faac7abc61f2a85df887116e Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 14 Sep 2018 14:02:22 +0800 Subject: [PATCH 018/470] update pom.xml --- hbase/hbase-side/pom.xml | 16 +--------------- hbase/hbase-sink/pom.xml | 38 +++++++++++++++++++------------------- hbase/pom.xml | 22 ++++++++++++++++++++++ mysql/mysql-side/pom.xml | 13 ------------- mysql/mysql-sink/pom.xml | 20 -------------------- mysql/pom.xml | 20 ++++++++++++++++++++ 6 files changed, 62 insertions(+), 67 deletions(-) diff --git a/hbase/hbase-side/pom.xml b/hbase/hbase-side/pom.xml index bf68e70b4..b73c99dea 100644 --- a/hbase/hbase-side/pom.xml +++ b/hbase/hbase-side/pom.xml @@ -11,19 +11,8 @@ sql.side.hbase hbase-side + jar - - com.dtstack.flinkx - sql.core - 1.0-SNAPSHOT - provided - - - - org.apache.hbase - hbase-client - 1.3.1 - org.hbase @@ -32,9 +21,6 @@ - - jar - diff --git a/hbase/hbase-sink/pom.xml b/hbase/hbase-sink/pom.xml index 193128a61..2d2f9e73d 100644 --- a/hbase/hbase-sink/pom.xml +++ b/hbase/hbase-sink/pom.xml @@ -12,27 +12,27 @@ sql.sink.hbase hbase-sink - - - junit - junit - 3.8.1 - test - + + + + + + + - - com.dtstack.flinkx - sql.core - 1.0-SNAPSHOT - provided - + + + + + + - - org.apache.hbase - hbase-client - 1.3.1 - - + + + + + + diff --git a/hbase/pom.xml b/hbase/pom.xml index 774b001ab..a34a59b11 100644 --- a/hbase/pom.xml +++ b/hbase/pom.xml @@ -16,5 +16,27 @@ hbase-side + + + junit + junit + 3.8.1 + test + + + com.dtstack.flinkx + sql.core + 1.0-SNAPSHOT + provided + + + + org.apache.hbase + hbase-client + 1.3.1 + + + + \ No newline at end of file diff --git a/mysql/mysql-side/pom.xml b/mysql/mysql-side/pom.xml index b8e2d92fc..031a37072 100644 --- a/mysql/mysql-side/pom.xml +++ b/mysql/mysql-side/pom.xml @@ -13,13 +13,6 @@ sql.side.mysql mysql-side - - com.dtstack.flinkx - sql.core - 1.0-SNAPSHOT - provided - - @@ -33,12 +26,6 @@ vertx-core 3.5.2 - - - mysql - mysql-connector-java - 5.1.46 - jar diff --git a/mysql/mysql-sink/pom.xml b/mysql/mysql-sink/pom.xml index 7df86cfe6..9a56f2712 100644 --- a/mysql/mysql-sink/pom.xml +++ b/mysql/mysql-sink/pom.xml @@ -20,31 +20,11 @@ - - junit - junit - 3.8.1 - test - - - - com.dtstack.flinkx - sql.core - 1.0-SNAPSHOT - provided - - org.apache.flink flink-jdbc ${flink.version} - - - mysql - mysql-connector-java - 5.1.17 - diff --git a/mysql/pom.xml b/mysql/pom.xml index fa1c2df63..aa2d20ab4 100644 --- a/mysql/pom.xml +++ b/mysql/pom.xml @@ -16,5 +16,25 @@ mysql-side + + + junit + junit + 3.8.1 + test + + + com.dtstack.flinkx + sql.core + 1.0-SNAPSHOT + provided + + + mysql + mysql-connector-java + 5.1.46 + + + \ No newline at end of file From c55b933db6e9f800e3feaa2f15aef3c82074095e Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 14 Sep 2018 14:28:59 +0800 Subject: [PATCH 019/470] comment --- core/src/main/java/com/dtstack/flink/sql/Main.java | 8 ++++---- .../java/com/dtstack/flink/sql/side/AsyncReqRow.java | 10 ++++------ .../java/com/dtstack/flink/sql/side/CacheMissVal.java | 2 +- .../com/dtstack/flink/sql/side/ParserJoinField.java | 4 ++-- .../java/com/dtstack/flink/sql/side/SideSQLParser.java | 10 +++++----- .../java/com/dtstack/flink/sql/side/SideSqlExec.java | 8 ++++---- .../java/com/dtstack/flink/sql/side/SideTableInfo.java | 2 +- .../com/dtstack/flink/sql/sink/StreamSinkFactory.java | 2 +- .../dtstack/flink/sql/source/IStreamSourceGener.java | 1 - .../dtstack/flink/sql/source/StreamSourceFactory.java | 4 ++-- .../dtstack/flink/sql/table/AbsSideTableParser.java | 2 +- .../com/dtstack/flink/sql/table/AbsSourceParser.java | 2 +- .../com/dtstack/flink/sql/table/AbsTableParser.java | 2 +- .../flink/sql/table/TableInfoParserFactory.java | 8 ++++---- .../java/com/dtstack/flink/sql/util/ByteUtils.java | 1 - .../com/dtstack/flink/sql/util/ConfigConstrant.java | 3 ++- .../java/com/dtstack/flink/sql/util/DtStringUtil.java | 4 ++-- .../main/java/com/dtstack/flink/sql/util/MD5Utils.java | 1 - .../main/java/com/dtstack/flink/sql/util/MathUtil.java | 1 - .../sql/watermarker/CustomerWaterMarkerForLong.java | 2 +- .../watermarker/CustomerWaterMarkerForTimeStamp.java | 2 +- .../flink/sql/watermarker/WaterMarkerAssigner.java | 2 +- 22 files changed, 38 insertions(+), 43 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 04a7997a0..e4cd6efb9 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -116,9 +116,9 @@ public static void main(String[] args) throws Exception { String deployMode = cl.getOptionValue("mode"); String confProp = cl.getOptionValue("confProp"); - Preconditions.checkNotNull(sql, "it requires input parameters sql"); - Preconditions.checkNotNull(name, "it requires input parameters name"); - Preconditions.checkNotNull(localSqlPluginPath, "it requires input parameters localSqlPluginPath"); + Preconditions.checkNotNull(sql, "parameters of sql is required"); + Preconditions.checkNotNull(name, "parameters of name is required"); + Preconditions.checkNotNull(localSqlPluginPath, "parameters of localSqlPluginPath is required"); sql = URLDecoder.decode(sql, Charsets.UTF_8.name()); SqlParser.setLocalSqlPluginRoot(localSqlPluginPath); @@ -133,7 +133,7 @@ public static void main(String[] args) throws Exception { DtClassLoader dtClassLoader = new DtClassLoader(new URL[]{}, threadClassLoader); Thread.currentThread().setContextClassLoader(dtClassLoader); - URLClassLoader parentClassloader = null; + URLClassLoader parentClassloader; if(!LOCAL_MODE.equals(deployMode)){ parentClassloader = (URLClassLoader) threadClassLoader.getParent(); }else{ diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java index 2007b3a0f..a335c9a8b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java @@ -44,9 +44,8 @@ import java.util.Map; /** - * 所有的继承接口命名规则:类型 + "AsyncReqRow" 比如==》MysqlAsyncReqRow - * 当前只支持Left join / inner join(join) - * FIXME 不支持right join + * All interfaces inherit naming rules: type + "AsyncReqRow" such as == "MysqlAsyncReqRow + * only support Left join / inner join(join),not support right join * Date: 2018/7/9 * Company: www.dtstack.com * @author xuchao @@ -72,14 +71,13 @@ public abstract class AsyncReqRow extends RichAsyncFunction { protected JoinType joinType; - //key:返回值位置,返回值在输入数据中的索引位置 + //key:Returns the value of the position, returns the index values ​​in the input data protected Map inFieldIndex = Maps.newHashMap(); protected Map sideFieldIndex = Maps.newHashMap(); protected SideTableInfo sideTableInfo; - //TODO 需要指定类型 protected AbsSideCache sideCache; public AsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, @@ -202,7 +200,7 @@ public void dealOneEqualCon(SqlNode sqlNode, String sideTableName){ protected void dealMissKey(Row input, ResultFuture resultFuture){ if(joinType == JoinType.LEFT){ - //保留left 表数据 + //Reserved left table data Row row = fillData(input, null); resultFuture.complete(Collections.singleton(row)); }else{ diff --git a/core/src/main/java/com/dtstack/flink/sql/side/CacheMissVal.java b/core/src/main/java/com/dtstack/flink/sql/side/CacheMissVal.java index ebcbddcbb..f4beb7fae 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/CacheMissVal.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/CacheMissVal.java @@ -24,7 +24,7 @@ import com.dtstack.flink.sql.side.cache.CacheObj; /** - * 仅仅用来标记未命中的维表数据 + * Only the data marked to dimension table miss * Date: 2018/8/28 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java b/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java index db6cd608f..df242a390 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java @@ -33,7 +33,7 @@ import java.util.List; /** - *FIXME 需要考虑是直接返回所有的字段然后在外层再包裹原先的查询? + * Need to consider is the direct return to the fields and then all wrapped in the outer layer of the original query? * Date: 2018/7/20 * Company: www.dtstack.com * @author xuchao @@ -42,7 +42,7 @@ public class ParserJoinField { /** - * 需要解析出selectlist和where中的字段信息 + * Need to parse the fields of information and where selectlist * @return */ public static List getRowTypeInfo(SqlNode sqlNode, JoinScope scope, boolean getAll){ diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 6342f70a0..7f165ac30 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -43,7 +43,7 @@ import static org.apache.calcite.sql.SqlKind.*; /** - * 解析sql,获取维表的执行信息 + * Parsing sql, obtain execution information dimension table * Date: 2018/7/24 * Company: www.dtstack.com * @author xuchao @@ -169,19 +169,19 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< private void dealSelectResultWithJoinInfo(JoinInfo joinInfo, SqlSelect sqlNode, Queue queueInfo){ - //SideJoinInfo重命名 + //SideJoinInfo rename if(joinInfo.checkIsSide()){ joinInfo.setSelectFields(sqlNode.getSelectList()); joinInfo.setSelectNode(sqlNode); if(joinInfo.isRightIsSideTable()){ - //判断left是不是一个简单表 + //Analyzing left is not a simple table if(joinInfo.getLeftNode().toString().contains("SELECT")){ queueInfo.offer(joinInfo.getLeftNode()); } queueInfo.offer(joinInfo); }else{ - //判断right是不是一个简单表 + //Determining right is not a simple table if(joinInfo.getRightNode().getKind() == SELECT){ queueInfo.offer(joinInfo.getLeftNode()); } @@ -189,7 +189,7 @@ private void dealSelectResultWithJoinInfo(JoinInfo joinInfo, SqlSelect sqlNode, queueInfo.offer(joinInfo); } - //更新from 节点 + //Update from node SqlOperator operator = new SqlAsOperator(); SqlParserPos sqlParserPos = new SqlParserPos(0, 0); String joinLeftTableName = joinInfo.getLeftTableName(); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index f290e88b8..57f05c0d1 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -79,7 +79,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl Queue exeQueue = sideSQLParser.getExeQueue(sql, sideTableMap.keySet()); Object pollObj = null; - //TODO 需要清理 + //need clean boolean preIsSideJoin = false; List replaceInfoList = Lists.newArrayList(); @@ -150,7 +150,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl RowTypeInfo typeInfo = new RowTypeInfo(targetTable.getSchema().getTypes(), targetTable.getSchema().getColumnNames()); DataStream adaptStream = tableEnv.toAppendStream(targetTable, org.apache.flink.types.Row.class); - //join side table 之前先 keyby ===>减少 维表在各个async 的缓存大小 + //join side table before keyby ===> Reducing the size of each dimension table cache of async if(sideTableInfo.isPartitionedJoin()){ List leftJoinColList = getConditionFields(joinInfo.getCondition(), joinInfo.getLeftTableAlias()); String[] leftJoinColArr = new String[leftJoinColList.size()]; @@ -159,7 +159,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl } AsyncReqRow asyncDbReq = loadAsyncReq(sideTableInfo.getType(), localSqlPluginPath, typeInfo, joinInfo, sideJoinFieldInfo, sideTableInfo); - //TODO 并行度应该设置为多少?超时时间设置? capacity设置? + //TODO How much should be set for the degree of parallelism? Timeout? capacity settings? DataStream dsOut = AsyncDataStream.orderedWait(adaptStream, asyncDbReq, 10000, TimeUnit.MILLISECONDS, 10) .setParallelism(sideTableInfo.getParallelism()); @@ -526,7 +526,7 @@ private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable primaryKeys){ diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java index c840eee10..58dc79de2 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java @@ -49,7 +49,7 @@ public abstract class SideTableInfo extends TableInfo implements Serializable { private int cacheSize = 10000; - private long cacheTimeout = 60 * 1000;//默认1分钟 + private long cacheTimeout = 60 * 1000;// private boolean partitionedJoin = false; diff --git a/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java b/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java index cc7ed0179..238b4f4e5 100644 --- a/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java @@ -27,7 +27,7 @@ import org.apache.flink.table.sinks.TableSink; /** - * 根据指定的sink type 加载jar,并初始化对象 + * Loads jar and initializes the object according to the specified sink type * Date: 2017/3/10 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/source/IStreamSourceGener.java b/core/src/main/java/com/dtstack/flink/sql/source/IStreamSourceGener.java index 530900dfe..b8b8f6edc 100644 --- a/core/src/main/java/com/dtstack/flink/sql/source/IStreamSourceGener.java +++ b/core/src/main/java/com/dtstack/flink/sql/source/IStreamSourceGener.java @@ -33,7 +33,6 @@ public interface IStreamSourceGener { /** - * 获取输入源 * @param sourceTableInfo * @param env * @param tableEnv diff --git a/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java b/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java index 458d8bc69..dca9e0fa3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java @@ -30,7 +30,7 @@ import org.apache.flink.table.api.java.StreamTableEnvironment; /** - * 创建streamTableSource + * streamTableSource * Date: 2017/3/10 * Company: www.dtstack.com * @author xuchao @@ -58,7 +58,7 @@ public static AbsSourceParser getSqlParser(String resultType, String sqlRootDir) } /** - * 根据指定的类型构造数据源 + * The configuration of the type specified data source * @param sourceTableInfo * @return */ diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java index 82c9af96e..e46a72494 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java @@ -35,7 +35,7 @@ public abstract class AbsSideTableParser extends AbsTableParser { - //解析create table的属性==>获取cache信息 + //Analytical create table attributes ==> Get information cache protected void parseCacheProp(SideTableInfo sideTableInfo, Map props){ if(props.containsKey(SideTableInfo.CACHE_KEY.toLowerCase())){ String cacheType = MathUtil.getString(props.get(SideTableInfo.CACHE_KEY.toLowerCase())); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java index aeb77679c..1b676ff8d 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java @@ -61,7 +61,7 @@ static void dealVirtualField(Matcher matcher, TableInfo tableInfo){ static void dealWaterMark(Matcher matcher, TableInfo tableInfo){ SourceTableInfo sourceTableInfo = (SourceTableInfo) tableInfo; String eventTimeField = matcher.group(1); - //FIXME 暂时不解析第二个row_time_field参数 + //FIXME Temporarily resolve the second parameter row_time_field Integer offset = MathUtil.getIntegerVal(matcher.group(3)); sourceTableInfo.setEventTimeField(eventTimeField); sourceTableInfo.setMaxOutOrderness(offset); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index 108e776bf..2fc2d799b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -97,7 +97,7 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ throw new RuntimeException(String.format("table [%s] field [%s] format error.", tableInfo.getName(), fieldRow)); } - //兼容可能在fieldName中出现空格的情况 + //Compatible situation may arise in space in the fieldName String[] filedNameArr = new String[filedInfoArr.length - 1]; System.arraycopy(filedInfoArr, 0, filedNameArr, 0, filedInfoArr.length - 1); String fieldName = String.join(" ", filedNameArr); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java index 73e255340..ae53743bd 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java @@ -34,7 +34,7 @@ import java.util.regex.Pattern; /** - * 解析创建表语句获得具体的表结构信息 + * Create table statement parsing table structure to obtain specific information * Date: 2018/6/25 * Company: www.dtstack.com * @author xuchao @@ -54,7 +54,7 @@ public class TableInfoParserFactory { private static Map sideTableInfoMap = Maps.newConcurrentMap(); - //加载插件中的解析 + //Parsing loaded plugin public static TableInfo parseWithTableType(int tableType, CreateTableParser.SqlParserResult parserResult, String localPluginRoot) throws Exception { AbsTableParser absTableParser = null; @@ -96,14 +96,14 @@ public static TableInfo parseWithTableType(int tableType, CreateTableParser.SqlP Map prop = Maps.newHashMap(); - //屏蔽大小写 + //Shield case parserResult.getPropMap().forEach((key,val) -> prop.put(key.toLowerCase(), val)); return absTableParser.getTableInfo(parserResult.getTableName(), parserResult.getFieldsInfoStr(), prop); } /** - * 如果表字段中包含PERIOD FOR SYSTEM_TIME则认为是一个维表 + * judge dim table of PERIOD FOR SYSTEM_TIME * @param tableField * @return */ diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ByteUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/ByteUtils.java index 05792e044..9642ea5dc 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ByteUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ByteUtils.java @@ -25,7 +25,6 @@ import java.nio.ByteBuffer; /** - * byte 转换工具 * Date: 2018/8/28 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ConfigConstrant.java b/core/src/main/java/com/dtstack/flink/sql/util/ConfigConstrant.java index 7d37d4003..160d8f411 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ConfigConstrant.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ConfigConstrant.java @@ -22,7 +22,8 @@ /** - * + * @Date: 2018年09月14日 下午14:23:37 + * @Company: www.dtstack.com * @author sishu.yss * */ diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index b54c3479f..f0b9dfc02 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -36,7 +36,7 @@ public class DtStringUtil { /** - * 根据指定分隔符分割字符串---忽略在引号里面的分隔符 + * Split the specified string delimiter --- ignored quotes delimiter * @param str * @param delimiter * @return @@ -73,7 +73,7 @@ public static List splitIgnoreQuota(String str, char delimiter){ } /*** - * 根据指定分隔符分割字符串---忽略在引号 和 括号 里面的分隔符 + * Split the specified string delimiter --- ignored in brackets and quotation marks delimiter * @param str * @param delimter * @return diff --git a/core/src/main/java/com/dtstack/flink/sql/util/MD5Utils.java b/core/src/main/java/com/dtstack/flink/sql/util/MD5Utils.java index 861f5ae5b..e248690f4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/MD5Utils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/MD5Utils.java @@ -27,7 +27,6 @@ import java.security.NoSuchAlgorithmException; /** - * Md5工具类 * Date: 2018/8/23 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/util/MathUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/MathUtil.java index cea0751e7..829cea461 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/MathUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/MathUtil.java @@ -21,7 +21,6 @@ package com.dtstack.flink.sql.util; /** - * 数字转换 * Date: 2017/4/21 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java index cbd050c27..fe8904efd 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java @@ -28,7 +28,7 @@ import org.slf4j.LoggerFactory; /** - * 自定义watermark---用于eventtime + * Custom watermark --- for eventtime * Date: 2017/12/28 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java index 47b988c24..aa6810066 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java @@ -31,7 +31,7 @@ import java.text.SimpleDateFormat; /** - * 自定义watermark---用于eventtime + * Custom watermark --- for eventtime * Date: 2017/12/28 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java index 2ef0d6a79..b095ea196 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java @@ -32,7 +32,7 @@ import org.apache.flink.util.Preconditions; /** - * 指定流的watermarker + * define watermarker * Date: 2018/6/29 * Company: www.dtstack.com * @author xuchao From 9a63a46e77d0d1fd340fc40923c5cf98286cbf80 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 14 Sep 2018 14:43:33 +0800 Subject: [PATCH 020/470] comment --- .../com/dtstack/flink/sql/side/hbase/ReplaceOperator.java | 2 +- .../com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java | 2 +- .../dtstack/flink/sql/side/hbase/table/HbaseSideParser.java | 2 +- .../flink/sql/side/hbase/table/HbaseSideTableInfo.java | 2 +- .../java/com/dtstack/flink/sql/sink/hbase/HbaseSink.java | 6 +++++- .../dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java | 5 +++++ .../dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java | 6 +++++- .../sql/source/kafka09/CustomerJsonDeserialization.java | 2 +- .../java/com/dtstack/flink/sql/launcher/LauncherMain.java | 6 ++++++ .../com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java | 3 +-- 10 files changed, 27 insertions(+), 9 deletions(-) diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceOperator.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceOperator.java index ec33a8413..c3c9b5cfe 100644 --- a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceOperator.java +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceOperator.java @@ -49,7 +49,7 @@ public String replaceStr(Map refData){ } /** - * 根据函数提供自定义的处理 + * The processing function to provide custom * @param replaceStr * @return */ diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java index f64af3ff9..af2053fe6 100644 --- a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java @@ -30,7 +30,7 @@ import java.util.regex.Pattern; /** - * rowkey 生成规则 + * rowkey rule * Date: 2018/8/23 * Company: www.dtstack.com * @author xuchao diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java index 71c1c7bad..48bb9d66a 100644 --- a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java @@ -32,7 +32,7 @@ import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; /** - * 定义的hbase 字段信息必须包含别名-->sql里面不允许出现“:” + * hbase field information must include the definition of an alias -> sql which does not allow ":" * Date: 2018/8/21 * Company: www.dtstack.com * @author xuchao diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideTableInfo.java b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideTableInfo.java index 6b8715e07..377ca393f 100644 --- a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideTableInfo.java +++ b/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideTableInfo.java @@ -28,7 +28,7 @@ import java.util.Map; /** - * hbase 维表填写的属性 + * hbase-dimensional form of property * Date: 2018/8/21 * Company: www.dtstack.com * @author xuchao diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseSink.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseSink.java index e4a6b9422..03d460066 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseSink.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseSink.java @@ -35,7 +35,11 @@ import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; - +/** + * Date: 2018/09/14 + * Company: www.dtstack.com + * @author sishu.yss + */ public class HbaseSink implements RetractStreamTableSink, IStreamSinkGener { protected String[] fieldNames; diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java index 686b05019..d5a3fd4b7 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseSinkParser.java @@ -29,6 +29,11 @@ import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; +/** + * Date: 2018/09/14 + * Company: www.dtstack.com + * @author sishu.yss + */ public class HbaseSinkParser extends AbsTableParser { diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java index 3f827f102..4e286fd45 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java @@ -23,9 +23,13 @@ import com.dtstack.flink.sql.table.TargetTableInfo; import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; - import java.util.Map; +/** + * Date: 2018/09/14 + * Company: www.dtstack.com + * @author sishu.yss + */ public class HbaseTableInfo extends TargetTableInfo { private static final String CURR_TYPE = "hbase"; diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/CustomerJsonDeserialization.java index f3f04430f..7e1b4a10a 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/CustomerJsonDeserialization.java @@ -34,7 +34,7 @@ import java.util.Iterator; /** - * 自定义的json字符串解析 + * json string parsing custom * Date: 2017/5/28 * Company: www.dtstack.com * @author xuchao diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 786a44de1..3c4cdc57f 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -32,6 +32,12 @@ import static com.dtstack.flink.sql.launcher.LauncherOptions.OPTION_LOCAL_SQL_PLUGIN_PATH; import static com.dtstack.flink.sql.launcher.LauncherOptions.OPTION_MODE; +/** + * Date: 2017/2/20 + * Company: www.dtstack.com + * @author xuchao + */ + public class LauncherMain { private static final String CORE_JAR = "core.jar"; diff --git a/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java index 712b5673b..46cbb05d0 100644 --- a/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java +++ b/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -69,7 +69,6 @@ public class MysqlAsyncReqRow extends AsyncReqRow { private final static String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; - //TODO 这三个参数应该如何设置? private final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 10; private final static int DEFAULT_VERTX_WORKER_POOL_SIZE = 20; @@ -252,7 +251,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except mySQLClient.getConnection(conn -> { if (conn.failed()) { - //处理失败情况 + //Treatment failures resultFuture.completeExceptionally(conn.cause()); return; } From 0e999cc688e4f429afbe657aabb9d9efad589453 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 14 Sep 2018 16:06:48 +0800 Subject: [PATCH 021/470] remove comment --- .../main/java/com/dtstack/flink/sql/parser/SqlParser.java | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java index 82f33d2b0..b5a1fbc74 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java @@ -51,7 +51,7 @@ public static void setLocalSqlPluginRoot(String localSqlPluginRoot){ } /** - * ------flink 支持的 sql 语法包括-------- + * flink support sql syntax * CREATE TABLE sls_stream() with (); * CREATE (TABLE|SCALA) FUNCTION fcnName WITH com.dtstack.com; * insert into tb1 select * from tb2; @@ -76,12 +76,9 @@ public static SqlTree parseSql(String sql) throws Exception { SqlTree sqlTree = new SqlTree(); for(String childSql : sqlArr){ - - if(Strings.isNullOrEmpty(childSql)){ continue; } - boolean result = false; for(IParser sqlParser : sqlParserList){ if(!sqlParser.verify(childSql)){ @@ -99,7 +96,7 @@ public static SqlTree parseSql(String sql) throws Exception { //解析exec-sql if(sqlTree.getExecSqlList().size() == 0){ - throw new RuntimeException("sql中没有可执行语句"); + throw new RuntimeException("sql no executable statement"); } for(InsertSqlParser.SqlParseResult result : sqlTree.getExecSqlList()){ From cb28f52bcaad54d3ce0ae6a143e21d149b9d8cb2 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Tue, 18 Sep 2018 14:21:28 +0800 Subject: [PATCH 022/470] Update README.md --- README.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/README.md b/README.md index 5d76d9d18..a98acd785 100644 --- a/README.md +++ b/README.md @@ -37,8 +37,6 @@ ``` mvn clean package -Dmaven.test.skip - -所有插件包会生成到plugins下 ``` 打包结束后,项目根目录下会产生plugins目录,plugins目录下存放编译好的数据同步插件包 From 933926a385a7e97d95713c65af672bf9619cc184 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Tue, 18 Sep 2018 15:47:30 +0800 Subject: [PATCH 023/470] update --- .gitignore | 3 +- core/pom.xml | 2 +- elasticsearch/pom.xml | 19 --- .../elasticsearch5-sink}/pom.xml | 22 +--- .../sink/elasticsearch/CustomerSinkFunc.java | 0 .../sink/elasticsearch/ElasticsearchSink.java | 0 .../flink/sql/sink/elasticsearch/EsUtil.java | 0 .../table/ElasticsearchSinkParser.java | 0 .../table/ElasticsearchTableInfo.java | 0 elasticsearch5/pom.xml | 34 +++++ hbase/hbase-side/pom.xml | 2 +- hbase/hbase-sink/pom.xml | 24 +--- hbase/pom.xml | 4 +- kafka09/kafka09-source/pom.xml | 21 +-- .../CustomerJsonDeserialization.java | 2 +- .../KafkaSource.java} | 8 +- .../table/KafkaSourceParser.java} | 14 +- .../table/KafkaSourceTableInfo.java} | 6 +- kafka09/pom.xml | 18 ++- kafka10/kafka10-source/pom.xml | 93 ++++++++++++++ .../kafka/CustomerJsonDeserialization.java | 114 +++++++++++++++++ .../flink/sql/source/kafka/KafkaSource.java | 83 ++++++++++++ .../source/kafka/table/KafkaSourceParser.java | 51 ++++++++ .../kafka/table/KafkaSourceTableInfo.java | 120 ++++++++++++++++++ .../test/java/com/dtstack/flinkx/AppTest.java | 58 +++++++++ kafka10/pom.xml | 36 ++++++ kafka11/kafka11-source/pom.xml | 93 ++++++++++++++ .../kafka/CustomerJsonDeserialization.java | 114 +++++++++++++++++ .../flink/sql/source/kafka/KafkaSource.java | 83 ++++++++++++ .../source/kafka/table/KafkaSourceParser.java | 51 ++++++++ .../kafka/table/KafkaSourceTableInfo.java | 120 ++++++++++++++++++ .../test/java/com/dtstack/flinkx/AppTest.java | 58 +++++++++ kafka11/pom.xml | 36 ++++++ launcher/pom.xml | 4 +- mysql/mysql-side/pom.xml | 2 +- mysql/mysql-sink/pom.xml | 2 +- mysql/pom.xml | 4 +- pom.xml | 4 +- 38 files changed, 1199 insertions(+), 106 deletions(-) delete mode 100644 elasticsearch/pom.xml rename {elasticsearch/elasticsearch-sink => elasticsearch5/elasticsearch5-sink}/pom.xml (86%) rename {elasticsearch/elasticsearch-sink => elasticsearch5/elasticsearch5-sink}/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java (100%) rename {elasticsearch/elasticsearch-sink => elasticsearch5/elasticsearch5-sink}/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java (100%) rename {elasticsearch/elasticsearch-sink => elasticsearch5/elasticsearch5-sink}/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/EsUtil.java (100%) rename {elasticsearch/elasticsearch-sink => elasticsearch5/elasticsearch5-sink}/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.java (100%) rename {elasticsearch/elasticsearch-sink => elasticsearch5/elasticsearch5-sink}/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java (100%) create mode 100644 elasticsearch5/pom.xml rename kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/{kafka09 => kafka}/CustomerJsonDeserialization.java (98%) rename kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/{kafka09/Kafka09Source.java => kafka/KafkaSource.java} (91%) rename kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/{kafka09/table/Kafka09SourceParser.java => kafka/table/KafkaSourceParser.java} (77%) rename kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/{kafka09/table/Kafka09SourceTableInfo.java => kafka/table/KafkaSourceTableInfo.java} (95%) create mode 100644 kafka10/kafka10-source/pom.xml create mode 100644 kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java create mode 100644 kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java create mode 100644 kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java create mode 100644 kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java create mode 100644 kafka10/kafka10-source/src/test/java/com/dtstack/flinkx/AppTest.java create mode 100644 kafka10/pom.xml create mode 100644 kafka11/kafka11-source/pom.xml create mode 100644 kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java create mode 100644 kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java create mode 100644 kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java create mode 100644 kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java create mode 100644 kafka11/kafka11-source/src/test/java/com/dtstack/flinkx/AppTest.java create mode 100644 kafka11/pom.xml diff --git a/.gitignore b/.gitignore index 5e8b08446..a40fcbf7b 100644 --- a/.gitignore +++ b/.gitignore @@ -1,8 +1,7 @@ target .idea/ /.idea/* -/target -target +target/ .class .project .classpath diff --git a/core/pom.xml b/core/pom.xml index 6dad09524..1040fcea6 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -2,7 +2,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> flink.sql - com.dtstack.flinkx + com.dtstack.flink 1.0-SNAPSHOT ../pom.xml diff --git a/elasticsearch/pom.xml b/elasticsearch/pom.xml deleted file mode 100644 index 03a1c1e3f..000000000 --- a/elasticsearch/pom.xml +++ /dev/null @@ -1,19 +0,0 @@ - - - - flink.sql - com.dtstack.flinkx - 1.0-SNAPSHOT - - 4.0.0 - - sql.elasticsearch - pom - - elasticsearch-sink - - - - \ No newline at end of file diff --git a/elasticsearch/elasticsearch-sink/pom.xml b/elasticsearch5/elasticsearch5-sink/pom.xml similarity index 86% rename from elasticsearch/elasticsearch-sink/pom.xml rename to elasticsearch5/elasticsearch5-sink/pom.xml index 600158e84..f41de1a25 100644 --- a/elasticsearch/elasticsearch-sink/pom.xml +++ b/elasticsearch5/elasticsearch5-sink/pom.xml @@ -3,30 +3,16 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> - sql.elasticsearch - com.dtstack.flinkx + sql.elasticsearch5 + com.dtstack.flink 1.0-SNAPSHOT 4.0.0 - sql.sink.elasticsearch - elasticsearch-sink + sql.sink.elasticsearch5 + elasticsearch5-sink - - junit - junit - 3.8.1 - test - - - - com.dtstack.flinkx - sql.core - 1.0-SNAPSHOT - provided - - org.apache.flink flink-connector-elasticsearch5_2.11 diff --git a/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java similarity index 100% rename from elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java rename to elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java diff --git a/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java similarity index 100% rename from elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java rename to elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java diff --git a/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/EsUtil.java b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/EsUtil.java similarity index 100% rename from elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/EsUtil.java rename to elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/EsUtil.java diff --git a/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.java b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.java similarity index 100% rename from elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.java rename to elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchSinkParser.java diff --git a/elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java similarity index 100% rename from elasticsearch/elasticsearch-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java rename to elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java diff --git a/elasticsearch5/pom.xml b/elasticsearch5/pom.xml new file mode 100644 index 000000000..45cc6c92d --- /dev/null +++ b/elasticsearch5/pom.xml @@ -0,0 +1,34 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.elasticsearch5 + pom + + elasticsearch5-sink + + + + junit + junit + 3.8.1 + test + + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + + + \ No newline at end of file diff --git a/hbase/hbase-side/pom.xml b/hbase/hbase-side/pom.xml index b73c99dea..31a8ea94b 100644 --- a/hbase/hbase-side/pom.xml +++ b/hbase/hbase-side/pom.xml @@ -4,7 +4,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> sql.hbase - com.dtstack.flinkx + com.dtstack.flink 1.0-SNAPSHOT 4.0.0 diff --git a/hbase/hbase-sink/pom.xml b/hbase/hbase-sink/pom.xml index 2d2f9e73d..44a698f60 100644 --- a/hbase/hbase-sink/pom.xml +++ b/hbase/hbase-sink/pom.xml @@ -4,7 +4,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> sql.hbase - com.dtstack.flinkx + com.dtstack.flink 1.0-SNAPSHOT 4.0.0 @@ -12,28 +12,6 @@ sql.sink.hbase hbase-sink - - - - - - - - - - - - - - - - - - - - - - diff --git a/hbase/pom.xml b/hbase/pom.xml index a34a59b11..54daa079b 100644 --- a/hbase/pom.xml +++ b/hbase/pom.xml @@ -4,7 +4,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> flink.sql - com.dtstack.flinkx + com.dtstack.flink 1.0-SNAPSHOT 4.0.0 @@ -24,7 +24,7 @@ test - com.dtstack.flinkx + com.dtstack.flink sql.core 1.0-SNAPSHOT provided diff --git a/kafka09/kafka09-source/pom.xml b/kafka09/kafka09-source/pom.xml index 371fe7cb9..992c3cdf3 100644 --- a/kafka09/kafka09-source/pom.xml +++ b/kafka09/kafka09-source/pom.xml @@ -2,7 +2,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> sql.kafka09 - com.dtstack.flinkx + com.dtstack.flink 1.0-SNAPSHOT @@ -20,19 +20,6 @@ - - junit - junit - 3.8.1 - test - - - - com.dtstack.flinkx - sql.core - 1.0-SNAPSHOT - provided - org.apache.flink @@ -89,14 +76,14 @@ - + - + diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java similarity index 98% rename from kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/CustomerJsonDeserialization.java rename to kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 7e1b4a10a..6c176badc 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -18,7 +18,7 @@ -package com.dtstack.flink.sql.source.kafka09; +package com.dtstack.flink.sql.source.kafka; import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/Kafka09Source.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java similarity index 91% rename from kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/Kafka09Source.java rename to kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 643b025c7..8e305e06d 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/Kafka09Source.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -18,10 +18,10 @@ -package com.dtstack.flink.sql.source.kafka09; +package com.dtstack.flink.sql.source.kafka; import com.dtstack.flink.sql.source.IStreamSourceGener; -import com.dtstack.flink.sql.source.kafka09.table.Kafka09SourceTableInfo; +import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -41,7 +41,7 @@ * @author xuchao */ -public class Kafka09Source implements IStreamSourceGener
{ +public class KafkaSource implements IStreamSourceGener
{ /** * Get kafka data source, you need to provide the data field names, data types @@ -53,7 +53,7 @@ public class Kafka09Source implements IStreamSourceGener
{ @Override public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { - Kafka09SourceTableInfo kafka09SourceTableInfo = (Kafka09SourceTableInfo) sourceTableInfo; + KafkaSourceTableInfo kafka09SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; String topicName = kafka09SourceTableInfo.getTopic(); Properties props = new Properties(); diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/table/Kafka09SourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java similarity index 77% rename from kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/table/Kafka09SourceParser.java rename to kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 51fccd660..898083826 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/table/Kafka09SourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -18,7 +18,7 @@ -package com.dtstack.flink.sql.source.kafka09.table; +package com.dtstack.flink.sql.source.kafka.table; import com.dtstack.flink.sql.table.AbsSourceParser; import com.dtstack.flink.sql.table.TableInfo; @@ -33,19 +33,19 @@ * @author xuchao */ -public class Kafka09SourceParser extends AbsSourceParser { +public class KafkaSourceParser extends AbsSourceParser { @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - Kafka09SourceTableInfo kafka09SourceTableInfo = new Kafka09SourceTableInfo(); + KafkaSourceTableInfo kafka09SourceTableInfo = new KafkaSourceTableInfo(); kafka09SourceTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, kafka09SourceTableInfo); - kafka09SourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(Kafka09SourceTableInfo.PARALLELISM_KEY.toLowerCase()))); - kafka09SourceTableInfo.setBootstrapServers(MathUtil.getString(props.get(Kafka09SourceTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase()))); - kafka09SourceTableInfo.setGroupId(MathUtil.getString(props.get(Kafka09SourceTableInfo.GROUPID_KEY.toLowerCase()))); - kafka09SourceTableInfo.setTopic(MathUtil.getString(props.get(Kafka09SourceTableInfo.TOPIC_KEY.toLowerCase()))); + kafka09SourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSourceTableInfo.PARALLELISM_KEY.toLowerCase()))); + kafka09SourceTableInfo.setBootstrapServers(MathUtil.getString(props.get(KafkaSourceTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase()))); + kafka09SourceTableInfo.setGroupId(MathUtil.getString(props.get(KafkaSourceTableInfo.GROUPID_KEY.toLowerCase()))); + kafka09SourceTableInfo.setTopic(MathUtil.getString(props.get(KafkaSourceTableInfo.TOPIC_KEY.toLowerCase()))); return kafka09SourceTableInfo; } } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/table/Kafka09SourceTableInfo.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java similarity index 95% rename from kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/table/Kafka09SourceTableInfo.java rename to kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 94691adca..29239ad75 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka09/table/Kafka09SourceTableInfo.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -18,7 +18,7 @@ -package com.dtstack.flink.sql.source.kafka09.table; +package com.dtstack.flink.sql.source.kafka.table; import com.dtstack.flink.sql.table.SourceTableInfo; import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; @@ -30,7 +30,7 @@ * @author xuchao */ -public class Kafka09SourceTableInfo extends SourceTableInfo { +public class KafkaSourceTableInfo extends SourceTableInfo { //version private static final String CURR_TYPE = "kafka09"; @@ -52,7 +52,7 @@ public class Kafka09SourceTableInfo extends SourceTableInfo { private String offset; - public Kafka09SourceTableInfo(){ + public KafkaSourceTableInfo(){ super.setType(CURR_TYPE); } diff --git a/kafka09/pom.xml b/kafka09/pom.xml index c9a2134e8..457a8d387 100644 --- a/kafka09/pom.xml +++ b/kafka09/pom.xml @@ -4,7 +4,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> flink.sql - com.dtstack.flinkx + com.dtstack.flink 1.0-SNAPSHOT @@ -17,5 +17,21 @@ kafka09-source + + + junit + junit + 3.8.1 + test + + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + \ No newline at end of file diff --git a/kafka10/kafka10-source/pom.xml b/kafka10/kafka10-source/pom.xml new file mode 100644 index 000000000..b00004633 --- /dev/null +++ b/kafka10/kafka10-source/pom.xml @@ -0,0 +1,93 @@ + + + sql.kafka10 + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.source.kafka10 + jar + + kafka10-source + http://maven.apache.org + + + UTF-8 + 1.4.0 + + + + + + org.apache.flink + flink-connector-kafka-0.10_2.11 + ${flink.version} + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + org.slf4j + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java new file mode 100644 index 000000000..9225464ab --- /dev/null +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -0,0 +1,114 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka; + + +import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Iterator; + +/** + * json string parsing custom + * Date: 2018/09/18 + * Company: www.dtstack.com + * @author sishu.yss + */ + +public class CustomerJsonDeserialization extends AbstractDeserializationSchema { + + private static final Logger logger = LoggerFactory.getLogger(CustomerJsonDeserialization.class); + + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Type information describing the result type. */ + private final TypeInformation typeInfo; + + /** Field names to parse. Indices match fieldTypes indices. */ + private final String[] fieldNames; + + /** Types to parse fields as. Indices match fieldNames indices. */ + private final TypeInformation[] fieldTypes; + + /** Flag indicating whether to fail on a missing field. */ + private boolean failOnMissingField; + + public CustomerJsonDeserialization(TypeInformation typeInfo){ + this.typeInfo = typeInfo; + + this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); + + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + } + + @Override + public Row deserialize(byte[] message) throws IOException { + try { + JsonNode root = objectMapper.readTree(message); + Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + JsonNode node = getIgnoreCase(root, fieldNames[i]); + + if (node == null) { + if (failOnMissingField) { + throw new IllegalStateException("Failed to find field with name '" + + fieldNames[i] + "'."); + } else { + row.setField(i, null); + } + } else { + // Read the value as specified type + Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + row.setField(i, value); + } + } + + return row; + } catch (Throwable t) { + throw new IOException("Failed to deserialize JSON object.", t); + } + } + + public void setFailOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; + } + + public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { + + Iterator iter = jsonNode.fieldNames(); + while (iter.hasNext()) { + String key1 = iter.next(); + if (key1.equalsIgnoreCase(key)) { + return jsonNode.get(key1); + } + } + + return null; + + } +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java new file mode 100644 index 000000000..a3ea8693b --- /dev/null +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -0,0 +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 com.dtstack.flink.sql.source.kafka; + +import com.dtstack.flink.sql.source.IStreamSourceGener; +import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; +import com.dtstack.flink.sql.table.SourceTableInfo; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.types.Row; + +import java.util.Properties; + +/** + * If eventtime field is specified, the default time field rowtime + * Date: 2018/09/18 + * Company: www.dtstack.com + * @author sishu.yss + */ + +public class KafkaSource implements IStreamSourceGener
{ + + /** + * Get kafka data source, you need to provide the data field names, data types + * If you do not specify auto.offset.reset, the default use groupoffset + * @param sourceTableInfo + * @return + */ + @SuppressWarnings("rawtypes") + @Override + public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { + + KafkaSourceTableInfo kafka010SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; + String topicName = kafka010SourceTableInfo.getTopic(); + + Properties props = new Properties(); + props.setProperty("bootstrap.servers", kafka010SourceTableInfo.getBootstrapServers()); + props.setProperty("auto.offset.reset", kafka010SourceTableInfo.getOffsetReset()); + //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) + + TypeInformation[] types = new TypeInformation[kafka010SourceTableInfo.getFields().length]; + for(int i = 0; i< kafka010SourceTableInfo.getFieldClasses().length; i++){ + types[i] = TypeInformation.of(kafka010SourceTableInfo.getFieldClasses()[i]); + } + + TypeInformation typeInformation = new RowTypeInfo(types, kafka010SourceTableInfo.getFields()); + FlinkKafkaConsumer010 kafkaSrc = new FlinkKafkaConsumer010(topicName, + new CustomerJsonDeserialization(typeInformation), props); + + //earliest,latest + if("earliest".equalsIgnoreCase(kafka010SourceTableInfo.getOffsetReset())){ + kafkaSrc.setStartFromEarliest(); + }else{ + kafkaSrc.setStartFromLatest(); + } + + String fields = StringUtils.join(kafka010SourceTableInfo.getFields(), ","); + return tableEnv.fromDataStream(env.addSource(kafkaSrc, typeInformation), fields); + } +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java new file mode 100644 index 000000000..5cba4e7f2 --- /dev/null +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -0,0 +1,51 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.table; + +import com.dtstack.flink.sql.table.AbsSourceParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/09/18 + * Company: www.dtstack.com + * @author sishu.yss + */ + +public class KafkaSourceParser extends AbsSourceParser { + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + + KafkaSourceTableInfo kafka10SourceTableInfo = new KafkaSourceTableInfo(); + kafka10SourceTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka10SourceTableInfo); + + kafka10SourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSourceTableInfo.PARALLELISM_KEY.toLowerCase()))); + kafka10SourceTableInfo.setBootstrapServers(MathUtil.getString(props.get(KafkaSourceTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase()))); + kafka10SourceTableInfo.setGroupId(MathUtil.getString(props.get(KafkaSourceTableInfo.GROUPID_KEY.toLowerCase()))); + kafka10SourceTableInfo.setTopic(MathUtil.getString(props.get(KafkaSourceTableInfo.TOPIC_KEY.toLowerCase()))); + return kafka10SourceTableInfo; + } +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java new file mode 100644 index 000000000..fff9cc8b9 --- /dev/null +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -0,0 +1,120 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.table; + +import com.dtstack.flink.sql.table.SourceTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * Reason: + * Date: 2018/09/18 + * Company: www.dtstack.com + * @author sishu.yss + */ + +public class KafkaSourceTableInfo extends SourceTableInfo { + + //version + private static final String CURR_TYPE = "kafka10"; + + public static final String BOOTSTRAPSERVERS_KEY = "bootstrapServers"; + + public static final String TOPIC_KEY = "topic"; + + public static final String GROUPID_KEY = "groupId"; + + private String bootstrapServers; + + private String topic; + + private String groupId; + + //latest, earliest + private String offsetReset = "latest"; + + private String offset; + + public KafkaSourceTableInfo(){ + super.setType(CURR_TYPE); + } + + + public String getBootstrapServers() { + return bootstrapServers; + } + + public void setBootstrapServers(String bootstrapServers) { + this.bootstrapServers = bootstrapServers; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public String getGroupId() { + return groupId; + } + + public void setGroupId(String groupId) { + this.groupId = groupId; + } + + public String getOffsetReset() { + return offsetReset; + } + + public void setOffsetReset(String offsetReset) { + if(offsetReset == null){ + return; + } + + this.offsetReset = offsetReset; + } + + public String getOffset() { + return offset; + } + + public void setOffset(String offset) { + this.offset = offset; + } + + @Override + public boolean check() { + Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(topic, "kafka of topic is required"); + Preconditions.checkNotNull(groupId, "kafka of groupId is required"); + Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") + || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + + return false; + } + + @Override + public String getType() { +// return super.getType() + SOURCE_SUFFIX; + return super.getType(); + } +} diff --git a/kafka10/kafka10-source/src/test/java/com/dtstack/flinkx/AppTest.java b/kafka10/kafka10-source/src/test/java/com/dtstack/flinkx/AppTest.java new file mode 100644 index 000000000..33a0233ac --- /dev/null +++ b/kafka10/kafka10-source/src/test/java/com/dtstack/flinkx/AppTest.java @@ -0,0 +1,58 @@ +/* + * 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 com.dtstack.flinkx; + +import junit.framework.Test; +import junit.framework.TestCase; +import junit.framework.TestSuite; + +/** + * Unit test for simple App. + */ +public class AppTest + extends TestCase +{ + /** + * Create the test case + * + * @param testName name of the test case + */ + public AppTest( String testName ) + { + super( testName ); + } + + /** + * @return the suite of tests being tested + */ + public static Test suite() + { + return new TestSuite( AppTest.class ); + } + + /** + * Rigourous Test :-) + */ + public void testApp() + { + assertTrue( true ); + } +} diff --git a/kafka10/pom.xml b/kafka10/pom.xml new file mode 100644 index 000000000..9c8c78bba --- /dev/null +++ b/kafka10/pom.xml @@ -0,0 +1,36 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.kafka10 + pom + + + kafka10-source + + + + + junit + junit + 3.8.1 + test + + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + + + \ No newline at end of file diff --git a/kafka11/kafka11-source/pom.xml b/kafka11/kafka11-source/pom.xml new file mode 100644 index 000000000..a1aa79668 --- /dev/null +++ b/kafka11/kafka11-source/pom.xml @@ -0,0 +1,93 @@ + + + sql.kafka11 + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.source.kafka11 + jar + + kafka11-source + http://maven.apache.org + + + UTF-8 + 1.4.0 + + + + + + org.apache.flink + flink-connector-kafka-0.11_2.11 + ${flink.version} + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + org.slf4j + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java new file mode 100644 index 000000000..9225464ab --- /dev/null +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -0,0 +1,114 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka; + + +import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Iterator; + +/** + * json string parsing custom + * Date: 2018/09/18 + * Company: www.dtstack.com + * @author sishu.yss + */ + +public class CustomerJsonDeserialization extends AbstractDeserializationSchema { + + private static final Logger logger = LoggerFactory.getLogger(CustomerJsonDeserialization.class); + + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Type information describing the result type. */ + private final TypeInformation typeInfo; + + /** Field names to parse. Indices match fieldTypes indices. */ + private final String[] fieldNames; + + /** Types to parse fields as. Indices match fieldNames indices. */ + private final TypeInformation[] fieldTypes; + + /** Flag indicating whether to fail on a missing field. */ + private boolean failOnMissingField; + + public CustomerJsonDeserialization(TypeInformation typeInfo){ + this.typeInfo = typeInfo; + + this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); + + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + } + + @Override + public Row deserialize(byte[] message) throws IOException { + try { + JsonNode root = objectMapper.readTree(message); + Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + JsonNode node = getIgnoreCase(root, fieldNames[i]); + + if (node == null) { + if (failOnMissingField) { + throw new IllegalStateException("Failed to find field with name '" + + fieldNames[i] + "'."); + } else { + row.setField(i, null); + } + } else { + // Read the value as specified type + Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + row.setField(i, value); + } + } + + return row; + } catch (Throwable t) { + throw new IOException("Failed to deserialize JSON object.", t); + } + } + + public void setFailOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; + } + + public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { + + Iterator iter = jsonNode.fieldNames(); + while (iter.hasNext()) { + String key1 = iter.next(); + if (key1.equalsIgnoreCase(key)) { + return jsonNode.get(key1); + } + } + + return null; + + } +} diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java new file mode 100644 index 000000000..c813c21eb --- /dev/null +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -0,0 +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 com.dtstack.flink.sql.source.kafka; + +import com.dtstack.flink.sql.source.IStreamSourceGener; +import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; +import com.dtstack.flink.sql.table.SourceTableInfo; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.types.Row; + +import java.util.Properties; + +/** + * If eventtime field is specified, the default time field rowtime + * Date: 2018/09/18 + * Company: www.dtstack.com + * @author sishu.yss + */ + +public class KafkaSource implements IStreamSourceGener
{ + + /** + * Get kafka data source, you need to provide the data field names, data types + * If you do not specify auto.offset.reset, the default use groupoffset + * @param sourceTableInfo + * @return + */ + @SuppressWarnings("rawtypes") + @Override + public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { + + KafkaSourceTableInfo kafka011SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; + String topicName = kafka011SourceTableInfo.getTopic(); + + Properties props = new Properties(); + props.setProperty("bootstrap.servers", kafka011SourceTableInfo.getBootstrapServers()); + props.setProperty("auto.offset.reset", kafka011SourceTableInfo.getOffsetReset()); + //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) + + TypeInformation[] types = new TypeInformation[kafka011SourceTableInfo.getFields().length]; + for(int i = 0; i< kafka011SourceTableInfo.getFieldClasses().length; i++){ + types[i] = TypeInformation.of(kafka011SourceTableInfo.getFieldClasses()[i]); + } + + TypeInformation typeInformation = new RowTypeInfo(types, kafka011SourceTableInfo.getFields()); + FlinkKafkaConsumer011 kafkaSrc = new FlinkKafkaConsumer011(topicName, + new CustomerJsonDeserialization(typeInformation), props); + + //earliest,latest + if("earliest".equalsIgnoreCase(kafka011SourceTableInfo.getOffsetReset())){ + kafkaSrc.setStartFromEarliest(); + }else{ + kafkaSrc.setStartFromLatest(); + } + + String fields = StringUtils.join(kafka011SourceTableInfo.getFields(), ","); + return tableEnv.fromDataStream(env.addSource(kafkaSrc, typeInformation), fields); + } +} diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java new file mode 100644 index 000000000..fff3cb9c9 --- /dev/null +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -0,0 +1,51 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.table; + +import com.dtstack.flink.sql.table.AbsSourceParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/09/18 + * Company: www.dtstack.com + * @author sishu.yss + */ + +public class KafkaSourceParser extends AbsSourceParser { + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + + KafkaSourceTableInfo kafka11SourceTableInfo = new KafkaSourceTableInfo(); + kafka11SourceTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka11SourceTableInfo); + + kafka11SourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSourceTableInfo.PARALLELISM_KEY.toLowerCase()))); + kafka11SourceTableInfo.setBootstrapServers(MathUtil.getString(props.get(KafkaSourceTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase()))); + kafka11SourceTableInfo.setGroupId(MathUtil.getString(props.get(KafkaSourceTableInfo.GROUPID_KEY.toLowerCase()))); + kafka11SourceTableInfo.setTopic(MathUtil.getString(props.get(KafkaSourceTableInfo.TOPIC_KEY.toLowerCase()))); + return kafka11SourceTableInfo; + } +} diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java new file mode 100644 index 000000000..4e7397970 --- /dev/null +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -0,0 +1,120 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.table; + +import com.dtstack.flink.sql.table.SourceTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * Reason: + * Date: 2018/09/18 + * Company: www.dtstack.com + * @author sishu.yss + */ + +public class KafkaSourceTableInfo extends SourceTableInfo { + + //version + private static final String CURR_TYPE = "kafka11"; + + public static final String BOOTSTRAPSERVERS_KEY = "bootstrapServers"; + + public static final String TOPIC_KEY = "topic"; + + public static final String GROUPID_KEY = "groupId"; + + private String bootstrapServers; + + private String topic; + + private String groupId; + + //latest, earliest + private String offsetReset = "latest"; + + private String offset; + + public KafkaSourceTableInfo(){ + super.setType(CURR_TYPE); + } + + + public String getBootstrapServers() { + return bootstrapServers; + } + + public void setBootstrapServers(String bootstrapServers) { + this.bootstrapServers = bootstrapServers; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public String getGroupId() { + return groupId; + } + + public void setGroupId(String groupId) { + this.groupId = groupId; + } + + public String getOffsetReset() { + return offsetReset; + } + + public void setOffsetReset(String offsetReset) { + if(offsetReset == null){ + return; + } + + this.offsetReset = offsetReset; + } + + public String getOffset() { + return offset; + } + + public void setOffset(String offset) { + this.offset = offset; + } + + @Override + public boolean check() { + Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(topic, "kafka of topic is required"); + Preconditions.checkNotNull(groupId, "kafka of groupId is required"); + Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") + || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + + return false; + } + + @Override + public String getType() { +// return super.getType() + SOURCE_SUFFIX; + return super.getType(); + } +} diff --git a/kafka11/kafka11-source/src/test/java/com/dtstack/flinkx/AppTest.java b/kafka11/kafka11-source/src/test/java/com/dtstack/flinkx/AppTest.java new file mode 100644 index 000000000..33a0233ac --- /dev/null +++ b/kafka11/kafka11-source/src/test/java/com/dtstack/flinkx/AppTest.java @@ -0,0 +1,58 @@ +/* + * 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 com.dtstack.flinkx; + +import junit.framework.Test; +import junit.framework.TestCase; +import junit.framework.TestSuite; + +/** + * Unit test for simple App. + */ +public class AppTest + extends TestCase +{ + /** + * Create the test case + * + * @param testName name of the test case + */ + public AppTest( String testName ) + { + super( testName ); + } + + /** + * @return the suite of tests being tested + */ + public static Test suite() + { + return new TestSuite( AppTest.class ); + } + + /** + * Rigourous Test :-) + */ + public void testApp() + { + assertTrue( true ); + } +} diff --git a/kafka11/pom.xml b/kafka11/pom.xml new file mode 100644 index 000000000..841e20be9 --- /dev/null +++ b/kafka11/pom.xml @@ -0,0 +1,36 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.kafka11 + pom + + + kafka11-source + + + + + junit + junit + 3.8.1 + test + + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + + + \ No newline at end of file diff --git a/launcher/pom.xml b/launcher/pom.xml index 78b2f6e6a..5e23b03ea 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -4,7 +4,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> flink.sql - com.dtstack.flinkx + com.dtstack.flink 1.0-SNAPSHOT 4.0.0 @@ -13,7 +13,7 @@ - com.dtstack.flinkx + com.dtstack.flink sql.core 1.0-SNAPSHOT diff --git a/mysql/mysql-side/pom.xml b/mysql/mysql-side/pom.xml index 031a37072..4b7ecde57 100644 --- a/mysql/mysql-side/pom.xml +++ b/mysql/mysql-side/pom.xml @@ -4,7 +4,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> sql.mysql - com.dtstack.flinkx + com.dtstack.flink 1.0-SNAPSHOT ../pom.xml diff --git a/mysql/mysql-sink/pom.xml b/mysql/mysql-sink/pom.xml index 9a56f2712..807e27fc0 100644 --- a/mysql/mysql-sink/pom.xml +++ b/mysql/mysql-sink/pom.xml @@ -2,7 +2,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> sql.mysql - com.dtstack.flinkx + com.dtstack.flink 1.0-SNAPSHOT ../pom.xml diff --git a/mysql/pom.xml b/mysql/pom.xml index aa2d20ab4..747646f74 100644 --- a/mysql/pom.xml +++ b/mysql/pom.xml @@ -4,7 +4,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> flink.sql - com.dtstack.flinkx + com.dtstack.flink 1.0-SNAPSHOT 4.0.0 @@ -24,7 +24,7 @@ test - com.dtstack.flinkx + com.dtstack.flink sql.core 1.0-SNAPSHOT provided diff --git a/pom.xml b/pom.xml index 84f93b0e9..2f49b99dd 100644 --- a/pom.xml +++ b/pom.xml @@ -2,12 +2,14 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - com.dtstack.flinkx + com.dtstack.flink flink.sql 1.0-SNAPSHOT core kafka09 + kafka10 + kafka11 mysql hbase elasticsearch From 5261849c417471d0ceb903e9200d22ab8e1d81b3 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 18 Sep 2018 16:14:06 +0800 Subject: [PATCH 024/470] =?UTF-8?q?=E8=B0=83=E6=95=B4side=20=E7=BB=93?= =?UTF-8?q?=E6=9E=84?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flink/sql/enums/ECacheType.java | 2 +- .../flink/sql/side/AllCacheOperator.java | 45 ++++ .../com/dtstack/flink/sql/side/AllReqRow.java | 34 +++ .../dtstack/flink/sql/side/AsyncReqRow.java | 125 +-------- .../flink/sql/side/LRUCacheOperator.java | 60 +++++ .../dtstack/flink/sql/side/SideReqRow.java | 244 ++++++++++++++++++ .../dtstack/flink/sql/side/SideSqlExec.java | 22 +- .../dtstack/flink/sql/side/SideTableInfo.java | 2 +- hbase/hbase-side/doc/readme.txt | 57 ---- hbase/hbase-side/hbase-all-side/pom.xml | 76 ++++++ hbase/hbase-side/hbase-async-side/pom.xml | 93 +++++++ .../sql/side/hbase/HbaseAsyncReqRow.java | 78 ++---- .../flink/sql/side/hbase/HbaseSideReqRow.java | 83 ++++++ .../rowkeydealer/AbsRowKeyModeDealer.java | 0 .../PreRowKeyModeDealerDealer.java | 2 +- .../rowkeydealer/RowKeyEqualModeDealer.java | 2 +- hbase/hbase-side/hbase-side-core/pom.xml | 15 ++ .../sql/side/hbase/Md5ReplaceOperator.java | 0 .../flink/sql/side/hbase/ReplaceInfo.java | 0 .../flink/sql/side/hbase/ReplaceOperator.java | 0 .../flink/sql/side/hbase/RowKeyBuilder.java | 0 .../sql/side/hbase/enums/EReplaceOpType.java | 0 .../sql/side/hbase/enums/EReplaceType.java | 0 .../sql/side/hbase/table/HbaseSideParser.java | 0 .../side/hbase/table/HbaseSideTableInfo.java | 0 .../sql/side/hbase/utils/HbaseUtils.java | 0 hbase/hbase-side/pom.xml | 75 +----- mysql/mysql-side/mysql-all-side/pom.xml | 87 +++++++ mysql/mysql-side/mysql-async-side/pom.xml | 101 ++++++++ .../sql/side/mysql/MysqlAsyncReqRow.java | 129 +-------- .../flink/sql/side/mysql/MysqlSideReqRow.java | 145 +++++++++++ mysql/mysql-side/mysql-side-core/pom.xml | 15 ++ .../sql/side/mysql/table/MysqlSideParser.java | 0 .../side/mysql/table/MysqlSideTableInfo.java | 0 mysql/mysql-side/pom.xml | 84 +----- mysql/pom.xml | 1 + 36 files changed, 1066 insertions(+), 511 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/AllCacheOperator.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/LRUCacheOperator.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/SideReqRow.java delete mode 100644 hbase/hbase-side/doc/readme.txt create mode 100644 hbase/hbase-side/hbase-all-side/pom.xml create mode 100644 hbase/hbase-side/hbase-async-side/pom.xml rename hbase/hbase-side/{ => hbase-async-side}/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java (72%) create mode 100644 hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseSideReqRow.java rename hbase/hbase-side/{ => hbase-async-side}/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbsRowKeyModeDealer.java (100%) rename hbase/hbase-side/{ => hbase-async-side}/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java (100%) rename hbase/hbase-side/{ => hbase-async-side}/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java (100%) create mode 100644 hbase/hbase-side/hbase-side-core/pom.xml rename hbase/hbase-side/{ => hbase-side-core}/src/main/java/com/dtstack/flink/sql/side/hbase/Md5ReplaceOperator.java (100%) rename hbase/hbase-side/{ => hbase-side-core}/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceInfo.java (100%) rename hbase/hbase-side/{ => hbase-side-core}/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceOperator.java (100%) rename hbase/hbase-side/{ => hbase-side-core}/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java (100%) rename hbase/hbase-side/{ => hbase-side-core}/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceOpType.java (100%) rename hbase/hbase-side/{ => hbase-side-core}/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceType.java (100%) rename hbase/hbase-side/{ => hbase-side-core}/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java (100%) rename hbase/hbase-side/{ => hbase-side-core}/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideTableInfo.java (100%) rename hbase/hbase-side/{ => hbase-side-core}/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseUtils.java (100%) create mode 100644 mysql/mysql-side/mysql-all-side/pom.xml create mode 100644 mysql/mysql-side/mysql-async-side/pom.xml rename mysql/mysql-side/{ => mysql-async-side}/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java (62%) create mode 100644 mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlSideReqRow.java create mode 100644 mysql/mysql-side/mysql-side-core/pom.xml rename mysql/mysql-side/{ => mysql-side-core}/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java (100%) rename mysql/mysql-side/{ => mysql-side-core}/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java (100%) diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/ECacheType.java b/core/src/main/java/com/dtstack/flink/sql/enums/ECacheType.java index 4cd6aaf4f..582148c2c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/enums/ECacheType.java +++ b/core/src/main/java/com/dtstack/flink/sql/enums/ECacheType.java @@ -27,7 +27,7 @@ * @author xuchao */ public enum ECacheType { - NONE, LRU; + NONE, LRU, ALL; public static boolean isValid(String type){ for(ECacheType tmpType : ECacheType.values()){ diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AllCacheOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/AllCacheOperator.java new file mode 100644 index 000000000..6d445f6e2 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/AllCacheOperator.java @@ -0,0 +1,45 @@ +/* + * 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 com.dtstack.flink.sql.side; + +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; + +import java.util.List; + +/** + * get plugin which implement from RichFlatMapFunction + * Date: 2018/9/18 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class AllCacheOperator { + + private static void loadFlatMap(){ + + } + + public static DataStream getSideJoinDataStream(DataStream inputStream, String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, + List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { + return null; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java new file mode 100644 index 000000000..fbe83e432 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java @@ -0,0 +1,34 @@ +/* + * 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 com.dtstack.flink.sql.side; + +import org.apache.calcite.interpreter.Row; +import org.apache.flink.api.common.functions.RichFlatMapFunction; + +/** + * Reason: + * Date: 2018/9/18 + * Company: www.dtstack.com + * @author xuchao + */ + +public abstract class AllReqRow extends RichFlatMapFunction{ +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java index a335c9a8b..bffdafcc8 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java @@ -53,50 +53,24 @@ public abstract class AsyncReqRow extends RichAsyncFunction { - private static final Logger LOG = LoggerFactory.getLogger(AsyncReqRow.class); - private static final long serialVersionUID = 2098635244857937717L; - protected RowTypeInfo rowTypeInfo; - - protected List outFieldInfoList; - - protected List equalFieldList = Lists.newArrayList(); - - protected List equalValIndex = Lists.newArrayList(); - - protected String sqlCondition = ""; - - protected String sideSelectFields = ""; - - protected JoinType joinType; + protected SideReqRow sideReqRow; - //key:Returns the value of the position, returns the index values ​​in the input data - protected Map inFieldIndex = Maps.newHashMap(); - - protected Map sideFieldIndex = Maps.newHashMap(); - - protected SideTableInfo sideTableInfo; - - protected AbsSideCache sideCache; - - public AsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, - SideTableInfo sideTableInfo){ - this.rowTypeInfo = rowTypeInfo; - this.outFieldInfoList = outFieldInfoList; - this.joinType = joinInfo.getJoinType(); - this.sideTableInfo = sideTableInfo; - parseSelectFields(joinInfo); - buildEqualInfo(joinInfo, sideTableInfo); + public AsyncReqRow(SideReqRow sideReqRow){ + this.sideReqRow = sideReqRow; } private void initCache(){ + SideTableInfo sideTableInfo = sideReqRow.getSideTableInfo(); if(sideTableInfo.getCacheType() == null || ECacheType.NONE.name().equalsIgnoreCase(sideTableInfo.getCacheType())){ return; } + AbsSideCache sideCache; if(ECacheType.LRU.name().equalsIgnoreCase(sideTableInfo.getCacheType())){ sideCache = new LRUSideCache(sideTableInfo); + sideReqRow.setSideCache(sideCache); }else{ throw new RuntimeException("not support side cache with type:" + sideTableInfo.getCacheType()); } @@ -105,101 +79,22 @@ private void initCache(){ } protected CacheObj getFromCache(String key){ - return sideCache.getFromCache(key); + return sideReqRow.getSideCache().getFromCache(key); } protected void putCache(String key, CacheObj value){ - sideCache.putCache(key, value); + sideReqRow.getSideCache().putCache(key, value); } protected boolean openCache(){ - return sideCache != null; + return sideReqRow.getSideCache() != null; } - public void parseSelectFields(JoinInfo joinInfo){ - String sideTableName = joinInfo.getSideTableName(); - String nonSideTableName = joinInfo.getNonSideTable(); - List fields = Lists.newArrayList(); - - int sideIndex = 0; - for( int i=0; i resultFuture){ - if(joinType == JoinType.LEFT){ + if(sideReqRow.getJoinType() == JoinType.LEFT){ //Reserved left table data Row row = fillData(input, null); resultFuture.complete(Collections.singleton(row)); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/LRUCacheOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/LRUCacheOperator.java new file mode 100644 index 000000000..9c995885d --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/LRUCacheOperator.java @@ -0,0 +1,60 @@ +/* + * 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 com.dtstack.flink.sql.side; + +import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.AsyncDataStream; +import org.apache.flink.streaming.api.datastream.DataStream; + +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** + * fill data with lru cache + * get data from External data source with async operator + * Date: 2018/9/18 + * Company: www.dtstack.com + * @author xuchao + */ + +public class LRUCacheOperator { + + private static AsyncReqRow loadAsyncReq(String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, + JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + String pathOfType = sideType + "side"; + String pluginJarPath = PluginUtil.getJarFileDirPath(pathOfType, sqlRootDir); + DtClassLoader dtClassLoader = (DtClassLoader) classLoader; + PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); + String className = PluginUtil.getSqlSideClassName(sideType, "side"); + return dtClassLoader.loadClass(className).asSubclass(AsyncReqRow.class) + .getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, SideTableInfo.class).newInstance(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + public static DataStream getSideJoinDataStream(DataStream inputStream, String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, + List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { + AsyncReqRow asyncDbReq = loadAsyncReq(sideType, sqlRootDir, rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + //TODO How much should be set for the degree of parallelism? Timeout? capacity settings? + return AsyncDataStream.orderedWait(inputStream, asyncDbReq, 10000, TimeUnit.MILLISECONDS, 10) + .setParallelism(sideTableInfo.getParallelism()); + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/SideReqRow.java new file mode 100644 index 000000000..f9b1be60e --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideReqRow.java @@ -0,0 +1,244 @@ +/* + * 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 com.dtstack.flink.sql.side; + +import com.dtstack.flink.sql.side.cache.AbsSideCache; +import org.apache.calcite.sql.JoinType; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2018/9/18 + * Company: www.dtstack.com + * @author xuchao + */ + +public abstract class SideReqRow { + + protected RowTypeInfo rowTypeInfo; + + protected List outFieldInfoList; + + protected List equalFieldList = Lists.newArrayList(); + + protected List equalValIndex = Lists.newArrayList(); + + protected String sqlCondition = ""; + + protected String sideSelectFields = ""; + + protected JoinType joinType; + + //key:Returns the value of the position, returns the index values ​​in the input data + protected Map inFieldIndex = Maps.newHashMap(); + + protected Map sideFieldIndex = Maps.newHashMap(); + + protected SideTableInfo sideTableInfo; + + protected AbsSideCache sideCache; + + public SideReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, + SideTableInfo sideTableInfo){ + this.rowTypeInfo = rowTypeInfo; + this.outFieldInfoList = outFieldInfoList; + this.joinType = joinInfo.getJoinType(); + this.sideTableInfo = sideTableInfo; + parseSelectFields(joinInfo); + buildEqualInfo(joinInfo, sideTableInfo); + } + + public void parseSelectFields(JoinInfo joinInfo){ + String sideTableName = joinInfo.getSideTableName(); + String nonSideTableName = joinInfo.getNonSideTable(); + List fields = Lists.newArrayList(); + + int sideIndex = 0; + for( int i=0; i getOutFieldInfoList() { + return outFieldInfoList; + } + + public void setOutFieldInfoList(List outFieldInfoList) { + this.outFieldInfoList = outFieldInfoList; + } + + public List getEqualFieldList() { + return equalFieldList; + } + + public void setEqualFieldList(List equalFieldList) { + this.equalFieldList = equalFieldList; + } + + public List getEqualValIndex() { + return equalValIndex; + } + + public void setEqualValIndex(List equalValIndex) { + this.equalValIndex = equalValIndex; + } + + public String getSqlCondition() { + return sqlCondition; + } + + public void setSqlCondition(String sqlCondition) { + this.sqlCondition = sqlCondition; + } + + public String getSideSelectFields() { + return sideSelectFields; + } + + public void setSideSelectFields(String sideSelectFields) { + this.sideSelectFields = sideSelectFields; + } + + public JoinType getJoinType() { + return joinType; + } + + public void setJoinType(JoinType joinType) { + this.joinType = joinType; + } + + public Map getInFieldIndex() { + return inFieldIndex; + } + + public void setInFieldIndex(Map inFieldIndex) { + this.inFieldIndex = inFieldIndex; + } + + public Map getSideFieldIndex() { + return sideFieldIndex; + } + + public void setSideFieldIndex(Map sideFieldIndex) { + this.sideFieldIndex = sideFieldIndex; + } + + public SideTableInfo getSideTableInfo() { + return sideTableInfo; + } + + public void setSideTableInfo(SideTableInfo sideTableInfo) { + this.sideTableInfo = sideTableInfo; + } + + public AbsSideCache getSideCache() { + return sideCache; + } + + public void setSideCache(AbsSideCache sideCache) { + this.sideCache = sideCache; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 57f05c0d1..7e813b3c0 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.side; import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlDataTypeSpec; @@ -158,10 +159,12 @@ public void exec(String sql, Map sideTableMap, StreamTabl adaptStream = adaptStream.keyBy(leftJoinColArr); } - AsyncReqRow asyncDbReq = loadAsyncReq(sideTableInfo.getType(), localSqlPluginPath, typeInfo, joinInfo, sideJoinFieldInfo, sideTableInfo); - //TODO How much should be set for the degree of parallelism? Timeout? capacity settings? - DataStream dsOut = AsyncDataStream.orderedWait(adaptStream, asyncDbReq, 10000, TimeUnit.MILLISECONDS, 10) - .setParallelism(sideTableInfo.getParallelism()); + DataStream dsOut = null; + if(ECacheType.ALL.name().equalsIgnoreCase(sideTableInfo.getCacheType())){ + + }else{ + dsOut = LRUCacheOperator.getSideJoinDataStream(adaptStream, sideTableInfo.getType(), localSqlPluginPath, typeInfo, joinInfo, sideJoinFieldInfo, sideTableInfo); + } HashBasedTable mappingTable = HashBasedTable.create(); RowTypeInfo sideOutTypeInfo = buildOutRowTypeInfo(sideJoinFieldInfo, mappingTable); @@ -359,17 +362,6 @@ public SqlNode filterNodeWithTargetName(SqlNode sqlNode, String targetTableName) return null; } - public AsyncReqRow loadAsyncReq(String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, - JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { - ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - String pathOfType = sideType + "side"; - String pluginJarPath = PluginUtil.getJarFileDirPath(pathOfType, sqlRootDir); - DtClassLoader dtClassLoader = (DtClassLoader) classLoader; - PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); - String className = PluginUtil.getSqlSideClassName(sideType, "side"); - return dtClassLoader.loadClass(className).asSubclass(AsyncReqRow.class) - .getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, SideTableInfo.class).newInstance(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); - } public void setLocalSqlPluginPath(String localSqlPluginPath){ this.localSqlPluginPath = localSqlPluginPath; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java index 58dc79de2..0abd55a92 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java @@ -45,7 +45,7 @@ public abstract class SideTableInfo extends TableInfo implements Serializable { public static final String PARTITIONED_JOIN_KEY = "partitionedJoin"; - private String cacheType = "none";//None or LRU + private String cacheType = "none";//None or LRU or ALL private int cacheSize = 10000; diff --git a/hbase/hbase-side/doc/readme.txt b/hbase/hbase-side/doc/readme.txt deleted file mode 100644 index 06d628f51..000000000 --- a/hbase/hbase-side/doc/readme.txt +++ /dev/null @@ -1,57 +0,0 @@ -1: example: -CREATE TABLE sideTable( - cf:name String as name, - cf:info int as info, - PRIMARY KEY(md5(name) + 'test'), - PERIOD FOR SYSTEM_TIME - )WITH( - type ='hbase', - zookeeperQuorum ='rdos1:2181', - zookeeperParent ='/hbase', - tableName ='workerinfo', - cache ='LRU', - cacheSize ='10000', - cacheTTLMs ='60000', - parallelism ='1', - partitionedJoin='true' - ); - -2: 格式: - CREATE TABLE tableName( - columnFamily:columnName type as alias, - ... - PRIMARY KEY(keyInfo), - PERIOD FOR SYSTEM_TIME - )WITH( - type ='hbase', - zookeeperQuorum ='ip:port', - zookeeperParent ='/hbase', - tableName ='tableNamae', - cache ='LRU', - cacheSize ='10000', - cacheTTLMs ='60000', - parallelism ='1', - partitionedJoin='false' - ); - - -tableName ==> 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同) -columnFamily:columnName ==> hbase中的列族名称和列名称 -alias ===> hbase 中的列对应到flink中注册的列名称 -PERIOD FOR SYSTEM_TIME ==> 关键字表明该定义的表为维表信息 -PRIMARY KEY(keyInfo) ==> 维表主键定义;hbase 维表为rowkey的构造方式; - 可选择的构造包括 md5(alias + alias), '常量',也包括上述方式的自由组合 -type ==> 表明维表的类型 -zookeeperQuorum ==> hbase 的zk地址;格式ip:port[;ip:port] -zookeeperParent ==> hbase 的zk parent路径 -tableName ==> hbase 的表名称 -cache ==> 维表缓存策略(NONE/LRU) -partitionedJoin ==> 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量) - -NONE: 不做内存缓存 - -LRU: -cacheSize ==> 缓存的条目数量 -cacheTTLMs ==> 缓存的过期时间(ms) - - diff --git a/hbase/hbase-side/hbase-all-side/pom.xml b/hbase/hbase-side/hbase-all-side/pom.xml new file mode 100644 index 000000000..ce4a98788 --- /dev/null +++ b/hbase/hbase-side/hbase-all-side/pom.xml @@ -0,0 +1,76 @@ + + + + sql.side.hbase + com.dtstack.flinkx + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.all.hbase + hbase-all-side + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/hbase/hbase-side/hbase-async-side/pom.xml b/hbase/hbase-side/hbase-async-side/pom.xml new file mode 100644 index 000000000..df8a13e91 --- /dev/null +++ b/hbase/hbase-side/hbase-async-side/pom.xml @@ -0,0 +1,93 @@ + + + + sql.side.hbase + com.dtstack.flinkx + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.async.hbase + hbase-async-side + + + + + com.dtstack.flinkx + sql.side.hbase.core + 1.0-SNAPSHOT + + + + org.hbase + asynchbase + 1.8.2 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java similarity index 72% rename from hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java rename to hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java index cec7a0400..1aafde9e5 100644 --- a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java @@ -24,12 +24,14 @@ import com.dtstack.flink.sql.side.AsyncReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideReqRow; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.hbase.rowkeydealer.AbsRowKeyModeDealer; import com.dtstack.flink.sql.side.hbase.rowkeydealer.PreRowKeyModeDealerDealer; import com.dtstack.flink.sql.side.hbase.rowkeydealer.RowKeyEqualModeDealer; import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo; +import com.dtstack.flink.sql.threadFactory.DTThreadFactory; import com.google.common.collect.Maps; import com.stumbleupon.async.Deferred; import org.apache.calcite.sql.SqlBasicCall; @@ -43,7 +45,7 @@ import org.hbase.async.HBaseClient; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.dtstack.flink.sql.threadFactory.DTThreadFactory; + import java.sql.Timestamp; import java.util.Collections; import java.util.List; @@ -67,8 +69,6 @@ public class HbaseAsyncReqRow extends AsyncReqRow { private static final int HBASE_WORKER_POOL_SIZE = 10; - private RowKeyBuilder rowKeyBuilder; - private transient HBaseClient hBaseClient; private transient AbsRowKeyModeDealer rowKeyMode; @@ -77,54 +77,23 @@ public class HbaseAsyncReqRow extends AsyncReqRow { private String[] colNames; - private Map colRefType; - public HbaseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); - } - - @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { - rowKeyBuilder = new RowKeyBuilder(); - if(sideTableInfo.getPrimaryKeys().size() < 1){ - throw new RuntimeException("Primary key dimension table must be filled"); - } - - HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo; - rowKeyBuilder.init(sideTableInfo.getPrimaryKeys().get(0)); + super(new HbaseSideReqRow(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); - colRefType = Maps.newHashMap(); - for(int i=0; i sqlNodeList = Lists.newArrayList(); - if(conditionNode.getKind() == SqlKind.AND){ - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); - }else{ - sqlNodeList.add(conditionNode); - } - - for(SqlNode sqlNode : sqlNodeList){ - dealOneEqualCon(sqlNode, sideTableName); - } - - tableName = hbaseSideTableInfo.getTableName(); - colNames = hbaseSideTableInfo.getColumnRealNames(); + tableName = ((HbaseSideTableInfo)sideTableInfo).getTableName(); + colNames = ((HbaseSideTableInfo)sideTableInfo).getColumnRealNames(); } + @Override public void open(Configuration parameters) throws Exception { + SideTableInfo sideTableInfo = sideReqRow.getSideTableInfo(); HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo; ExecutorService executorService =new ThreadPoolExecutor(HBASE_WORKER_POOL_SIZE, HBASE_WORKER_POOL_SIZE, 0L, TimeUnit.MILLISECONDS, - new LinkedBlockingQueue<>(),new DTThreadFactory("hbase-aysnc")); + new LinkedBlockingQueue<>(), new DTThreadFactory("hbase-aysnc")); hBaseClient = new HBaseClient(hbaseSideTableInfo.getHost(), hbaseSideTableInfo.getParent(), executorService); + try { Deferred deferred = hBaseClient.ensureTableExists(tableName) .addCallbacks(arg -> new CheckResult(true, ""), arg -> new CheckResult(false, arg.toString())); @@ -138,29 +107,32 @@ public void open(Configuration parameters) throws Exception { throw new RuntimeException("create hbase connection fail:", e); } + HbaseSideReqRow hbaseSideReqRow = (HbaseSideReqRow) sideReqRow; if(hbaseSideTableInfo.isPreRowKey()){ - rowKeyMode = new PreRowKeyModeDealerDealer(colRefType, colNames, hBaseClient, - openCache(), joinType, outFieldInfoList, inFieldIndex, sideFieldIndex); + rowKeyMode = new PreRowKeyModeDealerDealer(hbaseSideReqRow.getColRefType(), colNames, hBaseClient, + openCache(), sideReqRow.getJoinType(), sideReqRow.getOutFieldInfoList(), + sideReqRow.getInFieldIndex(), sideReqRow.getSideFieldIndex()); }else{ - rowKeyMode = new RowKeyEqualModeDealer(colRefType, colNames, hBaseClient, - openCache(), joinType, outFieldInfoList, inFieldIndex, sideFieldIndex); + rowKeyMode = new RowKeyEqualModeDealer(hbaseSideReqRow.getColRefType(), colNames, hBaseClient, + openCache(), sideReqRow.getJoinType(), sideReqRow.getOutFieldInfoList(), + sideReqRow.getInFieldIndex(), sideReqRow.getSideFieldIndex()); } } @Override public void asyncInvoke(Row input, ResultFuture resultFuture) throws Exception { Map refData = Maps.newHashMap(); - for (int i = 0; i < equalValIndex.size(); i++) { - Integer conValIndex = equalValIndex.get(i); + for (int i = 0; i < sideReqRow.getEqualValIndex().size(); i++) { + Integer conValIndex = sideReqRow.getEqualValIndex().get(i); Object equalObj = input.getField(conValIndex); if(equalObj == null){ resultFuture.complete(null); } - refData.put(equalFieldList.get(i), equalObj); + refData.put(sideReqRow.getEqualFieldList().get(i), equalObj); } - String rowKeyStr = rowKeyBuilder.getRowKey(refData); + String rowKeyStr = ((HbaseSideReqRow)sideReqRow).getRowKeyBuilder().getRowKey(refData); //get from cache if(openCache()){ @@ -182,15 +154,15 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except } } - rowKeyMode.asyncGetData(tableName, rowKeyStr, input, resultFuture, sideCache); + rowKeyMode.asyncGetData(tableName, rowKeyStr, input, resultFuture, sideReqRow.getSideCache()); } @Override protected Row fillData(Row input, Object sideInput){ List sideInputList = (List) sideInput; - Row row = new Row(outFieldInfoList.size()); - for(Map.Entry entry : inFieldIndex.entrySet()){ + Row row = new Row(sideReqRow.getOutFieldInfoList().size()); + for(Map.Entry entry : sideReqRow.getInFieldIndex().entrySet()){ Object obj = input.getField(entry.getValue()); if(obj instanceof Timestamp){ obj = ((Timestamp)obj).getTime(); @@ -198,7 +170,7 @@ protected Row fillData(Row input, Object sideInput){ row.setField(entry.getKey(), obj); } - for(Map.Entry entry : sideFieldIndex.entrySet()){ + for(Map.Entry entry : sideReqRow.getSideFieldIndex().entrySet()){ if(sideInputList == null){ row.setField(entry.getKey(), null); }else{ diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseSideReqRow.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseSideReqRow.java new file mode 100644 index 000000000..5f8316ccd --- /dev/null +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseSideReqRow.java @@ -0,0 +1,83 @@ +package com.dtstack.flink.sql.side.hbase; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideReqRow; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2018/9/18 + * Company: www.dtstack.com + * @author xuchao + */ + +public class HbaseSideReqRow extends SideReqRow { + + private RowKeyBuilder rowKeyBuilder; + + private Map colRefType; + + public HbaseSideReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + rowKeyBuilder = new RowKeyBuilder(); + if(sideTableInfo.getPrimaryKeys().size() < 1){ + throw new RuntimeException("Primary key dimension table must be filled"); + } + + HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo; + rowKeyBuilder.init(sideTableInfo.getPrimaryKeys().get(0)); + + colRefType = Maps.newHashMap(); + for(int i=0; i sqlNodeList = Lists.newArrayList(); + if(conditionNode.getKind() == SqlKind.AND){ + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); + }else{ + sqlNodeList.add(conditionNode); + } + + for(SqlNode sqlNode : sqlNodeList){ + dealOneEqualCon(sqlNode, sideTableName); + } + + } + + public RowKeyBuilder getRowKeyBuilder() { + return rowKeyBuilder; + } + + public void setRowKeyBuilder(RowKeyBuilder rowKeyBuilder) { + this.rowKeyBuilder = rowKeyBuilder; + } + + public Map getColRefType() { + return colRefType; + } + + public void setColRefType(Map colRefType) { + this.colRefType = colRefType; + } +} diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbsRowKeyModeDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbsRowKeyModeDealer.java similarity index 100% rename from hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbsRowKeyModeDealer.java rename to hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbsRowKeyModeDealer.java diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java similarity index 100% rename from hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java rename to hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java index a5e536ff8..d077f3493 100644 --- a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java @@ -21,11 +21,11 @@ package com.dtstack.flink.sql.side.hbase.rowkeydealer; import com.dtstack.flink.sql.enums.ECacheContentType; -import com.dtstack.flink.sql.side.hbase.utils.HbaseUtils; import com.dtstack.flink.sql.side.CacheMissVal; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.cache.AbsSideCache; import com.dtstack.flink.sql.side.cache.CacheObj; +import com.dtstack.flink.sql.side.hbase.utils.HbaseUtils; import com.google.common.collect.Maps; import org.apache.calcite.sql.JoinType; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java similarity index 100% rename from hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java rename to hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java index 5437ac8c1..a85f58c4b 100644 --- a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java @@ -21,11 +21,11 @@ package com.dtstack.flink.sql.side.hbase.rowkeydealer; import com.dtstack.flink.sql.enums.ECacheContentType; -import com.dtstack.flink.sql.side.hbase.utils.HbaseUtils; import com.dtstack.flink.sql.side.CacheMissVal; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.cache.AbsSideCache; import com.dtstack.flink.sql.side.cache.CacheObj; +import com.dtstack.flink.sql.side.hbase.utils.HbaseUtils; import com.google.common.collect.Maps; import org.apache.calcite.sql.JoinType; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; diff --git a/hbase/hbase-side/hbase-side-core/pom.xml b/hbase/hbase-side/hbase-side-core/pom.xml new file mode 100644 index 000000000..600e37fb7 --- /dev/null +++ b/hbase/hbase-side/hbase-side-core/pom.xml @@ -0,0 +1,15 @@ + + + + sql.side.hbase + com.dtstack.flinkx + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.hbase.core + jar + + \ No newline at end of file diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/Md5ReplaceOperator.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/Md5ReplaceOperator.java similarity index 100% rename from hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/Md5ReplaceOperator.java rename to hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/Md5ReplaceOperator.java diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceInfo.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceInfo.java similarity index 100% rename from hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceInfo.java rename to hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceInfo.java diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceOperator.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceOperator.java similarity index 100% rename from hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceOperator.java rename to hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/ReplaceOperator.java diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java similarity index 100% rename from hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java rename to hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceOpType.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceOpType.java similarity index 100% rename from hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceOpType.java rename to hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceOpType.java diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceType.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceType.java similarity index 100% rename from hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceType.java rename to hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/enums/EReplaceType.java diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java similarity index 100% rename from hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java rename to hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideTableInfo.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideTableInfo.java similarity index 100% rename from hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideTableInfo.java rename to hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideTableInfo.java diff --git a/hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseUtils.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseUtils.java similarity index 100% rename from hbase/hbase-side/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseUtils.java rename to hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/utils/HbaseUtils.java diff --git a/hbase/hbase-side/pom.xml b/hbase/hbase-side/pom.xml index b73c99dea..b54c6c9d0 100644 --- a/hbase/hbase-side/pom.xml +++ b/hbase/hbase-side/pom.xml @@ -11,76 +11,13 @@ sql.side.hbase hbase-side - jar - + + hbase-all-side + hbase-async-side + hbase-side-core + - - org.hbase - asynchbase - 1.8.2 - - + pom - - - - org.apache.maven.plugins - maven-shade-plugin - 1.4 - - - package - - shade - - - - - - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - - - - maven-antrun-plugin - 1.2 - - - copy-resources - - package - - run - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/mysql/mysql-side/mysql-all-side/pom.xml b/mysql/mysql-side/mysql-all-side/pom.xml new file mode 100644 index 000000000..f502aa608 --- /dev/null +++ b/mysql/mysql-side/mysql-all-side/pom.xml @@ -0,0 +1,87 @@ + + + + sql.mysql + com.dtstack.flinkx + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.all.mysql + mysql-all-side + + jar + + + + com.dtstack.flinkx + sql.side.mysql.core + 1.0-SNAPSHOT + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/mysql/mysql-side/mysql-async-side/pom.xml b/mysql/mysql-side/mysql-async-side/pom.xml new file mode 100644 index 000000000..4571d8d66 --- /dev/null +++ b/mysql/mysql-side/mysql-async-side/pom.xml @@ -0,0 +1,101 @@ + + + + sql.side.mysql + com.dtstack.flinkx + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.async.mysql + + mysql-async-side + + jar + + + + + + io.vertx + vertx-jdbc-client + 3.5.2 + + + + io.vertx + vertx-core + 3.5.2 + + + + com.dtstack.flinkx + sql.side.mysql.core + 1.0-SNAPSHOT + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java similarity index 62% rename from mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java rename to mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java index 46cbb05d0..6230d3174 100644 --- a/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -21,13 +21,14 @@ package com.dtstack.flink.sql.side.mysql; import com.dtstack.flink.sql.enums.ECacheContentType; -import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; import com.dtstack.flink.sql.side.AsyncReqRow; import com.dtstack.flink.sql.side.CacheMissVal; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideReqRow; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.cache.CacheObj; +import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; import io.vertx.core.Vertx; import io.vertx.core.VertxOptions; import io.vertx.core.json.JsonArray; @@ -77,129 +78,16 @@ public class MysqlAsyncReqRow extends AsyncReqRow { public MysqlAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); - } - - @Override - public void parseSelectFields(JoinInfo joinInfo){ - String sideTableName = joinInfo.getSideTableName(); - String nonSideTableName = joinInfo.getNonSideTable(); - List fields = Lists.newArrayList(); - - int sideIndex = 0; - for( int i=0; i sqlNodeList = Lists.newArrayList(); - if(conditionNode.getKind() == SqlKind.AND){ - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); - }else{ - sqlNodeList.add(conditionNode); - } - - for(SqlNode sqlNode : sqlNodeList){ - dealOneEqualCon(sqlNode, sideTableName); - } - - sqlCondition = "select ${selectField} from ${tableName} where "; - for(int i=0; i resultFuture) throws Exception { JsonArray inputParams = new JsonArray(); - for(Integer conValIndex : equalValIndex){ + for(Integer conValIndex : sideReqRow.getEqualValIndex()){ Object equalObj = input.getField(conValIndex); if(equalObj == null){ resultFuture.complete(null); @@ -257,6 +145,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except } final SQLConnection connection = conn.result(); + String sqlCondition = sideReqRow.getSqlCondition(); connection.queryWithParams(sqlCondition, inputParams, rs -> { if (rs.failed()) { LOG.error("Cannot retrieve the data from the database"); @@ -300,8 +189,8 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except @Override public Row fillData(Row input, Object line){ JsonArray jsonArray = (JsonArray) line; - Row row = new Row(outFieldInfoList.size()); - for(Map.Entry entry : inFieldIndex.entrySet()){ + Row row = new Row(sideReqRow.getOutFieldInfoList().size()); + for(Map.Entry entry : sideReqRow.getInFieldIndex().entrySet()){ Object obj = input.getField(entry.getValue()); if(obj instanceof Timestamp){ obj = ((Timestamp)obj).getTime(); @@ -309,7 +198,7 @@ public Row fillData(Row input, Object line){ row.setField(entry.getKey(), obj); } - for(Map.Entry entry : sideFieldIndex.entrySet()){ + for(Map.Entry entry : sideReqRow.getInFieldIndex().entrySet()){ if(jsonArray == null){ row.setField(entry.getKey(), null); }else{ diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlSideReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlSideReqRow.java new file mode 100644 index 000000000..4a4527175 --- /dev/null +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlSideReqRow.java @@ -0,0 +1,145 @@ +package com.dtstack.flink.sql.side.mysql; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideReqRow; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.List; + +/** + * Reason: + * Date: 2018/9/18 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class MysqlSideReqRow extends SideReqRow { + + public MysqlSideReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + MysqlSideTableInfo mysqlSideTableInfo = (MysqlSideTableInfo) sideTableInfo; + + String sideTableName = joinInfo.getSideTableName(); + + SqlNode conditionNode = joinInfo.getCondition(); + + List sqlNodeList = Lists.newArrayList(); + if(conditionNode.getKind() == SqlKind.AND){ + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); + }else{ + sqlNodeList.add(conditionNode); + } + + for(SqlNode sqlNode : sqlNodeList){ + dealOneEqualCon(sqlNode, sideTableName); + } + + sqlCondition = "select ${selectField} from ${tableName} where "; + for(int i=0; i fields = Lists.newArrayList(); + + int sideIndex = 0; + for( int i=0; i + + + sql.side.mysql + com.dtstack.flinkx + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.mysql.core + + + \ No newline at end of file diff --git a/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java b/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java similarity index 100% rename from mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java rename to mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java diff --git a/mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java b/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java similarity index 100% rename from mysql/mysql-side/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java rename to mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java diff --git a/mysql/mysql-side/pom.xml b/mysql/mysql-side/pom.xml index 031a37072..df60240b0 100644 --- a/mysql/mysql-side/pom.xml +++ b/mysql/mysql-side/pom.xml @@ -12,84 +12,12 @@ sql.side.mysql mysql-side - - - - - io.vertx - vertx-jdbc-client - 3.5.2 - + + mysql-async-side + mysql-all-side + mysql-side-core + - - io.vertx - vertx-core - 3.5.2 - - - - jar - - - - - org.apache.maven.plugins - maven-shade-plugin - 1.4 - - - package - - shade - - - - - - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - - - - maven-antrun-plugin - 1.2 - - - copy-resources - - package - - run - - - - - - - - - - - - - - - - - + pom \ No newline at end of file diff --git a/mysql/pom.xml b/mysql/pom.xml index aa2d20ab4..6afcd9719 100644 --- a/mysql/pom.xml +++ b/mysql/pom.xml @@ -14,6 +14,7 @@ mysql-sink mysql-side + mysql-all-side From 750dbd07f620edfe90e9ce606348d6c91a2e0995 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 20 Sep 2018 16:34:56 +0800 Subject: [PATCH 025/470] mysql side with all-cache Strategy --- .../flink/sql/parser/CreateTableParser.java | 8 +- .../flink/sql/side/AllCacheOperator.java | 45 ---- .../com/dtstack/flink/sql/side/AllReqRow.java | 37 ++- .../dtstack/flink/sql/side/AsyncReqRow.java | 29 +-- .../side/{SideReqRow.java => SideInfo.java} | 22 +- .../dtstack/flink/sql/side/SideSqlExec.java | 10 +- .../flink/sql/side/StreamSideFactory.java | 15 +- .../SideAsyncOperator.java} | 16 +- .../operator/SideWithAllCacheOperator.java | 69 ++++++ .../flink/sql/sink/StreamSinkFactory.java | 30 ++- .../flink/sql/source/StreamSourceFactory.java | 19 +- .../sql/table/TableInfoParserFactory.java | 5 +- .../dtstack/flink/sql/util/DtStringUtil.java | 70 ++++++ .../dtstack/flink/sql/util/PluginUtil.java | 4 +- .../flink/sql/side/SideSqlExecTest.java | 57 ++++- hbase/hbase-side/hbase-all-side/pom.xml | 9 +- hbase/hbase-side/hbase-async-side/pom.xml | 11 +- .../sql/side/hbase/HbaseAsyncReqRow.java | 39 ++- ...ideReqRow.java => HbaseAsyncSideInfo.java} | 8 +- hbase/hbase-side/hbase-side-core/pom.xml | 3 +- kafka09/kafka09-source/pom.xml | 6 +- kafka10/kafka10-source/pom.xml | 6 +- kafka11/kafka11-source/pom.xml | 6 +- mysql/mysql-side/mysql-all-side/pom.xml | 13 +- .../flink/sql/side/mysql/MysqlAllReqRow.java | 222 ++++++++++++++++++ .../sql/side/mysql/MysqlAllSideInfo.java | 95 ++++++++ mysql/mysql-side/mysql-async-side/pom.xml | 12 +- .../sql/side/mysql/MysqlAsyncReqRow.java | 21 +- ...ideReqRow.java => MysqlAsyncSideInfo.java} | 54 ++--- mysql/mysql-side/mysql-side-core/pom.xml | 13 +- mysql/mysql-side/pom.xml | 2 +- mysql/pom.xml | 1 - pom.xml | 2 +- 33 files changed, 748 insertions(+), 211 deletions(-) delete mode 100644 core/src/main/java/com/dtstack/flink/sql/side/AllCacheOperator.java rename core/src/main/java/com/dtstack/flink/sql/side/{SideReqRow.java => SideInfo.java} (89%) rename core/src/main/java/com/dtstack/flink/sql/side/{LRUCacheOperator.java => operator/SideAsyncOperator.java} (82%) create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java rename hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/{HbaseSideReqRow.java => HbaseAsyncSideInfo.java} (89%) create mode 100644 mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java create mode 100644 mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java rename mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/{MysqlSideReqRow.java => MysqlAsyncSideInfo.java} (74%) diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java index 72443f301..d10d6825e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java @@ -20,8 +20,10 @@ package com.dtstack.flink.sql.parser; +import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import java.util.List; import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -70,9 +72,9 @@ private Map parseProp(String propsStr){ String[] strs = propsStr.trim().split("'\\s*,"); Map propMap = Maps.newHashMap(); for(int i=0; i ss = DtStringUtil.splitIgnoreQuota(strs[i], '='); + String key = ss.get(0).trim(); + String value = ss.get(1).trim().replaceAll("'", "").trim(); propMap.put(key, value); } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AllCacheOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/AllCacheOperator.java deleted file mode 100644 index 6d445f6e2..000000000 --- a/core/src/main/java/com/dtstack/flink/sql/side/AllCacheOperator.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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 com.dtstack.flink.sql.side; - -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.streaming.api.datastream.DataStream; - -import java.util.List; - -/** - * get plugin which implement from RichFlatMapFunction - * Date: 2018/9/18 - * Company: www.dtstack.com - * - * @author xuchao - */ - -public class AllCacheOperator { - - private static void loadFlatMap(){ - - } - - public static DataStream getSideJoinDataStream(DataStream inputStream, String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, - List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { - return null; - } -} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java index fbe83e432..303260742 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java @@ -20,8 +20,15 @@ package com.dtstack.flink.sql.side; -import org.apache.calcite.interpreter.Row; +import com.dtstack.flink.sql.threadFactory.DTThreadFactory; import org.apache.flink.api.common.functions.RichFlatMapFunction; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.types.Row; + +import java.sql.SQLException; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; /** * Reason: @@ -31,4 +38,32 @@ */ public abstract class AllReqRow extends RichFlatMapFunction{ + + protected SideInfo sideInfo; + + private ScheduledExecutorService es; + + public AllReqRow(SideInfo sideInfo){ + this.sideInfo = sideInfo; + + } + + protected abstract Row fillData(Row input, Object sideInput); + + protected abstract void initCache() throws SQLException; + + protected abstract void reloadCache(); + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + initCache(); + System.out.println("----- all cacheRef init end-----"); + + //start reload cache thread + SideTableInfo sideTableInfo = sideInfo.getSideTableInfo(); + es = Executors.newSingleThreadScheduledExecutor(new DTThreadFactory("cache-all-reload")); + es.scheduleAtFixedRate(() -> reloadCache(), sideTableInfo.getCacheTimeout(), sideTableInfo.getCacheTimeout(), TimeUnit.MILLISECONDS); + } + } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java index bffdafcc8..6617bb29f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java @@ -25,23 +25,12 @@ import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.cache.LRUSideCache; import org.apache.calcite.sql.JoinType; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlNode; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.streaming.api.functions.async.RichAsyncFunction; import org.apache.flink.types.Row; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.Collections; -import java.util.List; -import java.util.Map; /** * All interfaces inherit naming rules: type + "AsyncReqRow" such as == "MysqlAsyncReqRow @@ -55,14 +44,14 @@ public abstract class AsyncReqRow extends RichAsyncFunction { private static final long serialVersionUID = 2098635244857937717L; - protected SideReqRow sideReqRow; + protected SideInfo sideInfo; - public AsyncReqRow(SideReqRow sideReqRow){ - this.sideReqRow = sideReqRow; + public AsyncReqRow(SideInfo sideInfo){ + this.sideInfo = sideInfo; } private void initCache(){ - SideTableInfo sideTableInfo = sideReqRow.getSideTableInfo(); + SideTableInfo sideTableInfo = sideInfo.getSideTableInfo(); if(sideTableInfo.getCacheType() == null || ECacheType.NONE.name().equalsIgnoreCase(sideTableInfo.getCacheType())){ return; } @@ -70,7 +59,7 @@ private void initCache(){ AbsSideCache sideCache; if(ECacheType.LRU.name().equalsIgnoreCase(sideTableInfo.getCacheType())){ sideCache = new LRUSideCache(sideTableInfo); - sideReqRow.setSideCache(sideCache); + sideInfo.setSideCache(sideCache); }else{ throw new RuntimeException("not support side cache with type:" + sideTableInfo.getCacheType()); } @@ -79,22 +68,22 @@ private void initCache(){ } protected CacheObj getFromCache(String key){ - return sideReqRow.getSideCache().getFromCache(key); + return sideInfo.getSideCache().getFromCache(key); } protected void putCache(String key, CacheObj value){ - sideReqRow.getSideCache().putCache(key, value); + sideInfo.getSideCache().putCache(key, value); } protected boolean openCache(){ - return sideReqRow.getSideCache() != null; + return sideInfo.getSideCache() != null; } protected abstract Row fillData(Row input, Object sideInput); protected void dealMissKey(Row input, ResultFuture resultFuture){ - if(sideReqRow.getJoinType() == JoinType.LEFT){ + if(sideInfo.getJoinType() == JoinType.LEFT){ //Reserved left table data Row row = fillData(input, null); resultFuture.complete(Collections.singleton(row)); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java similarity index 89% rename from core/src/main/java/com/dtstack/flink/sql/side/SideReqRow.java rename to core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java index f9b1be60e..924ef53f2 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideReqRow.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java @@ -30,6 +30,7 @@ import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import java.io.Serializable; import java.util.List; import java.util.Map; @@ -40,7 +41,7 @@ * @author xuchao */ -public abstract class SideReqRow { +public abstract class SideInfo implements Serializable{ protected RowTypeInfo rowTypeInfo; @@ -56,17 +57,21 @@ public abstract class SideReqRow { protected JoinType joinType; - //key:Returns the value of the position, returns the index values ​​in the input data + //key:Returns the value of the position, value: the ref field index​in the input table protected Map inFieldIndex = Maps.newHashMap(); + //key:Returns the value of the position, value: the ref field index​in the side table protected Map sideFieldIndex = Maps.newHashMap(); + //key:Returns the value of the position, value: the ref field name​in the side table + protected Map sideFieldNameIndex = Maps.newHashMap(); + protected SideTableInfo sideTableInfo; protected AbsSideCache sideCache; - public SideReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, - SideTableInfo sideTableInfo){ + public SideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, + SideTableInfo sideTableInfo){ this.rowTypeInfo = rowTypeInfo; this.outFieldInfoList = outFieldInfoList; this.joinType = joinInfo.getJoinType(); @@ -86,6 +91,7 @@ public void parseSelectFields(JoinInfo joinInfo){ if(fieldInfo.getTable().equalsIgnoreCase(sideTableName)){ fields.add(fieldInfo.getFieldName()); sideFieldIndex.put(i, sideIndex); + sideFieldNameIndex.put(i, fieldInfo.getFieldName()); sideIndex++; }else if(fieldInfo.getTable().equalsIgnoreCase(nonSideTableName)){ int nonSideIndex = rowTypeInfo.getFieldIndex(fieldInfo.getFieldName()); @@ -241,4 +247,12 @@ public AbsSideCache getSideCache() { public void setSideCache(AbsSideCache sideCache) { this.sideCache = sideCache; } + + public Map getSideFieldNameIndex() { + return sideFieldNameIndex; + } + + public void setSideFieldNameIndex(Map sideFieldNameIndex) { + this.sideFieldNameIndex = sideFieldNameIndex; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 7e813b3c0..3aed54554 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -20,9 +20,9 @@ package com.dtstack.flink.sql.side; -import com.dtstack.flink.sql.classloader.DtClassLoader; import com.dtstack.flink.sql.enums.ECacheType; -import com.dtstack.flink.sql.util.PluginUtil; +import com.dtstack.flink.sql.side.operator.SideAsyncOperator; +import com.dtstack.flink.sql.side.operator.SideWithAllCacheOperator; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlDataTypeSpec; import org.apache.calcite.sql.SqlIdentifier; @@ -42,7 +42,6 @@ import org.apache.flink.calcite.shaded.com.google.common.collect.HashBasedTable; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; -import org.apache.flink.streaming.api.datastream.AsyncDataStream; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; @@ -51,7 +50,6 @@ import java.util.List; import java.util.Map; import java.util.Queue; -import java.util.concurrent.TimeUnit; import static org.apache.calcite.sql.SqlKind.*; @@ -161,9 +159,9 @@ public void exec(String sql, Map sideTableMap, StreamTabl DataStream dsOut = null; if(ECacheType.ALL.name().equalsIgnoreCase(sideTableInfo.getCacheType())){ - + dsOut = SideWithAllCacheOperator.getSideJoinDataStream(adaptStream, sideTableInfo.getType(), localSqlPluginPath, typeInfo, joinInfo, sideJoinFieldInfo, sideTableInfo); }else{ - dsOut = LRUCacheOperator.getSideJoinDataStream(adaptStream, sideTableInfo.getType(), localSqlPluginPath, typeInfo, joinInfo, sideJoinFieldInfo, sideTableInfo); + dsOut = SideAsyncOperator.getSideJoinDataStream(adaptStream, sideTableInfo.getType(), localSqlPluginPath, typeInfo, joinInfo, sideJoinFieldInfo, sideTableInfo); } HashBasedTable mappingTable = HashBasedTable.create(); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java b/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java index 20281950b..7d8907301 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java @@ -26,7 +26,7 @@ import com.dtstack.flink.sql.util.PluginUtil; /** - * 流处理 + * get specify side parser * Date: 2018/7/25 * Company: www.dtstack.com * @author xuchao @@ -36,12 +36,19 @@ public class StreamSideFactory { private static final String CURR_TYPE = "side"; - public static AbsTableParser getSqlParser(String resultType, String sqlRootDir) throws Exception { + private static final String SIDE_DIR_TMPL = "%s%sside"; + + public static AbsTableParser getSqlParser(String pluginType, String sqlRootDir, String cacheType) throws Exception { + + cacheType = cacheType == null ? "async" : cacheType; + String sideDir = String.format(SIDE_DIR_TMPL, pluginType, cacheType); ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - String pluginJarPath = PluginUtil.getJarFileDirPath(resultType + CURR_TYPE, sqlRootDir); + String pluginJarPath = PluginUtil.getJarFileDirPath(sideDir, sqlRootDir); + DtClassLoader dtClassLoader = (DtClassLoader) classLoader; PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); - String className = PluginUtil.getSqlParserClassName(resultType, CURR_TYPE); + String className = PluginUtil.getSqlParserClassName(pluginType, CURR_TYPE); + Class sideParser = dtClassLoader.loadClass(className); if(!AbsSideTableParser.class.isAssignableFrom(sideParser)){ throw new RuntimeException("class " + sideParser.getName() + " not subClass of AbsSideTableParser"); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/LRUCacheOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java similarity index 82% rename from core/src/main/java/com/dtstack/flink/sql/side/LRUCacheOperator.java rename to core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java index 9c995885d..5e9da7605 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/LRUCacheOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java @@ -17,9 +17,13 @@ */ -package com.dtstack.flink.sql.side; +package com.dtstack.flink.sql.side.operator; import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.side.AsyncReqRow; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.AsyncDataStream; @@ -36,16 +40,18 @@ * @author xuchao */ -public class LRUCacheOperator { +public class SideAsyncOperator { + + private static final String PATH_FORMAT = "%sasyncside"; private static AsyncReqRow loadAsyncReq(String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, - JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { + JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - String pathOfType = sideType + "side"; + String pathOfType = String.format(PATH_FORMAT, sideType); String pluginJarPath = PluginUtil.getJarFileDirPath(pathOfType, sqlRootDir); DtClassLoader dtClassLoader = (DtClassLoader) classLoader; PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); - String className = PluginUtil.getSqlSideClassName(sideType, "side"); + String className = PluginUtil.getSqlSideClassName(sideType, "side", "Async"); return dtClassLoader.loadClass(className).asSubclass(AsyncReqRow.class) .getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, SideTableInfo.class).newInstance(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java new file mode 100644 index 000000000..72a67d00b --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java @@ -0,0 +1,69 @@ +/* + * 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 com.dtstack.flink.sql.side.operator; + +import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.side.AllReqRow; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; + +import java.net.MalformedURLException; +import java.util.List; + +/** + * get plugin which implement from RichFlatMapFunction + * Date: 2018/9/18 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class SideWithAllCacheOperator { + + private static final String PATH_FORMAT = "%sallside"; + + private static AllReqRow loadFlatMap(String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, + JoinInfo joinInfo, List outFieldInfoList, + SideTableInfo sideTableInfo) throws Exception { + + ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + String pathOfType = String.format(PATH_FORMAT, sideType); + String pluginJarPath = PluginUtil.getJarFileDirPath(pathOfType, sqlRootDir); + + DtClassLoader dtClassLoader = (DtClassLoader) classLoader; + PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); + String className = PluginUtil.getSqlSideClassName(sideType, "side", "All"); + + return dtClassLoader.loadClass(className).asSubclass(AllReqRow.class).getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, SideTableInfo.class) + .newInstance(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + + + } + + public static DataStream getSideJoinDataStream(DataStream inputStream, String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, + List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { + AllReqRow allReqRow = loadFlatMap(sideType, sqlRootDir, rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + return inputStream.flatMap(allReqRow); + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java b/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java index 238b4f4e5..9ef30f97c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java @@ -23,7 +23,9 @@ import com.dtstack.flink.sql.classloader.DtClassLoader; import com.dtstack.flink.sql.table.AbsTableParser; import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.table.sinks.TableSink; /** @@ -37,14 +39,23 @@ public class StreamSinkFactory { public static String CURR_TYPE = "sink"; - public static AbsTableParser getSqlParser(String resultType, String sqlRootDir) throws Exception { - String parserType = resultType + CURR_TYPE.substring(0, 1).toUpperCase() + CURR_TYPE.substring(1); + private static final String DIR_NAME_FORMAT = "%ssink"; + + public static AbsTableParser getSqlParser(String pluginType, String sqlRootDir) throws Exception { ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - String pluginJarPath = PluginUtil.getJarFileDirPath(resultType + CURR_TYPE, sqlRootDir); + + if(!(classLoader instanceof DtClassLoader)){ + throw new RuntimeException("it's not a correct classLoader instance, it's type must be DtClassLoader!"); + } + DtClassLoader dtClassLoader = (DtClassLoader) classLoader; + + String pluginJarPath = PluginUtil.getJarFileDirPath(String.format(DIR_NAME_FORMAT, pluginType), sqlRootDir); + PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); - String className = PluginUtil.getSqlParserClassName(resultType, CURR_TYPE); + String className = PluginUtil.getSqlParserClassName(pluginType, CURR_TYPE); Class targetParser = dtClassLoader.loadClass(className); + if(!AbsTableParser.class.isAssignableFrom(targetParser)){ throw new RuntimeException("class " + targetParser.getName() + " not subClass of AbsTableParser"); } @@ -59,13 +70,16 @@ public static TableSink getTableSink(TargetTableInfo targetTableInfo, String loc throw new RuntimeException("it's not a correct classLoader instance, it's type must be DtClassLoader!"); } - String resultType = targetTableInfo.getType(); - String pluginJarDirPath = PluginUtil.getJarFileDirPath(resultType + CURR_TYPE, localSqlRootDir); - String className = PluginUtil.getGenerClassName(resultType, CURR_TYPE); - DtClassLoader dtClassLoader = (DtClassLoader) classLoader; + + String pluginType = targetTableInfo.getType(); + String pluginJarDirPath = PluginUtil.getJarFileDirPath(String.format(DIR_NAME_FORMAT, pluginType), localSqlRootDir); + PluginUtil.addPluginJar(pluginJarDirPath, dtClassLoader); + + String className = PluginUtil.getGenerClassName(pluginType, CURR_TYPE); Class sinkClass = dtClassLoader.loadClass(className); + if(!IStreamSinkGener.class.isAssignableFrom(sinkClass)){ throw new RuntimeException("class " + sinkClass + " not subClass of IStreamSinkGener"); } diff --git a/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java b/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java index dca9e0fa3..b8dfe66e2 100644 --- a/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java @@ -24,7 +24,9 @@ import com.dtstack.flink.sql.classloader.DtClassLoader; import com.dtstack.flink.sql.table.AbsSourceParser; import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; @@ -40,15 +42,19 @@ public class StreamSourceFactory { private static final String CURR_TYPE = "source"; + private static final String DIR_NAME_FORMAT = "%ssource"; - public static AbsSourceParser getSqlParser(String resultType, String sqlRootDir) throws Exception { + public static AbsSourceParser getSqlParser(String pluginType, String sqlRootDir) throws Exception { - String parserType = resultType + CURR_TYPE.substring(0, 1).toUpperCase() + CURR_TYPE.substring(1); ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - String pluginJarPath = PluginUtil.getJarFileDirPath(resultType +CURR_TYPE, sqlRootDir); + + String pluginJarPath = PluginUtil.getJarFileDirPath(String.format(DIR_NAME_FORMAT, pluginType), sqlRootDir); + DtClassLoader dtClassLoader = (DtClassLoader) classLoader; PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); - String className = PluginUtil.getSqlParserClassName(resultType, CURR_TYPE); + + String typeNoVersion = DtStringUtil.getPluginTypeWithoutVersion(pluginType); + String className = PluginUtil.getSqlParserClassName(typeNoVersion, CURR_TYPE); Class sourceParser = dtClassLoader.loadClass(className); if(!AbsSourceParser.class.isAssignableFrom(sourceParser)){ throw new RuntimeException("class " + sourceParser.getName() + " not subClass of AbsSourceParser"); @@ -66,10 +72,11 @@ public static Table getStreamSource(SourceTableInfo sourceTableInfo, StreamExecu StreamTableEnvironment tableEnv, String sqlRootDir) throws Exception { String sourceTypeStr = sourceTableInfo.getType(); + String typeNoVersion = DtStringUtil.getPluginTypeWithoutVersion(sourceTypeStr); ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - String pluginJarPath = PluginUtil.getJarFileDirPath(sourceTypeStr + CURR_TYPE, sqlRootDir); - String className = PluginUtil.getGenerClassName(sourceTypeStr, CURR_TYPE); + String pluginJarPath = PluginUtil.getJarFileDirPath(String.format(DIR_NAME_FORMAT, sourceTypeStr), sqlRootDir); + String className = PluginUtil.getGenerClassName(typeNoVersion, CURR_TYPE); DtClassLoader dtClassLoader = (DtClassLoader) classLoader; PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java index ae53743bd..f74e7b620 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java @@ -20,8 +20,10 @@ package com.dtstack.flink.sql.table; +import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.enums.ETableType; import com.dtstack.flink.sql.parser.CreateTableParser; +import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.StreamSideFactory; import com.dtstack.flink.sql.sink.StreamSinkFactory; import com.dtstack.flink.sql.source.StreamSourceFactory; @@ -77,7 +79,8 @@ public static TableInfo parseWithTableType(int tableType, CreateTableParser.SqlP }else{ absTableParser = sideTableInfoMap.get(type); if(absTableParser == null){ - absTableParser = StreamSideFactory.getSqlParser(type, localPluginRoot); + String cacheType = MathUtil.getString(props.get(SideTableInfo.CACHE_KEY)); + absTableParser = StreamSideFactory.getSqlParser(type, localPluginRoot, cacheType); sideTableInfoMap.put(type, absTableParser); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index f0b9dfc02..db74480d6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -21,10 +21,15 @@ package com.dtstack.flink.sql.util; import com.dtstack.flink.sql.enums.ColumnType; +import org.apache.flink.calcite.shaded.com.google.common.base.Strings; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; import java.math.BigDecimal; import java.util.ArrayList; import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; /** * Reason: @@ -35,6 +40,8 @@ public class DtStringUtil { + private static final Pattern NO_VERSION_PATTERN = Pattern.compile("([a-zA-Z]+).*"); + /** * Split the specified string delimiter --- ignored quotes delimiter * @param str @@ -135,6 +142,69 @@ public static String col2string(Object column, String type) { return result.toString(); } + public static String getPluginTypeWithoutVersion(String engineType){ + + Matcher matcher = NO_VERSION_PATTERN.matcher(engineType); + if(!matcher.find()){ + return engineType; + } + + return matcher.group(1); + } + + /** + * add specify params to dbUrl + * @param dbUrl + * @param addParams + * @param isForce true:replace exists param + * @return + */ + public static String addJdbcParam(String dbUrl, Map addParams, boolean isForce){ + + if(Strings.isNullOrEmpty(dbUrl)){ + throw new RuntimeException("dburl can't be empty string, please check it."); + } + + if(addParams == null || addParams.size() == 0){ + return dbUrl; + } + + String[] splits = dbUrl.split("\\?"); + String preStr = splits[0]; + Map params = Maps.newHashMap(); + if(splits.length > 1){ + String existsParamStr = splits[1]; + String[] existsParams = existsParamStr.split("&"); + for(String oneParam : existsParams){ + String[] kv = oneParam.split("="); + if(kv.length != 2){ + throw new RuntimeException("illegal dbUrl:" + dbUrl); + } + params.put(kv[0], kv[1]); + } + } + for(Map.Entry addParam : addParams.entrySet()){ + if(!isForce && params.containsKey(addParam.getKey())){ + continue; + } + + params.put(addParam.getKey(), addParam.getValue()); + } + + //rebuild dbURL + StringBuilder sb = new StringBuilder(); + boolean isFirst = true; + for(Map.Entry param : params.entrySet()){ + if(!isFirst){ + sb.append("&"); + } + + sb.append(param.getKey()).append("=").append(param.getValue()); + isFirst = false; + } + + return preStr + "?" + sb.toString(); + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java index ed258cb03..812e22b42 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java @@ -75,8 +75,8 @@ public static String getSqlParserClassName(String pluginTypeName, String type){ } - public static String getSqlSideClassName(String pluginTypeName, String type){ - String pluginClassName = upperCaseFirstChar(pluginTypeName) + "AsyncReqRow"; + public static String getSqlSideClassName(String pluginTypeName, String type, String operatorType){ + String pluginClassName = upperCaseFirstChar(pluginTypeName) + operatorType + "ReqRow"; return CLASS_PRE_STR + "." + type.toLowerCase() + "." + pluginTypeName + "." + pluginClassName; } diff --git a/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java b/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java index e909b95e1..e29277fee 100644 --- a/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java +++ b/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java @@ -165,6 +165,61 @@ public void testRunHbaseSideTable() throws Exception { test(sql); } + @Test + public void testMysqlAllCache() throws Exception { + String sql = "CREATE TABLE MyTable(\n" + + " channel STRING,\n" + + " pv INT,\n" + + " xctime bigint,\n" + + " CHARACTER_LENGTH(channel) as timeLeng,\n" + + " WATERMARK FOR xctime AS withOffset(xctime,1000)\n" + + " )WITH(\n" + + " type='kafka09',\n" + + " bootstrapServers='172.16.8.198:9092',\n" + + " offsetReset='latest',\n" + + " topic='nbTest1'\n" + + " );\n" + + "CREATE TABLE MyResult(\n" + + " channel STRING,\n" + + " pv INT\n" + + " )WITH(\n" + + " type='mysql',\n" + + " url='jdbc:mysql://172.16.8.104:3306/test?charset=utf8',\n" + + " userName='dtstack',\n" + + " password='abc123',\n" + + " tableName='pv'\n" + + " );\n" + + "create table sideTable(\n" + + " channel String,\n" + + " xccount int,\n" + + " PRIMARY KEY(channel),\n" + + " PERIOD FOR SYSTEM_TIME\n" + + " )WITH(\n" + + " type='mysql',\n" + + " url='jdbc:mysql://172.16.8.104:3306/test?charset=utf8',\n" + + " userName='dtstack',\n" + + " password='abc123',\n" + + " tableName='sidetest',\n" + + " cache = 'ALL'\n" + + " );\n" + + "insert \n" + + "into\n" + + " MyResult\n" + + " select\n" + + " a.channel,\n" + + " b.xccount \n" + + " from\n" + + " MyTable a \n" + + " join\n" + + " sideTable b \n" + + " on a.channel=b.channel \n" + + " where\n" + + " b.channel = 'xc' \n" + + " and a.pv=10"; + + test(sql); + } + public void test(String sql) throws Exception { List paramList = Lists.newArrayList(); paramList.add("-sql"); @@ -173,7 +228,7 @@ public void test(String sql) throws Exception { paramList.add("-name"); paramList.add("xc"); paramList.add("-localSqlPluginPath"); - paramList.add("D:\\gitspace\\flink-sql-plugin\\plugins"); + paramList.add("D:\\gitspace\\flinkStreamSQL\\plugins"); paramList.add("-mode"); paramList.add("local"); paramList.add("-addjar"); diff --git a/hbase/hbase-side/hbase-all-side/pom.xml b/hbase/hbase-side/hbase-all-side/pom.xml index ce4a98788..f0f08689d 100644 --- a/hbase/hbase-side/hbase-all-side/pom.xml +++ b/hbase/hbase-side/hbase-all-side/pom.xml @@ -4,8 +4,9 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> sql.side.hbase - com.dtstack.flinkx + com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 @@ -58,14 +59,14 @@ - + - + diff --git a/hbase/hbase-side/hbase-async-side/pom.xml b/hbase/hbase-side/hbase-async-side/pom.xml index df8a13e91..6b8640c3c 100644 --- a/hbase/hbase-side/hbase-async-side/pom.xml +++ b/hbase/hbase-side/hbase-async-side/pom.xml @@ -4,8 +4,9 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> sql.side.hbase - com.dtstack.flinkx + com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 @@ -15,7 +16,7 @@ - com.dtstack.flinkx + com.dtstack.flink sql.side.hbase.core 1.0-SNAPSHOT @@ -73,14 +74,14 @@ - + - + diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java index 1aafde9e5..ae4be351c 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java @@ -24,7 +24,6 @@ import com.dtstack.flink.sql.side.AsyncReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideReqRow; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.hbase.rowkeydealer.AbsRowKeyModeDealer; @@ -34,11 +33,7 @@ import com.dtstack.flink.sql.threadFactory.DTThreadFactory; import com.google.common.collect.Maps; import com.stumbleupon.async.Deferred; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.types.Row; @@ -78,7 +73,7 @@ public class HbaseAsyncReqRow extends AsyncReqRow { private String[] colNames; public HbaseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(new HbaseSideReqRow(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + super(new HbaseAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); tableName = ((HbaseSideTableInfo)sideTableInfo).getTableName(); colNames = ((HbaseSideTableInfo)sideTableInfo).getColumnRealNames(); @@ -87,7 +82,7 @@ public HbaseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List resultFuture) throws Exception { Map refData = Maps.newHashMap(); - for (int i = 0; i < sideReqRow.getEqualValIndex().size(); i++) { - Integer conValIndex = sideReqRow.getEqualValIndex().get(i); + for (int i = 0; i < sideInfo.getEqualValIndex().size(); i++) { + Integer conValIndex = sideInfo.getEqualValIndex().get(i); Object equalObj = input.getField(conValIndex); if(equalObj == null){ resultFuture.complete(null); } - refData.put(sideReqRow.getEqualFieldList().get(i), equalObj); + refData.put(sideInfo.getEqualFieldList().get(i), equalObj); } - String rowKeyStr = ((HbaseSideReqRow)sideReqRow).getRowKeyBuilder().getRowKey(refData); + String rowKeyStr = ((HbaseAsyncSideInfo)sideInfo).getRowKeyBuilder().getRowKey(refData); //get from cache if(openCache()){ @@ -154,15 +149,15 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except } } - rowKeyMode.asyncGetData(tableName, rowKeyStr, input, resultFuture, sideReqRow.getSideCache()); + rowKeyMode.asyncGetData(tableName, rowKeyStr, input, resultFuture, sideInfo.getSideCache()); } @Override protected Row fillData(Row input, Object sideInput){ List sideInputList = (List) sideInput; - Row row = new Row(sideReqRow.getOutFieldInfoList().size()); - for(Map.Entry entry : sideReqRow.getInFieldIndex().entrySet()){ + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ Object obj = input.getField(entry.getValue()); if(obj instanceof Timestamp){ obj = ((Timestamp)obj).getTime(); @@ -170,7 +165,7 @@ protected Row fillData(Row input, Object sideInput){ row.setField(entry.getKey(), obj); } - for(Map.Entry entry : sideReqRow.getSideFieldIndex().entrySet()){ + for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ if(sideInputList == null){ row.setField(entry.getKey(), null); }else{ diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseSideReqRow.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncSideInfo.java similarity index 89% rename from hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseSideReqRow.java rename to hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncSideInfo.java index 5f8316ccd..4ff25ad9c 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseSideReqRow.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncSideInfo.java @@ -2,7 +2,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideReqRow; +import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo; import org.apache.calcite.sql.SqlBasicCall; @@ -22,13 +22,15 @@ * @author xuchao */ -public class HbaseSideReqRow extends SideReqRow { +public class HbaseAsyncSideInfo extends SideInfo { + + private static final long serialVersionUID = 257688427401088045L; private RowKeyBuilder rowKeyBuilder; private Map colRefType; - public HbaseSideReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + public HbaseAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/hbase/hbase-side/hbase-side-core/pom.xml b/hbase/hbase-side/hbase-side-core/pom.xml index 600e37fb7..3ac00ed3a 100644 --- a/hbase/hbase-side/hbase-side-core/pom.xml +++ b/hbase/hbase-side/hbase-side-core/pom.xml @@ -4,8 +4,9 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> sql.side.hbase - com.dtstack.flinkx + com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 diff --git a/kafka09/kafka09-source/pom.xml b/kafka09/kafka09-source/pom.xml index 992c3cdf3..e0dcb3dc2 100644 --- a/kafka09/kafka09-source/pom.xml +++ b/kafka09/kafka09-source/pom.xml @@ -76,14 +76,14 @@ - + - + diff --git a/kafka10/kafka10-source/pom.xml b/kafka10/kafka10-source/pom.xml index b00004633..052576816 100644 --- a/kafka10/kafka10-source/pom.xml +++ b/kafka10/kafka10-source/pom.xml @@ -75,14 +75,14 @@ - + - + diff --git a/kafka11/kafka11-source/pom.xml b/kafka11/kafka11-source/pom.xml index a1aa79668..055ec49a7 100644 --- a/kafka11/kafka11-source/pom.xml +++ b/kafka11/kafka11-source/pom.xml @@ -75,14 +75,14 @@ - + - + diff --git a/mysql/mysql-side/mysql-all-side/pom.xml b/mysql/mysql-side/mysql-all-side/pom.xml index f502aa608..4d6feb236 100644 --- a/mysql/mysql-side/mysql-all-side/pom.xml +++ b/mysql/mysql-side/mysql-all-side/pom.xml @@ -3,9 +3,10 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> - sql.mysql - com.dtstack.flinkx + sql.side.mysql + com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 @@ -16,7 +17,7 @@ - com.dtstack.flinkx + com.dtstack.flink sql.side.mysql.core 1.0-SNAPSHOT @@ -68,14 +69,14 @@ - + - + diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java new file mode 100644 index 000000000..78a32ce32 --- /dev/null +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java @@ -0,0 +1,222 @@ +package com.dtstack.flink.sql.side.mysql; + +import com.dtstack.flink.sql.side.AllReqRow; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; +import com.dtstack.flink.sql.threadFactory.DTThreadFactory; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.calcite.sql.JoinType; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Timestamp; +import java.util.Calendar; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +/** + * side operator with cache for all(period reload) + * Date: 2018/9/19 + * Company: www.dtstack.com + * @author xuchao + */ + +public class MysqlAllReqRow extends AllReqRow{ + + private static final long serialVersionUID = 2098635140857937717L; + + private static final Logger LOG = LoggerFactory.getLogger(MysqlAllReqRow.class); + + private static final String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; + + private static final int CONN_RETRY_NUM = 3; + + private static final int FETCH_SIZE = 1000; + + private AtomicReference>>> cacheRef = new AtomicReference<>(); + + public MysqlAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo){ + super(new MysqlAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + + @Override + protected Row fillData(Row input, Object sideInput) { + Map cacheInfo = (Map) sideInput; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ + Object obj = input.getField(entry.getValue()); + if(obj instanceof Timestamp){ + obj = ((Timestamp)obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for(Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()){ + if(cacheInfo == null){ + row.setField(entry.getKey(), null); + }else{ + row.setField(entry.getKey(), cacheInfo.get(entry.getValue())); + } + } + + return row; + } + + @Override + protected void initCache() throws SQLException { + Map>> newCache = Maps.newConcurrentMap(); + cacheRef.set(newCache); + loadData(newCache); + } + + @Override + protected void reloadCache() { + //reload cacheRef and replace to old cacheRef + Map>> newCache = Maps.newConcurrentMap(); + try { + loadData(newCache); + } catch (SQLException e) { + LOG.error("", e); + } + + cacheRef.set(newCache); + LOG.info("----- mysql all cacheRef reload end:{}", Calendar.getInstance()); + } + + + @Override + public void flatMap(Row value, Collector out) throws Exception { + List inputParams = Lists.newArrayList(); + for(Integer conValIndex : sideInfo.getEqualValIndex()){ + Object equalObj = value.getField(conValIndex); + if(equalObj == null){ + out.collect(null); + } + + inputParams.add(equalObj); + } + + String key = buildKey(inputParams); + List> cacheList = cacheRef.get().get(key); + if(CollectionUtils.isEmpty(cacheList)){ + if(sideInfo.getJoinType() == JoinType.LEFT){ + Row row = fillData(value, null); + out.collect(row); + }else{ + return; + } + + return; + } + + for(Map one : cacheList){ + out.collect(fillData(value, one)); + } + + } + + private String buildKey(List equalValList){ + StringBuilder sb = new StringBuilder(""); + for(Object equalVal : equalValList){ + sb.append(equalVal).append("_"); + } + + return sb.toString(); + } + + private String buildKey(Map val, List equalFieldList){ + StringBuilder sb = new StringBuilder(""); + for(String equalField : equalFieldList){ + sb.append(val.get(equalField)).append("_"); + } + + return sb.toString(); + } + + private Connection getConn(String dbURL, String userName, String password){ + try{ + Class.forName(MYSQL_DRIVER); + //add param useCursorFetch=true + Map addParams = Maps.newHashMap(); + addParams.put("useCursorFetch", "true"); + String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams,true); + return DriverManager.getConnection(targetDbUrl, userName, password); + }catch (Exception e){ + LOG.error("", e); + throw new RuntimeException("", e); + } + } + + + private void loadData(Map>> tmpCache) throws SQLException { + MysqlSideTableInfo tableInfo = (MysqlSideTableInfo) sideInfo.getSideTableInfo(); + Connection connection = null; + + try{ + for(int i=0; i oneRow = Maps.newHashMap(); + for(String fieldName : sideFieldNames){ + oneRow.put(fieldName.trim(), resultSet.getObject(fieldName.trim())); + } + + String cacheKey = buildKey(oneRow, sideInfo.getEqualFieldList()); + List> list = tmpCache.computeIfAbsent(cacheKey, key -> Lists.newArrayList()); + list.add(oneRow); + } + }catch (Exception e){ + LOG.error("", e); + }finally { + if(connection != null){ + connection.close(); + } + } + } +} diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java new file mode 100644 index 000000000..298311a05 --- /dev/null +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java @@ -0,0 +1,95 @@ +package com.dtstack.flink.sql.side.mysql; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.List; + +/** + * Reason: + * Date: 2018/9/19 + * Company: www.dtstack.com + * @author xuchao + */ + +public class MysqlAllSideInfo extends SideInfo{ + + private static final long serialVersionUID = -5858335638589472158L; + + public MysqlAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + MysqlSideTableInfo mysqlSideTableInfo = (MysqlSideTableInfo) sideTableInfo; + + sqlCondition = "select ${selectField} from ${tableName} "; + sqlCondition = sqlCondition.replace("${tableName}", mysqlSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); + System.out.println("---------side_exe_sql-----\n" + sqlCondition); + } + + @Override + public void parseSelectFields(JoinInfo joinInfo){ + String sideTableName = joinInfo.getSideTableName(); + String nonSideTableName = joinInfo.getNonSideTable(); + List fields = Lists.newArrayList(); + + int sideIndex = 0; + for( int i=0; i sqlNodeList = Lists.newArrayList(); + if(conditionNode.getKind() == SqlKind.AND){ + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); + }else{ + sqlNodeList.add(conditionNode); + } + + for(SqlNode sqlNode : sqlNodeList){ + dealOneEqualCon(sqlNode, sideTableName); + } + + if(CollectionUtils.isEmpty(equalFieldList)){ + throw new RuntimeException("no join condition found after table " + joinInfo.getLeftTableName()); + } + + for(String equalField : equalFieldList){ + if(fields.contains(equalField)){ + continue; + } + + fields.add(equalField); + } + + sideSelectFields = String.join(",", fields); + } +} diff --git a/mysql/mysql-side/mysql-async-side/pom.xml b/mysql/mysql-side/mysql-async-side/pom.xml index 4571d8d66..a7cf1a70e 100644 --- a/mysql/mysql-side/mysql-async-side/pom.xml +++ b/mysql/mysql-side/mysql-async-side/pom.xml @@ -4,8 +4,9 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> sql.side.mysql - com.dtstack.flinkx + com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 @@ -31,10 +32,11 @@ - com.dtstack.flinkx + com.dtstack.flink sql.side.mysql.core 1.0-SNAPSHOT + @@ -83,14 +85,14 @@ - + - + diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java index 6230d3174..979f43a7b 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -25,7 +25,6 @@ import com.dtstack.flink.sql.side.CacheMissVal; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideReqRow; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; @@ -36,10 +35,6 @@ import io.vertx.ext.jdbc.JDBCClient; import io.vertx.ext.sql.SQLClient; import io.vertx.ext.sql.SQLConnection; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.configuration.Configuration; @@ -78,16 +73,15 @@ public class MysqlAsyncReqRow extends AsyncReqRow { public MysqlAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(new MysqlSideReqRow(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + super(new MysqlAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } - //配置暂时走默认配置 @Override public void open(Configuration parameters) throws Exception { super.open(parameters); JsonObject mySQLClientConfig = new JsonObject(); - MysqlSideTableInfo mysqlSideTableInfo = (MysqlSideTableInfo) sideReqRow.getSideTableInfo(); + MysqlSideTableInfo mysqlSideTableInfo = (MysqlSideTableInfo) sideInfo.getSideTableInfo(); mySQLClientConfig.put("url", mysqlSideTableInfo.getUrl()) .put("driver_class", MYSQL_DRIVER) .put("max_pool_size", DEFAULT_MAX_DB_CONN_POOL_SIZE) @@ -98,7 +92,6 @@ public void open(Configuration parameters) throws Exception { vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); vo.setWorkerPoolSize(DEFAULT_VERTX_WORKER_POOL_SIZE); Vertx vertx = Vertx.vertx(vo); - //mySQLClient = JDBCClient.createShared(vertx, mySQLClientConfig, "MySQLPool"); mySQLClient = JDBCClient.createNonShared(vertx, mySQLClientConfig); } @@ -106,7 +99,7 @@ public void open(Configuration parameters) throws Exception { public void asyncInvoke(Row input, ResultFuture resultFuture) throws Exception { JsonArray inputParams = new JsonArray(); - for(Integer conValIndex : sideReqRow.getEqualValIndex()){ + for(Integer conValIndex : sideInfo.getEqualValIndex()){ Object equalObj = input.getField(conValIndex); if(equalObj == null){ resultFuture.complete(null); @@ -145,7 +138,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except } final SQLConnection connection = conn.result(); - String sqlCondition = sideReqRow.getSqlCondition(); + String sqlCondition = sideInfo.getSqlCondition(); connection.queryWithParams(sqlCondition, inputParams, rs -> { if (rs.failed()) { LOG.error("Cannot retrieve the data from the database"); @@ -189,8 +182,8 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except @Override public Row fillData(Row input, Object line){ JsonArray jsonArray = (JsonArray) line; - Row row = new Row(sideReqRow.getOutFieldInfoList().size()); - for(Map.Entry entry : sideReqRow.getInFieldIndex().entrySet()){ + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ Object obj = input.getField(entry.getValue()); if(obj instanceof Timestamp){ obj = ((Timestamp)obj).getTime(); @@ -198,7 +191,7 @@ public Row fillData(Row input, Object line){ row.setField(entry.getKey(), obj); } - for(Map.Entry entry : sideReqRow.getInFieldIndex().entrySet()){ + for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ if(jsonArray == null){ row.setField(entry.getKey(), null); }else{ diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlSideReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncSideInfo.java similarity index 74% rename from mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlSideReqRow.java rename to mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncSideInfo.java index 4a4527175..18f1220d9 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlSideReqRow.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncSideInfo.java @@ -1,8 +1,26 @@ +/* + * 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 com.dtstack.flink.sql.side.mysql; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideReqRow; +import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; import org.apache.calcite.sql.SqlBasicCall; @@ -22,9 +40,11 @@ * @author xuchao */ -public class MysqlSideReqRow extends SideReqRow { +public class MysqlAsyncSideInfo extends SideInfo { - public MysqlSideReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + private static final long serialVersionUID = -5931494270201575201L; + + public MysqlAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @@ -61,34 +81,6 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { System.out.println("---------side_exe_sql-----\n" + sqlCondition); } - @Override - public void parseSelectFields(JoinInfo joinInfo){ - String sideTableName = joinInfo.getSideTableName(); - String nonSideTableName = joinInfo.getNonSideTable(); - List fields = Lists.newArrayList(); - - int sideIndex = 0; - for( int i=0; i sql.side.mysql - com.dtstack.flinkx + com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 sql.side.mysql.core - + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + jar \ No newline at end of file diff --git a/mysql/mysql-side/pom.xml b/mysql/mysql-side/pom.xml index 335f90daa..78957c1e0 100644 --- a/mysql/mysql-side/pom.xml +++ b/mysql/mysql-side/pom.xml @@ -13,9 +13,9 @@ sql.side.mysql mysql-side + mysql-side-core mysql-async-side mysql-all-side - mysql-side-core pom diff --git a/mysql/pom.xml b/mysql/pom.xml index a0d3e1b56..747646f74 100644 --- a/mysql/pom.xml +++ b/mysql/pom.xml @@ -14,7 +14,6 @@ mysql-sink mysql-side - mysql-all-side diff --git a/pom.xml b/pom.xml index 2f49b99dd..b1d75c899 100644 --- a/pom.xml +++ b/pom.xml @@ -12,7 +12,7 @@ kafka11 mysql hbase - elasticsearch + elasticsearch5 launcher pom From 0d14dc700725b6688e054cbf1d7b354d9877765e Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 20 Sep 2018 17:28:07 +0800 Subject: [PATCH 026/470] modify base flinx version to 1.5.3 --- elasticsearch5/elasticsearch5-sink/pom.xml | 2 +- kafka09/kafka09-source/pom.xml | 5 ---- kafka10/kafka10-source/pom.xml | 5 ---- kafka11/kafka11-source/pom.xml | 5 ---- .../sql/launcher/ClusterClientFactory.java | 26 ++++++++++--------- mysql/mysql-sink/pom.xml | 5 ---- pom.xml | 2 +- 7 files changed, 16 insertions(+), 34 deletions(-) diff --git a/elasticsearch5/elasticsearch5-sink/pom.xml b/elasticsearch5/elasticsearch5-sink/pom.xml index f41de1a25..8e2533da0 100644 --- a/elasticsearch5/elasticsearch5-sink/pom.xml +++ b/elasticsearch5/elasticsearch5-sink/pom.xml @@ -16,7 +16,7 @@ org.apache.flink flink-connector-elasticsearch5_2.11 - 1.4.0 + ${flink.version} diff --git a/kafka09/kafka09-source/pom.xml b/kafka09/kafka09-source/pom.xml index e0dcb3dc2..59e05c020 100644 --- a/kafka09/kafka09-source/pom.xml +++ b/kafka09/kafka09-source/pom.xml @@ -14,11 +14,6 @@ kafka09-source http://maven.apache.org - - UTF-8 - 1.4.0 - - diff --git a/kafka10/kafka10-source/pom.xml b/kafka10/kafka10-source/pom.xml index 052576816..365f98048 100644 --- a/kafka10/kafka10-source/pom.xml +++ b/kafka10/kafka10-source/pom.xml @@ -13,11 +13,6 @@ kafka10-source http://maven.apache.org - - UTF-8 - 1.4.0 - - diff --git a/kafka11/kafka11-source/pom.xml b/kafka11/kafka11-source/pom.xml index 055ec49a7..98182aa64 100644 --- a/kafka11/kafka11-source/pom.xml +++ b/kafka11/kafka11-source/pom.xml @@ -13,11 +13,6 @@ kafka11-source http://maven.apache.org - - UTF-8 - 1.4.0 - - diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 960398906..83f635745 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -19,9 +19,11 @@ package com.dtstack.flink.sql.launcher; import org.apache.commons.lang.StringUtils; +import org.apache.flink.client.deployment.ClusterRetrieveException; import org.apache.flink.client.deployment.StandaloneClusterDescriptor; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.StandaloneClusterClient; +import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; @@ -29,6 +31,7 @@ import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterClient; import org.apache.flink.yarn.YarnClusterDescriptor; +import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; @@ -55,7 +58,7 @@ */ public class ClusterClientFactory { - public static ClusterClient createClusterClient(Properties props) { + public static ClusterClient createClusterClient(Properties props) throws ClusterRetrieveException { String clientType = props.getProperty(OPTION_MODE); if(clientType.equals(ClusterMode.MODE_STANDALONE)) { return createStandaloneClient(props); @@ -65,20 +68,20 @@ public static ClusterClient createClusterClient(Properties props) { throw new IllegalArgumentException("Unsupported cluster client type: "); } - public static StandaloneClusterClient createStandaloneClient(Properties props) { + public static RestClusterClient createStandaloneClient(Properties props) throws ClusterRetrieveException { String flinkConfDir = props.getProperty(LauncherOptions.OPTION_FLINK_CONF_DIR); Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); StandaloneClusterDescriptor descriptor = new StandaloneClusterDescriptor(config); - StandaloneClusterClient clusterClient = descriptor.retrieve(null); + RestClusterClient clusterClient = descriptor.retrieve(null); clusterClient.setDetached(true); return clusterClient; } - public static YarnClusterClient createYarnClient(Properties props) { + public static ClusterClient createYarnClient(Properties props) { String flinkConfDir = props.getProperty(LauncherOptions.OPTION_FLINK_CONF_DIR); Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); String yarnConfDir = props.getProperty(LauncherOptions.OPTION_YARN_CONF_DIR); - org.apache.hadoop.conf.Configuration yarnConf = new YarnConfiguration(); + YarnConfiguration yarnConf = new YarnConfiguration(); if(StringUtils.isNotBlank(yarnConfDir)) { try { @@ -96,6 +99,7 @@ public boolean accept(File dir, String name) { return false; } }); + if(xmlFileList != null) { for(File xmlFile : xmlFileList) { yarnConf.addResource(xmlFile.toURI().toURL()); @@ -105,7 +109,7 @@ public boolean accept(File dir, String name) { YarnClient yarnClient = YarnClient.createYarnClient(); yarnClient.init(yarnConf); yarnClient.start(); - String applicationId = null; + ApplicationId applicationId = null; Set set = new HashSet<>(); set.add("Apache Flink"); @@ -129,24 +133,22 @@ public boolean accept(File dir, String name) { if(thisMemory > maxMemory || thisMemory == maxMemory && thisCores > maxCores) { maxMemory = thisMemory; maxCores = thisCores; - applicationId = report.getApplicationId().toString(); + applicationId = report.getApplicationId(); } } - if(org.apache.commons.lang3.StringUtils.isEmpty(applicationId)) { + if(org.apache.commons.lang3.StringUtils.isEmpty(applicationId.toString())) { throw new RuntimeException("No flink session found on yarn cluster."); } - yarnClient.stop(); - - AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor(config, "."); + AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor(config, yarnConf, ".", yarnClient, false); Field confField = AbstractYarnClusterDescriptor.class.getDeclaredField("conf"); confField.setAccessible(true); haYarnConf(yarnConf); confField.set(clusterDescriptor, yarnConf); - YarnClusterClient clusterClient = clusterDescriptor.retrieve(applicationId); + ClusterClient clusterClient = clusterDescriptor.retrieve(applicationId); clusterClient.setDetached(true); return clusterClient; } diff --git a/mysql/mysql-sink/pom.xml b/mysql/mysql-sink/pom.xml index 807e27fc0..9fd82deb3 100644 --- a/mysql/mysql-sink/pom.xml +++ b/mysql/mysql-sink/pom.xml @@ -14,11 +14,6 @@ mysql-sink http://maven.apache.org - - UTF-8 - 1.4.0 - - org.apache.flink diff --git a/pom.xml b/pom.xml index b1d75c899..0922f739b 100644 --- a/pom.xml +++ b/pom.xml @@ -22,7 +22,7 @@ UTF-8 - 1.4.0 + 1.5.3 From 8eee095c33f29f32f07df81c7a3546f893270860 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 20 Sep 2018 19:49:46 +0800 Subject: [PATCH 027/470] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index a98acd785..998430f1d 100644 --- a/README.md +++ b/README.md @@ -11,7 +11,7 @@ # 后续开发计划 * 增加全局缓存功能 - * sql增加临时表功能 + * 增加临时表功能 * 增加redis维表功能 * 增加mongodb维表功能 * 增加oracle维表,结果表功能 From 53b080e379d91988f9bfbc9b8bc5bdaa76bfd66a Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 27 Sep 2018 15:37:23 +0800 Subject: [PATCH 028/470] add classpath to cache file --- core/src/main/java/com/dtstack/flink/sql/Main.java | 12 +++++++++++- .../java/com/dtstack/flink/sql/util/PluginUtil.java | 8 +++++++- .../flink/sql/launcher/ClusterClientFactory.java | 9 +++------ 3 files changed, 21 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index e4cd6efb9..4e85c61b5 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql; import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.parser.CreateFuncParser; import com.dtstack.flink.sql.parser.InsertSqlParser; import com.dtstack.flink.sql.side.SideSqlExec; @@ -83,6 +84,8 @@ public class Main { + private static final String CLASS_FILE_NAME_FMT = "class_path_%d"; + private static final ObjectMapper objMapper = new ObjectMapper(); private static final Logger LOG = LoggerFactory.getLogger(Main.class); @@ -272,8 +275,9 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en classPathSet.add( PluginUtil.getRemoteJarFilePath(tableInfo.getType(), TargetTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); } else if(tableInfo instanceof SideTableInfo){ + String sideOperator = ECacheType.ALL.name().equals(((SideTableInfo) tableInfo).getCacheType()) ? "all" : "async"; sideTableMap.put(tableInfo.getName(), (SideTableInfo) tableInfo); - classPathSet.add(PluginUtil.getRemoteJarFilePath(tableInfo.getType(), SideTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); + classPathSet.add(PluginUtil.getRemoteSideJarFilePath(tableInfo.getType(), sideOperator, SideTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); }else { throw new RuntimeException("not support table type:" + tableInfo.getType()); } @@ -281,6 +285,12 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en //The plug-in information corresponding to the table is loaded into the classPath env addEnvClassPath(env, classPathSet); + int i = 0; + for(URL url : classPathSet){ + String classFileName = String.format(CLASS_FILE_NAME_FMT, i); + env.registerCachedFile(url.getPath(), classFileName, true); + i++; + } } private static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws IOException { diff --git a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java index 812e22b42..04e5fa31a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java @@ -94,12 +94,18 @@ public static Properties stringToProperties(String str) throws IOException{ return properties; } - public static URL getRemoteJarFilePath(String pluginType, String tableType,String remoteSqlRootDir) throws MalformedURLException { + public static URL getRemoteJarFilePath(String pluginType, String tableType, String remoteSqlRootDir) throws MalformedURLException { String dirName = pluginType + tableType.toLowerCase(); String jarName = String.format("%s-%s.jar", pluginType, tableType.toLowerCase()); return new URL("file:" + remoteSqlRootDir + SP + dirName + SP + jarName); } + public static URL getRemoteSideJarFilePath(String pluginType, String sideOperator, String tableType, String remoteSqlRootDir) throws MalformedURLException { + String dirName = pluginType + sideOperator + tableType.toLowerCase(); + String jarName = String.format("%s-%s-%s.jar", pluginType, sideOperator, tableType.toLowerCase()); + return new URL("file:" + remoteSqlRootDir + SP + dirName + SP + jarName); + } + public static String upperCaseFirstChar(String str){ return str.substring(0, 1).toUpperCase() + str.substring(1); } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 83f635745..1b7a02371 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -107,6 +107,7 @@ public boolean accept(File dir, String name) { } YarnClient yarnClient = YarnClient.createYarnClient(); + haYarnConf(yarnConf); yarnClient.init(yarnConf); yarnClient.start(); ApplicationId applicationId = null; @@ -138,16 +139,12 @@ public boolean accept(File dir, String name) { } - if(org.apache.commons.lang3.StringUtils.isEmpty(applicationId.toString())) { + if(StringUtils.isEmpty(applicationId.toString())) { throw new RuntimeException("No flink session found on yarn cluster."); } - AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor(config, yarnConf, ".", yarnClient, false); - Field confField = AbstractYarnClusterDescriptor.class.getDeclaredField("conf"); - confField.setAccessible(true); - haYarnConf(yarnConf); - confField.set(clusterDescriptor, yarnConf); + AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor(config, yarnConf, ".", yarnClient, false); ClusterClient clusterClient = clusterDescriptor.retrieve(applicationId); clusterClient.setDetached(true); return clusterClient; From 93e7138d96854693b5d42d7af65addb846166524 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 28 Sep 2018 11:31:17 +0800 Subject: [PATCH 029/470] Update README.md --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 998430f1d..6e278a02c 100644 --- a/README.md +++ b/README.md @@ -16,6 +16,7 @@ * 增加mongodb维表功能 * 增加oracle维表,结果表功能 * 增加SQlServer维表,结果表功能 + * 增加kafka结果表功能 ## 1 快速起步 ### 1.1 运行模式 From 79c30cc32caaeff49b5c8499a0edad3fecdd3281 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 28 Sep 2018 11:34:35 +0800 Subject: [PATCH 030/470] Update kafka09Source.md --- docs/kafka09Source.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/kafka09Source.md b/docs/kafka09Source.md index e809ed916..84e43c56f 100644 --- a/docs/kafka09Source.md +++ b/docs/kafka09Source.md @@ -1,5 +1,7 @@ ## 1.格式: ``` +数据现在支持json格式{"xx":"bb","cc":"dd"} + CREATE TABLE tableName( colName colType, ... @@ -55,4 +57,4 @@ CREATE TABLE MyTable( topic ='nbTest1', parallelism ='1' ); -``` \ No newline at end of file +``` From 0638033ee398a662405c3172091cdd43703acc7a Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 28 Sep 2018 14:12:02 +0800 Subject: [PATCH 031/470] add start shell --- .gitignore | 1 + bin/submit.sh | 40 +++++++++++++++++++++ launcher/.gitignore | 12 +++++++ launcher/pom.xml | 85 +++++++++++++++++++++++++++++++++++++++++++++ 4 files changed, 138 insertions(+) create mode 100644 bin/submit.sh create mode 100644 launcher/.gitignore diff --git a/.gitignore b/.gitignore index a40fcbf7b..c88786513 100644 --- a/.gitignore +++ b/.gitignore @@ -8,3 +8,4 @@ target/ *.eclipse.* *.iml plugins/ +lib/ diff --git a/bin/submit.sh b/bin/submit.sh new file mode 100644 index 000000000..a03315342 --- /dev/null +++ b/bin/submit.sh @@ -0,0 +1,40 @@ +#!/usr/bin/env bash + +# +# 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. +# + +set -e + +export SQL_HOME="$(cd "`dirname "$0"`"/..; pwd)" + +# Find the java binary +if [ -n "${JAVA_HOME}" ]; then + JAVA_RUN="${JAVA_HOME}/bin/java" +else + if [ `command -v java` ]; then + JAVA_RUN="java" + else + echo "JAVA_HOME is not set" >&2 + exit 1 + fi +fi + +JAR_DIR=$SQL_HOME/lib/* +CLASS_NAME=com.dtstack.flink.sql.launcher.LauncherMain + +echo "sql submit ..." +nohup $JAVA_RUN -cp $JAR_DIR $CLASS_NAME $@ & \ No newline at end of file diff --git a/launcher/.gitignore b/launcher/.gitignore new file mode 100644 index 000000000..a4f1c3e53 --- /dev/null +++ b/launcher/.gitignore @@ -0,0 +1,12 @@ +target +.idea/ +/.idea/* +target/ +.class +.project +.classpath +*.eclipse.* +*.iml +plugins/ +lib/ +dependency-reduced-pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 5e23b03ea..927f6e2f3 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -31,4 +31,89 @@ + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.0.0 + + + package + + shade + + + + + + + + + + reference.conf + + + + core-default.xml + + + + core-site.xml + + + + yarn-default.xml + + + + mapred-default.xml + + + + mapred-site.xml + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + \ No newline at end of file From fc66f7748bdf08e3e691607596a28530f2cb3c3c Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 28 Sep 2018 14:17:58 +0800 Subject: [PATCH 032/470] Update README.md --- README.md | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index a98acd785..795edfcd8 100644 --- a/README.md +++ b/README.md @@ -37,14 +37,18 @@ ``` mvn clean package -Dmaven.test.skip +打包结束后,项目根目录下会产生plugins目录,plugins目录下存放编译好的数据同步插件包,在lib目下存放job提交的包 ``` -打包结束后,项目根目录下会产生plugins目录,plugins目录下存放编译好的数据同步插件包 +### 1.4 启动 +#### 1.4.1 启动命令 -### 1.4 启动 +``` +sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack/150_flinkplugin/sqlplugin -localSqlPluginPath D:\gitspace\flinkStreamSQL\plugins -mode yarn -flinkconf D:\flink_home\kudu150etc -yarnconf D:\hadoop\etc\hadoopkudu -confProp {\"time.characteristic\":\"EventTime\",\"sql.checkpoint.interval\":10000} +``` -#### 1.4.1 命令行参数选项 +#### 1.4.2 命令行参数选项 * **model** * 描述:执行模式,也就是flink集群的工作模式 From 5aaba540404f517e2cd3958baffa3e8f45f62722 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 28 Sep 2018 14:28:01 +0800 Subject: [PATCH 033/470] Update README.md --- README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 795edfcd8..befd865f0 100644 --- a/README.md +++ b/README.md @@ -37,6 +37,7 @@ ``` mvn clean package -Dmaven.test.skip + 打包结束后,项目根目录下会产生plugins目录,plugins目录下存放编译好的数据同步插件包,在lib目下存放job提交的包 ``` @@ -112,7 +113,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack ## 2 结构 ### 2.1 源表插件 -* [kafka09 源表插件](docs/kafka09Source.md) +* [kafka 源表插件](docs/kafka09Source.md) ### 2.2 结果表插件 * [elasticsearch 结果表插件](docs/elasticsearchSink.md) From cf3e737c657b5b4febd1083984c12338a00eef7f Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 28 Sep 2018 14:30:12 +0800 Subject: [PATCH 034/470] comment --- docs/{kafka09Source.md => kafkaSource.md} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename docs/{kafka09Source.md => kafkaSource.md} (98%) diff --git a/docs/kafka09Source.md b/docs/kafkaSource.md similarity index 98% rename from docs/kafka09Source.md rename to docs/kafkaSource.md index e809ed916..cc5b671b9 100644 --- a/docs/kafka09Source.md +++ b/docs/kafkaSource.md @@ -16,7 +16,7 @@ CREATE TABLE tableName( ``` ## 2.支持的版本 - kafka09 + kafka09,kafka10,kafka11 ## 3.表结构定义 From 427f200b971240b0940e5b8536b21ffa5bf843f6 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 28 Sep 2018 14:31:00 +0800 Subject: [PATCH 035/470] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index befd865f0..9f3eef21f 100644 --- a/README.md +++ b/README.md @@ -113,7 +113,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack ## 2 结构 ### 2.1 源表插件 -* [kafka 源表插件](docs/kafka09Source.md) +* [kafka 源表插件](docs/kafkaSource.md) ### 2.2 结果表插件 * [elasticsearch 结果表插件](docs/elasticsearchSink.md) From d6fb44a28e9da5387624d279dd63470c4d837aec Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 28 Sep 2018 19:36:05 +0800 Subject: [PATCH 036/470] fix can't find mysqlasyncside plugin bug --- core/src/main/java/com/dtstack/flink/sql/Main.java | 4 +++- .../main/java/com/dtstack/flink/sql/util/PluginUtil.java | 8 +++++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index e4cd6efb9..e31d6125b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql; import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.parser.CreateFuncParser; import com.dtstack.flink.sql.parser.InsertSqlParser; import com.dtstack.flink.sql.side.SideSqlExec; @@ -272,8 +273,9 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en classPathSet.add( PluginUtil.getRemoteJarFilePath(tableInfo.getType(), TargetTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); } else if(tableInfo instanceof SideTableInfo){ + String sideOperator = ECacheType.ALL.name().equals(((SideTableInfo) tableInfo).getCacheType()) ? "all" : "async"; sideTableMap.put(tableInfo.getName(), (SideTableInfo) tableInfo); - classPathSet.add(PluginUtil.getRemoteJarFilePath(tableInfo.getType(), SideTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); + classPathSet.add(PluginUtil.getRemoteSideJarFilePath(tableInfo.getType(), sideOperator, SideTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); }else { throw new RuntimeException("not support table type:" + tableInfo.getType()); } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java index 812e22b42..04e5fa31a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java @@ -94,12 +94,18 @@ public static Properties stringToProperties(String str) throws IOException{ return properties; } - public static URL getRemoteJarFilePath(String pluginType, String tableType,String remoteSqlRootDir) throws MalformedURLException { + public static URL getRemoteJarFilePath(String pluginType, String tableType, String remoteSqlRootDir) throws MalformedURLException { String dirName = pluginType + tableType.toLowerCase(); String jarName = String.format("%s-%s.jar", pluginType, tableType.toLowerCase()); return new URL("file:" + remoteSqlRootDir + SP + dirName + SP + jarName); } + public static URL getRemoteSideJarFilePath(String pluginType, String sideOperator, String tableType, String remoteSqlRootDir) throws MalformedURLException { + String dirName = pluginType + sideOperator + tableType.toLowerCase(); + String jarName = String.format("%s-%s-%s.jar", pluginType, sideOperator, tableType.toLowerCase()); + return new URL("file:" + remoteSqlRootDir + SP + dirName + SP + jarName); + } + public static String upperCaseFirstChar(String str){ return str.substring(0, 1).toUpperCase() + str.substring(1); } From 72c787360e2d64cbfc937b95fb054e5304962281 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 28 Sep 2018 19:48:05 +0800 Subject: [PATCH 037/470] fix can't find mysqlasyncside plugin bug --- .../com/dtstack/flink/sql/side/StreamSideFactory.java | 8 +++----- .../java/com/dtstack/flink/sql/util/PluginUtil.java | 11 +++++++++++ 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java b/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java index 7d8907301..4eb78b4c4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.side; import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.table.AbsSideTableParser; import com.dtstack.flink.sql.table.AbsTableParser; import com.dtstack.flink.sql.util.PluginUtil; @@ -36,14 +37,11 @@ public class StreamSideFactory { private static final String CURR_TYPE = "side"; - private static final String SIDE_DIR_TMPL = "%s%sside"; - public static AbsTableParser getSqlParser(String pluginType, String sqlRootDir, String cacheType) throws Exception { - cacheType = cacheType == null ? "async" : cacheType; - String sideDir = String.format(SIDE_DIR_TMPL, pluginType, cacheType); + String sideOperator = ECacheType.ALL.name().equals(cacheType) ? "all" : "async"; ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - String pluginJarPath = PluginUtil.getJarFileDirPath(sideDir, sqlRootDir); + String pluginJarPath = PluginUtil.getSideJarFileDirPath(pluginType, sideOperator, "side", sqlRootDir); DtClassLoader dtClassLoader = (DtClassLoader) classLoader; PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); diff --git a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java index 04e5fa31a..611cef37f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java @@ -63,6 +63,17 @@ public static String getJarFileDirPath(String type, String sqlRootDir){ return jarPath; } + public static String getSideJarFileDirPath(String pluginType, String sideOperator, String tableType, String sqlRootDir) throws MalformedURLException { + String dirName = sqlRootDir + SP + pluginType + sideOperator + tableType.toLowerCase(); + File jarFile = new File(dirName); + + if(!jarFile.exists()){ + throw new RuntimeException(String.format("path %s not exists!!!", dirName)); + } + + return dirName; + } + public static String getGenerClassName(String pluginTypeName, String type) throws IOException { String pluginClassName = upperCaseFirstChar(pluginTypeName) + upperCaseFirstChar(type); return CLASS_PRE_STR + "." + type.toLowerCase() + "." + pluginTypeName + "." + pluginClassName; From d9d094aa46f292ef6161644964fa4186749c0fdd Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 28 Sep 2018 20:02:28 +0800 Subject: [PATCH 038/470] fix can't find mysql side plugin bug --- .../com/dtstack/flink/sql/side/StreamSideFactory.java | 8 +++----- .../java/com/dtstack/flink/sql/util/PluginUtil.java | 11 +++++++++++ 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java b/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java index 7d8907301..4eb78b4c4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.side; import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.table.AbsSideTableParser; import com.dtstack.flink.sql.table.AbsTableParser; import com.dtstack.flink.sql.util.PluginUtil; @@ -36,14 +37,11 @@ public class StreamSideFactory { private static final String CURR_TYPE = "side"; - private static final String SIDE_DIR_TMPL = "%s%sside"; - public static AbsTableParser getSqlParser(String pluginType, String sqlRootDir, String cacheType) throws Exception { - cacheType = cacheType == null ? "async" : cacheType; - String sideDir = String.format(SIDE_DIR_TMPL, pluginType, cacheType); + String sideOperator = ECacheType.ALL.name().equals(cacheType) ? "all" : "async"; ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - String pluginJarPath = PluginUtil.getJarFileDirPath(sideDir, sqlRootDir); + String pluginJarPath = PluginUtil.getSideJarFileDirPath(pluginType, sideOperator, "side", sqlRootDir); DtClassLoader dtClassLoader = (DtClassLoader) classLoader; PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); diff --git a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java index 04e5fa31a..611cef37f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java @@ -63,6 +63,17 @@ public static String getJarFileDirPath(String type, String sqlRootDir){ return jarPath; } + public static String getSideJarFileDirPath(String pluginType, String sideOperator, String tableType, String sqlRootDir) throws MalformedURLException { + String dirName = sqlRootDir + SP + pluginType + sideOperator + tableType.toLowerCase(); + File jarFile = new File(dirName); + + if(!jarFile.exists()){ + throw new RuntimeException(String.format("path %s not exists!!!", dirName)); + } + + return dirName; + } + public static String getGenerClassName(String pluginTypeName, String type) throws IOException { String pluginClassName = upperCaseFirstChar(pluginTypeName) + upperCaseFirstChar(type); return CLASS_PRE_STR + "." + type.toLowerCase() + "." + pluginTypeName + "." + pluginClassName; From 81255d45ef447cb41b19888f03ce4d4e0323017e Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sun, 30 Sep 2018 10:36:15 +0800 Subject: [PATCH 039/470] fix for bug of hbase.sync.side plugin :java.lang.IllegalStateException: Detected both log4j-over-slf4j.jar AND slf4j-log4j12.jar on the class path, preempting StackOverflowError. See also http://www.slf4j.org/codes.html#log4jDelegationLoop for more details. --- hbase/hbase-side/hbase-async-side/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hbase/hbase-side/hbase-async-side/pom.xml b/hbase/hbase-side/hbase-async-side/pom.xml index 6b8640c3c..522d503fd 100644 --- a/hbase/hbase-side/hbase-async-side/pom.xml +++ b/hbase/hbase-side/hbase-async-side/pom.xml @@ -43,7 +43,7 @@ - + org.slf4j:slf4j-log4j12 From c639a5c877f13712373685f3d3cc4df6534331dd Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sun, 30 Sep 2018 10:44:29 +0800 Subject: [PATCH 040/470] edit readme item of confProp --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 9f3eef21f..d374097c9 100644 --- a/README.md +++ b/README.md @@ -87,7 +87,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * **confProp** * 描述:一些参数设置 * 格式: json - * 必选:否 + * 必选:是 (如无参数填写空json即可) * 默认值:无 * 可选参数: * sql.env.parallelism: 默认并行度设置 From 87d558973bf579cc9f3983a126f9e85b642818e2 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sun, 30 Sep 2018 11:00:23 +0800 Subject: [PATCH 041/470] fix for bug of hbase.sync.side plugin :java.lang.IllegalStateException: Detected both log4j-over-slf4j.jar AND slf4j-log4j12.jar on the class path, preempting StackOverflowError. See also http://www.slf4j.org/codes.html#log4jDelegationLoop for more details. --- hbase/hbase-side/hbase-async-side/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hbase/hbase-side/hbase-async-side/pom.xml b/hbase/hbase-side/hbase-async-side/pom.xml index 6b8640c3c..522d503fd 100644 --- a/hbase/hbase-side/hbase-async-side/pom.xml +++ b/hbase/hbase-side/hbase-async-side/pom.xml @@ -43,7 +43,7 @@ - + org.slf4j:slf4j-log4j12 From 06bc381d431922137d468ea8e38edbe16603d6b3 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sun, 30 Sep 2018 11:33:16 +0800 Subject: [PATCH 042/470] add mysql-side sql demo --- launcher/job/mysqlsideSql.txt | 52 +++++++++++++++++++++++++++++++++++ 1 file changed, 52 insertions(+) create mode 100644 launcher/job/mysqlsideSql.txt diff --git a/launcher/job/mysqlsideSql.txt b/launcher/job/mysqlsideSql.txt new file mode 100644 index 000000000..8dc9bf9ff --- /dev/null +++ b/launcher/job/mysqlsideSql.txt @@ -0,0 +1,52 @@ +CREATE TABLE MyTable( + channel STRING, + pv INT, + xctime bigint, + CHARACTER_LENGTH(channel) as timeLeng, + WATERMARK FOR xctime AS withOffset(xctime,1000) + )WITH( + type='kafka09', + bootstrapServers='172.16.8.198:9092', + offsetReset='latest', + topic='nbTest1' + ); +CREATE TABLE MyResult( + channel STRING, + pv INT + )WITH( + type='mysql', + url='jdbc:mysql://172.16.8.104:3306/test?charset=utf8', + userName='dtstack', + password='abc123', + tableName='pv' + ); + +create table sideTable( + channel String, + xccount int, + PRIMARY KEY(channel), + PERIOD FOR SYSTEM_TIME + )WITH( + type='mysql', + url='jdbc:mysql://172.16.8.104:3306/test?charset=utf8', + userName='dtstack', + password='abc123', + tableName='sidetest', + cache = 'LRU', + cacheTTLMs='10000' + ); + +insert +into + MyResult + select + a.channel, + b.xccount + from + MyTable a + join + sideTable b + on a.channel=b.channel + where + b.channel = 'xc' + and a.pv=10; \ No newline at end of file From 5c93bdb2468b33c2dddd2554e225e5a1cb12cb73 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Tue, 9 Oct 2018 14:57:06 +0800 Subject: [PATCH 043/470] Update README.md --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index ce3279f05..0e4932a6b 100644 --- a/README.md +++ b/README.md @@ -12,8 +12,8 @@ # 后续开发计划 * 增加全局缓存功能 * 增加临时表功能 - * 增加redis维表功能 - * 增加mongodb维表功能 + * 增加redis维表,结果表功能 + * 增加mongodb维表,结果表功能 * 增加oracle维表,结果表功能 * 增加SQlServer维表,结果表功能 * 增加kafka结果表功能 From 27603ea55ee2cdfb58f15a2d1bc9882d6fae1115 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 10 Oct 2018 15:23:16 +0800 Subject: [PATCH 044/470] add savepoint --- .../com/dtstack/flink/sql}/ClusterMode.java | 12 +- .../main/java/com/dtstack/flink/sql/Main.java | 6 +- .../sql/launcher/ClusterClientFactory.java | 26 ++--- .../flink/sql/launcher/LauncherMain.java | 22 ++-- .../sql/launcher/LauncherOptionParser.java | 102 +++++++++-------- .../flink/sql/launcher/LauncherOptions.java | 108 ++++++++++++++++-- 6 files changed, 187 insertions(+), 89 deletions(-) rename {launcher/src/main/java/com/dtstack/flink/sql/launcher => core/src/main/java/com/dtstack/flink/sql}/ClusterMode.java (79%) diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterMode.java b/core/src/main/java/com/dtstack/flink/sql/ClusterMode.java similarity index 79% rename from launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterMode.java rename to core/src/main/java/com/dtstack/flink/sql/ClusterMode.java index 590aba48e..a73730840 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterMode.java +++ b/core/src/main/java/com/dtstack/flink/sql/ClusterMode.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.dtstack.flink.sql.launcher; +package com.dtstack.flink.sql; /** * This class defines three running mode of FlinkX @@ -24,12 +24,14 @@ * Company: www.dtstack.com * @author huyifan.zju@163.com */ -public class ClusterMode { +public enum ClusterMode { - public static final String MODE_LOCAL = "local"; + local(0),standalone(1),yarn(2),yarnPer(3); - public static final String MODE_STANDALONE = "standalone"; + private int type; - public static final String MODE_YARN = "yarn"; + ClusterMode(int type){ + this.type = type; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index e4cd6efb9..41ee3ab43 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -87,8 +87,6 @@ public class Main { private static final Logger LOG = LoggerFactory.getLogger(Main.class); - private static final String LOCAL_MODE = "local"; - private static final int failureRate = 3; private static final int failureInterval = 6; //min @@ -134,7 +132,7 @@ public static void main(String[] args) throws Exception { Thread.currentThread().setContextClassLoader(dtClassLoader); URLClassLoader parentClassloader; - if(!LOCAL_MODE.equals(deployMode)){ + if(!ClusterMode.local.name().equals(deployMode)){ parentClassloader = (URLClassLoader) threadClassLoader.getParent(); }else{ parentClassloader = dtClassLoader; @@ -284,7 +282,7 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en } private static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws IOException { - StreamExecutionEnvironment env = !LOCAL_MODE.equals(deployMode) ? + StreamExecutionEnvironment env = !ClusterMode.local.name().equals(deployMode) ? StreamExecutionEnvironment.getExecutionEnvironment() : new MyLocalStreamEnvironment(); diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 960398906..06c759997 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -33,7 +33,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; - +import com.dtstack.flink.sql.ClusterMode; import java.io.File; import java.io.FilenameFilter; import java.lang.reflect.Field; @@ -42,10 +42,8 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.Set; -import static com.dtstack.flink.sql.launcher.LauncherOptions.*; /** * The Factory of ClusterClient @@ -55,18 +53,18 @@ */ public class ClusterClientFactory { - public static ClusterClient createClusterClient(Properties props) { - String clientType = props.getProperty(OPTION_MODE); - if(clientType.equals(ClusterMode.MODE_STANDALONE)) { - return createStandaloneClient(props); - } else if(clientType.equals(ClusterMode.MODE_YARN)) { - return createYarnClient(props); + public static ClusterClient createClusterClient(LauncherOptions launcherOptions) { + String mode = launcherOptions.getMode(); + if(mode.equals(ClusterMode.standalone.name())) { + return createStandaloneClient(launcherOptions); + } else if(mode.equals(ClusterMode.yarn.name())) { + return createYarnClient(launcherOptions); } throw new IllegalArgumentException("Unsupported cluster client type: "); } - public static StandaloneClusterClient createStandaloneClient(Properties props) { - String flinkConfDir = props.getProperty(LauncherOptions.OPTION_FLINK_CONF_DIR); + public static StandaloneClusterClient createStandaloneClient(LauncherOptions launcherOptions) { + String flinkConfDir = launcherOptions.getFlinkconf(); Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); StandaloneClusterDescriptor descriptor = new StandaloneClusterDescriptor(config); StandaloneClusterClient clusterClient = descriptor.retrieve(null); @@ -74,10 +72,10 @@ public static StandaloneClusterClient createStandaloneClient(Properties props) { return clusterClient; } - public static YarnClusterClient createYarnClient(Properties props) { - String flinkConfDir = props.getProperty(LauncherOptions.OPTION_FLINK_CONF_DIR); + public static YarnClusterClient createYarnClient(LauncherOptions launcherOptions) { + String flinkConfDir = launcherOptions.getFlinkconf(); Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); - String yarnConfDir = props.getProperty(LauncherOptions.OPTION_YARN_CONF_DIR); + String yarnConfDir =launcherOptions.getYarnconf(); org.apache.hadoop.conf.Configuration yarnConf = new YarnConfiguration(); if(StringUtils.isNotBlank(yarnConfDir)) { try { diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 3c4cdc57f..0bb25424c 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -22,15 +22,14 @@ import avro.shaded.com.google.common.collect.Lists; import com.dtstack.flink.sql.Main; +import org.apache.commons.lang.BooleanUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; - import java.io.File; import java.util.List; - -import static com.dtstack.flink.sql.launcher.ClusterMode.MODE_LOCAL; -import static com.dtstack.flink.sql.launcher.LauncherOptions.OPTION_LOCAL_SQL_PLUGIN_PATH; -import static com.dtstack.flink.sql.launcher.LauncherOptions.OPTION_MODE; +import com.dtstack.flink.sql.ClusterMode; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; /** * Date: 2017/2/20 @@ -51,18 +50,21 @@ private static String getLocalCoreJarPath(String localSqlRootJar){ public static void main(String[] args) throws Exception { LauncherOptionParser optionParser = new LauncherOptionParser(args); - String mode = (String) optionParser.getVal(OPTION_MODE); + LauncherOptions launcherOptions = optionParser.getLauncherOptions(); + String mode = launcherOptions.getMode(); List argList = optionParser.getProgramExeArgList(); - - if(mode.equals(MODE_LOCAL)) { + if(mode.equals(ClusterMode.local.name())) { String[] localArgs = argList.toArray(new String[argList.size()]); Main.main(localArgs); } else { - ClusterClient clusterClient = ClusterClientFactory.createClusterClient(optionParser.getProperties()); - String pluginRoot = (String) optionParser.getVal(OPTION_LOCAL_SQL_PLUGIN_PATH); + ClusterClient clusterClient = ClusterClientFactory.createClusterClient(launcherOptions); + String pluginRoot = launcherOptions.getLocalSqlPluginPath(); File jarFile = new File(getLocalCoreJarPath(pluginRoot)); String[] remoteArgs = argList.toArray(new String[argList.size()]); PackagedProgram program = new PackagedProgram(jarFile, Lists.newArrayList(), remoteArgs); + if(StringUtils.isNotBlank(launcherOptions.getSavePointPath())){ + program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getSavePointPath(), BooleanUtils.toBoolean(launcherOptions.getAllowNonRestoredState()))); + } clusterClient.run(program, 1); clusterClient.shutdown(); } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java index 50484dd02..ac83e55cd 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java @@ -19,23 +19,19 @@ package com.dtstack.flink.sql.launcher; import avro.shaded.com.google.common.collect.Lists; +import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.cli.BasicParser; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Options; import org.apache.commons.lang.StringUtils; import org.apache.flink.hadoop.shaded.com.google.common.base.Charsets; import org.apache.flink.hadoop.shaded.com.google.common.base.Preconditions; - import java.io.File; import java.io.FileInputStream; import java.net.URLEncoder; import java.util.List; import java.util.Map; -import java.util.Properties; - -import static com.dtstack.flink.sql.launcher.LauncherOptions.*; -import static com.dtstack.flink.sql.launcher.ClusterMode.*; - +import com.dtstack.flink.sql.ClusterMode; /** * The Parser of Launcher commandline options @@ -45,14 +41,36 @@ */ public class LauncherOptionParser { + public static final String OPTION_MODE = "mode"; + + public static final String OPTION_NAME = "name"; + + public static final String OPTION_SQL = "sql"; + + public static final String OPTION_FLINK_CONF_DIR = "flinkconf"; + + public static final String OPTION_YARN_CONF_DIR = "yarnconf"; + + public static final String OPTION_LOCAL_SQL_PLUGIN_PATH = "localSqlPluginPath"; + + public static final String OPTION_REMOTE_SQL_PLUGIN_PATH = "remoteSqlPluginPath"; + + public static final String OPTION_ADDJAR = "addjar"; + + public static final String OPTION_CONF_PROP = "confProp"; + + public static final String OPTION_SAVE_POINT_PATH = "savePointPath"; + + public static final String OPTION_ALLOW_NON_RESTORED_STATE = "allowNonRestoredState"; + private Options options = new Options(); private BasicParser parser = new BasicParser(); - private Properties properties = new Properties(); + private LauncherOptions properties = new LauncherOptions(); public LauncherOptionParser(String[] args) { - options.addOption(LauncherOptions.OPTION_MODE, true, "Running mode"); + options.addOption(OPTION_MODE, true, "Running mode"); options.addOption(OPTION_SQL, true, "Job sql file"); options.addOption(OPTION_NAME, true, "Job name"); options.addOption(OPTION_FLINK_CONF_DIR, true, "Flink configuration directory"); @@ -62,11 +80,14 @@ public LauncherOptionParser(String[] args) { options.addOption(OPTION_CONF_PROP, true, "sql ref prop,eg specify event time"); options.addOption(OPTION_YARN_CONF_DIR, true, "Yarn and hadoop configuration directory"); + options.addOption(OPTION_SAVE_POINT_PATH, true, "Savepoint restore path"); + options.addOption(OPTION_ALLOW_NON_RESTORED_STATE, true, "Flag indicating whether non restored state is allowed if the savepoint"); + try { CommandLine cl = parser.parse(options, args); - String mode = cl.getOptionValue(OPTION_MODE, MODE_LOCAL); + String mode = cl.getOptionValue(OPTION_MODE, ClusterMode.local.name()); //check mode - properties.put(OPTION_MODE, mode); + properties.setMode(mode); String job = Preconditions.checkNotNull(cl.getOptionValue(OPTION_SQL), "Must specify job file using option '" + OPTION_SQL + "'"); @@ -76,78 +97,65 @@ public LauncherOptionParser(String[] args) { in.read(filecontent); String content = new String(filecontent, "UTF-8"); String sql = URLEncoder.encode(content, Charsets.UTF_8.name()); - properties.put(OPTION_SQL, sql); - + properties.setSql(sql); String localPlugin = Preconditions.checkNotNull(cl.getOptionValue(OPTION_LOCAL_SQL_PLUGIN_PATH)); - properties.put(OPTION_LOCAL_SQL_PLUGIN_PATH, localPlugin); - + properties.setLocalSqlPluginPath(localPlugin); String remotePlugin = cl.getOptionValue(OPTION_REMOTE_SQL_PLUGIN_PATH); - if(!mode.equalsIgnoreCase(ClusterMode.MODE_LOCAL)){ + if(!ClusterMode.local.name().equals(mode)){ Preconditions.checkNotNull(remotePlugin); - properties.put(OPTION_REMOTE_SQL_PLUGIN_PATH, remotePlugin); + properties.setRemoteSqlPluginPath(remotePlugin); } - String name = Preconditions.checkNotNull(cl.getOptionValue(OPTION_NAME)); - properties.put(OPTION_NAME, name); - + properties.setName(name); String addJar = cl.getOptionValue(OPTION_ADDJAR); if(StringUtils.isNotBlank(addJar)){ - properties.put(OPTION_ADDJAR, addJar); + properties.setAddjar(addJar); } - String confProp = cl.getOptionValue(OPTION_CONF_PROP); if(StringUtils.isNotBlank(confProp)){ - properties.put(OPTION_CONF_PROP, confProp); + properties.setConfProp(confProp); } - String flinkConfDir = cl.getOptionValue(OPTION_FLINK_CONF_DIR); if(StringUtils.isNotBlank(flinkConfDir)) { - properties.put(OPTION_FLINK_CONF_DIR, flinkConfDir); + properties.setFlinkconf(flinkConfDir); } String yarnConfDir = cl.getOptionValue(OPTION_YARN_CONF_DIR); if(StringUtils.isNotBlank(yarnConfDir)) { - properties.put(OPTION_YARN_CONF_DIR, yarnConfDir); + properties.setYarnconf(yarnConfDir); + } + + String savePointPath = cl.getOptionValue(OPTION_SAVE_POINT_PATH); + if(StringUtils.isNotBlank(savePointPath)) { + properties.setSavePointPath(savePointPath); + } + + String allow_non = cl.getOptionValue(OPTION_ALLOW_NON_RESTORED_STATE); + if(StringUtils.isNotBlank(allow_non)) { + properties.setAllowNonRestoredState(allow_non); } } catch (Exception e) { throw new RuntimeException(e); } - } - public Properties getProperties(){ + public LauncherOptions getLauncherOptions(){ return properties; } - public Object getVal(String key){ - return properties.get(key); - } - - public List getAllArgList(){ + public List getProgramExeArgList() throws Exception { + Map mapConf = PluginUtil.ObjectToMap(properties); List args = Lists.newArrayList(); - for(Map.Entry one : properties.entrySet()){ - args.add("-" + one.getKey().toString()); - args.add(one.getValue().toString()); - } - - return args; - } - - public List getProgramExeArgList(){ - List args = Lists.newArrayList(); - for(Map.Entry one : properties.entrySet()){ - String key = one.getKey().toString(); + for(Map.Entry one : mapConf.entrySet()){ + String key = one.getKey(); if(OPTION_FLINK_CONF_DIR.equalsIgnoreCase(key) || OPTION_YARN_CONF_DIR.equalsIgnoreCase(key)){ continue; } - args.add("-" + key); args.add(one.getValue().toString()); } - return args; } - } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java index 2169bb698..ab4a276bb 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java @@ -26,23 +26,113 @@ */ public class LauncherOptions { - public static final String OPTION_MODE = "mode"; + private String mode; - public static final String OPTION_NAME = "name"; + private String name; - public static final String OPTION_SQL = "sql"; + private String sql; - public static final String OPTION_FLINK_CONF_DIR = "flinkconf"; + private String flinkconf; - public static final String OPTION_YARN_CONF_DIR = "yarnconf"; + private String yarnconf; - public static final String OPTION_LOCAL_SQL_PLUGIN_PATH = "localSqlPluginPath"; + private String localSqlPluginPath; - public static final String OPTION_REMOTE_SQL_PLUGIN_PATH = "remoteSqlPluginPath"; + private String remoteSqlPluginPath ; - public static final String OPTION_ADDJAR = "addjar"; + private String addjar; - public static final String OPTION_CONF_PROP = "confProp"; + private String confProp; + private String savePointPath; + private String allowNonRestoredState = "false"; + + public String getMode() { + return mode; + } + + public void setMode(String mode) { + this.mode = mode; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public String getSql() { + return sql; + } + + public void setSql(String sql) { + this.sql = sql; + } + + public String getFlinkconf() { + return flinkconf; + } + + public void setFlinkconf(String flinkconf) { + this.flinkconf = flinkconf; + } + + public String getYarnconf() { + return yarnconf; + } + + public void setYarnconf(String yarnconf) { + this.yarnconf = yarnconf; + } + + public String getLocalSqlPluginPath() { + return localSqlPluginPath; + } + + public void setLocalSqlPluginPath(String localSqlPluginPath) { + this.localSqlPluginPath = localSqlPluginPath; + } + + public String getRemoteSqlPluginPath() { + return remoteSqlPluginPath; + } + + public void setRemoteSqlPluginPath(String remoteSqlPluginPath) { + this.remoteSqlPluginPath = remoteSqlPluginPath; + } + + public String getAddjar() { + return addjar; + } + + public void setAddjar(String addjar) { + this.addjar = addjar; + } + + public String getConfProp() { + return confProp; + } + + public void setConfProp(String confProp) { + this.confProp = confProp; + } + + public String getSavePointPath() { + return savePointPath; + } + + public void setSavePointPath(String savePointPath) { + this.savePointPath = savePointPath; + } + + public String getAllowNonRestoredState() { + return allowNonRestoredState; + } + + public void setAllowNonRestoredState(String allowNonRestoredState) { + this.allowNonRestoredState = allowNonRestoredState; + } } From 6d64c2463f12e148d48418f8c625da00f30c83eb Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 10 Oct 2018 15:27:14 +0800 Subject: [PATCH 045/470] Update README.md --- README.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/README.md b/README.md index 0e4932a6b..966dd68d9 100644 --- a/README.md +++ b/README.md @@ -111,6 +111,16 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * 描述:Hadoop配置文件(包括hdfs和yarn)所在的目录(单机模式下不需要),如/hadoop/etc/hadoop * 必选:否 * 默认值:无 + +* **savePointPath** + * 描述:任务恢复点的路径 + * 必选:否 + * 默认值:无 + +* **allowNonRestoredState** + * 描述:指示保存点是否允许非还原状态的标志 + * 必选:否 + * 默认值:false ## 2 结构 ### 2.1 源表插件 From 36706de47025a98747dc945c32d73c36b6deef75 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 10 Oct 2018 15:56:41 +0800 Subject: [PATCH 046/470] add savepoint --- .../com/dtstack/flink/sql}/ClusterMode.java | 19 ++- .../sql/launcher/ClusterClientFactory.java | 30 ++--- .../flink/sql/launcher/LauncherMain.java | 23 ++-- .../sql/launcher/LauncherOptionParser.java | 100 +++++++++------- .../flink/sql/launcher/LauncherOptions.java | 110 ++++++++++++++++-- 5 files changed, 189 insertions(+), 93 deletions(-) rename {launcher/src/main/java/com/dtstack/flink/sql/launcher => core/src/main/java/com/dtstack/flink/sql}/ClusterMode.java (70%) diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterMode.java b/core/src/main/java/com/dtstack/flink/sql/ClusterMode.java similarity index 70% rename from launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterMode.java rename to core/src/main/java/com/dtstack/flink/sql/ClusterMode.java index 590aba48e..024a31854 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterMode.java +++ b/core/src/main/java/com/dtstack/flink/sql/ClusterMode.java @@ -1,3 +1,4 @@ + /** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -16,20 +17,18 @@ * limitations under the License. */ -package com.dtstack.flink.sql.launcher; +package com.dtstack.flink.sql; /** - * This class defines three running mode of FlinkX - * - * Company: www.dtstack.com - * @author huyifan.zju@163.com + * Created by sishu.yss on 2018/10/10. */ -public class ClusterMode { - - public static final String MODE_LOCAL = "local"; +public enum ClusterMode { - public static final String MODE_STANDALONE = "standalone"; + local(0),standalone(1),yarn(2),yarnPer(3); - public static final String MODE_YARN = "yarn"; + private int type; + ClusterMode(int type){ + this.type = type; + } } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 1b7a02371..15458b883 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -22,33 +22,27 @@ import org.apache.flink.client.deployment.ClusterRetrieveException; import org.apache.flink.client.deployment.StandaloneClusterDescriptor; import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.client.program.StandaloneClusterClient; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; -import org.apache.flink.yarn.YarnClusterClient; import org.apache.flink.yarn.YarnClusterDescriptor; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; - import java.io.File; import java.io.FilenameFilter; -import java.lang.reflect.Field; import java.util.EnumSet; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Properties; import java.util.Set; - -import static com.dtstack.flink.sql.launcher.LauncherOptions.*; +import com.dtstack.flink.sql.ClusterMode; /** * The Factory of ClusterClient @@ -58,18 +52,18 @@ */ public class ClusterClientFactory { - public static ClusterClient createClusterClient(Properties props) throws ClusterRetrieveException { - String clientType = props.getProperty(OPTION_MODE); - if(clientType.equals(ClusterMode.MODE_STANDALONE)) { - return createStandaloneClient(props); - } else if(clientType.equals(ClusterMode.MODE_YARN)) { - return createYarnClient(props); + public static ClusterClient createClusterClient(LauncherOptions launcherOptions) throws ClusterRetrieveException { + String mode = launcherOptions.getMode(); + if(mode.equals(ClusterMode.standalone.name())) { + return createStandaloneClient(launcherOptions); + } else if(mode.equals(ClusterMode.yarn.name())) { + return createYarnClient(launcherOptions); } throw new IllegalArgumentException("Unsupported cluster client type: "); } - public static RestClusterClient createStandaloneClient(Properties props) throws ClusterRetrieveException { - String flinkConfDir = props.getProperty(LauncherOptions.OPTION_FLINK_CONF_DIR); + public static RestClusterClient createStandaloneClient(LauncherOptions launcherOptions) throws ClusterRetrieveException { + String flinkConfDir = launcherOptions.getFlinkconf(); Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); StandaloneClusterDescriptor descriptor = new StandaloneClusterDescriptor(config); RestClusterClient clusterClient = descriptor.retrieve(null); @@ -77,10 +71,10 @@ public static RestClusterClient createStandaloneClient(Properties props) throws return clusterClient; } - public static ClusterClient createYarnClient(Properties props) { - String flinkConfDir = props.getProperty(LauncherOptions.OPTION_FLINK_CONF_DIR); + public static ClusterClient createYarnClient(LauncherOptions launcherOptions) { + String flinkConfDir = launcherOptions.getFlinkconf(); Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); - String yarnConfDir = props.getProperty(LauncherOptions.OPTION_YARN_CONF_DIR); + String yarnConfDir = launcherOptions.getYarnconf(); YarnConfiguration yarnConf = new YarnConfiguration(); if(StringUtils.isNotBlank(yarnConfDir)) { try { diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 3c4cdc57f..55f085c98 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -24,13 +24,12 @@ import com.dtstack.flink.sql.Main; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; - import java.io.File; import java.util.List; - -import static com.dtstack.flink.sql.launcher.ClusterMode.MODE_LOCAL; -import static com.dtstack.flink.sql.launcher.LauncherOptions.OPTION_LOCAL_SQL_PLUGIN_PATH; -import static com.dtstack.flink.sql.launcher.LauncherOptions.OPTION_MODE; +import com.dtstack.flink.sql.ClusterMode; +import org.apache.flink.table.shaded.org.apache.commons.lang.StringUtils; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.table.shaded.org.apache.commons.lang.BooleanUtils; /** * Date: 2017/2/20 @@ -39,7 +38,6 @@ */ public class LauncherMain { - private static final String CORE_JAR = "core.jar"; private static String SP = File.separator; @@ -51,18 +49,21 @@ private static String getLocalCoreJarPath(String localSqlRootJar){ public static void main(String[] args) throws Exception { LauncherOptionParser optionParser = new LauncherOptionParser(args); - String mode = (String) optionParser.getVal(OPTION_MODE); + LauncherOptions launcherOptions = optionParser.getLauncherOptions(); + String mode = launcherOptions.getMode(); List argList = optionParser.getProgramExeArgList(); - - if(mode.equals(MODE_LOCAL)) { + if(mode.equals(ClusterMode.local.name())) { String[] localArgs = argList.toArray(new String[argList.size()]); Main.main(localArgs); } else { - ClusterClient clusterClient = ClusterClientFactory.createClusterClient(optionParser.getProperties()); - String pluginRoot = (String) optionParser.getVal(OPTION_LOCAL_SQL_PLUGIN_PATH); + ClusterClient clusterClient = ClusterClientFactory.createClusterClient(launcherOptions); + String pluginRoot = launcherOptions.getLocalSqlPluginPath(); File jarFile = new File(getLocalCoreJarPath(pluginRoot)); String[] remoteArgs = argList.toArray(new String[argList.size()]); PackagedProgram program = new PackagedProgram(jarFile, Lists.newArrayList(), remoteArgs); + if(StringUtils.isNotBlank(launcherOptions.getSavePointPath())){ + program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getSavePointPath(), BooleanUtils.toBoolean(launcherOptions.getAllowNonRestoredState()))); + } clusterClient.run(program, 1); clusterClient.shutdown(); } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java index 50484dd02..99ea41489 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java @@ -25,16 +25,14 @@ import org.apache.commons.lang.StringUtils; import org.apache.flink.hadoop.shaded.com.google.common.base.Charsets; import org.apache.flink.hadoop.shaded.com.google.common.base.Preconditions; - +import com.dtstack.flink.sql.util.PluginUtil; import java.io.File; import java.io.FileInputStream; import java.net.URLEncoder; import java.util.List; import java.util.Map; -import java.util.Properties; +import com.dtstack.flink.sql.ClusterMode; -import static com.dtstack.flink.sql.launcher.LauncherOptions.*; -import static com.dtstack.flink.sql.launcher.ClusterMode.*; /** @@ -45,14 +43,36 @@ */ public class LauncherOptionParser { + public static final String OPTION_MODE = "mode"; + + public static final String OPTION_NAME = "name"; + + public static final String OPTION_SQL = "sql"; + + public static final String OPTION_FLINK_CONF_DIR = "flinkconf"; + + public static final String OPTION_YARN_CONF_DIR = "yarnconf"; + + public static final String OPTION_LOCAL_SQL_PLUGIN_PATH = "localSqlPluginPath"; + + public static final String OPTION_REMOTE_SQL_PLUGIN_PATH = "remoteSqlPluginPath"; + + public static final String OPTION_ADDJAR = "addjar"; + + public static final String OPTION_CONF_PROP = "confProp"; + + public static final String OPTION_SAVE_POINT_PATH = "savePointPath"; + + public static final String OPTION_ALLOW_NON_RESTORED_STATE = "allowNonRestoredState"; + private Options options = new Options(); private BasicParser parser = new BasicParser(); - private Properties properties = new Properties(); + private LauncherOptions properties = new LauncherOptions(); public LauncherOptionParser(String[] args) { - options.addOption(LauncherOptions.OPTION_MODE, true, "Running mode"); + options.addOption(OPTION_MODE, true, "Running mode"); options.addOption(OPTION_SQL, true, "Job sql file"); options.addOption(OPTION_NAME, true, "Job name"); options.addOption(OPTION_FLINK_CONF_DIR, true, "Flink configuration directory"); @@ -62,11 +82,14 @@ public LauncherOptionParser(String[] args) { options.addOption(OPTION_CONF_PROP, true, "sql ref prop,eg specify event time"); options.addOption(OPTION_YARN_CONF_DIR, true, "Yarn and hadoop configuration directory"); + options.addOption(OPTION_SAVE_POINT_PATH, true, "Savepoint restore path"); + options.addOption(OPTION_ALLOW_NON_RESTORED_STATE, true, "Flag indicating whether non restored state is allowed if the savepoint"); + try { CommandLine cl = parser.parse(options, args); - String mode = cl.getOptionValue(OPTION_MODE, MODE_LOCAL); + String mode = cl.getOptionValue(OPTION_MODE, ClusterMode.local.name()); //check mode - properties.put(OPTION_MODE, mode); + properties.setMode(mode); String job = Preconditions.checkNotNull(cl.getOptionValue(OPTION_SQL), "Must specify job file using option '" + OPTION_SQL + "'"); @@ -76,78 +99,65 @@ public LauncherOptionParser(String[] args) { in.read(filecontent); String content = new String(filecontent, "UTF-8"); String sql = URLEncoder.encode(content, Charsets.UTF_8.name()); - properties.put(OPTION_SQL, sql); - + properties.setSql(sql); String localPlugin = Preconditions.checkNotNull(cl.getOptionValue(OPTION_LOCAL_SQL_PLUGIN_PATH)); - properties.put(OPTION_LOCAL_SQL_PLUGIN_PATH, localPlugin); - + properties.setLocalSqlPluginPath(localPlugin); String remotePlugin = cl.getOptionValue(OPTION_REMOTE_SQL_PLUGIN_PATH); - if(!mode.equalsIgnoreCase(ClusterMode.MODE_LOCAL)){ + if(!ClusterMode.local.name().equals(mode)){ Preconditions.checkNotNull(remotePlugin); - properties.put(OPTION_REMOTE_SQL_PLUGIN_PATH, remotePlugin); + properties.setRemoteSqlPluginPath(remotePlugin); } - String name = Preconditions.checkNotNull(cl.getOptionValue(OPTION_NAME)); - properties.put(OPTION_NAME, name); - + properties.setName(name); String addJar = cl.getOptionValue(OPTION_ADDJAR); if(StringUtils.isNotBlank(addJar)){ - properties.put(OPTION_ADDJAR, addJar); + properties.setAddjar(addJar); } - String confProp = cl.getOptionValue(OPTION_CONF_PROP); if(StringUtils.isNotBlank(confProp)){ - properties.put(OPTION_CONF_PROP, confProp); + properties.setConfProp(confProp); } - String flinkConfDir = cl.getOptionValue(OPTION_FLINK_CONF_DIR); if(StringUtils.isNotBlank(flinkConfDir)) { - properties.put(OPTION_FLINK_CONF_DIR, flinkConfDir); + properties.setFlinkconf(flinkConfDir); } String yarnConfDir = cl.getOptionValue(OPTION_YARN_CONF_DIR); if(StringUtils.isNotBlank(yarnConfDir)) { - properties.put(OPTION_YARN_CONF_DIR, yarnConfDir); + properties.setYarnconf(yarnConfDir); + } + + String savePointPath = cl.getOptionValue(OPTION_SAVE_POINT_PATH); + if(StringUtils.isNotBlank(savePointPath)) { + properties.setSavePointPath(savePointPath); + } + + String allow_non = cl.getOptionValue(OPTION_ALLOW_NON_RESTORED_STATE); + if(StringUtils.isNotBlank(allow_non)) { + properties.setAllowNonRestoredState(allow_non); } } catch (Exception e) { throw new RuntimeException(e); } - } - public Properties getProperties(){ + public LauncherOptions getLauncherOptions(){ return properties; } - public Object getVal(String key){ - return properties.get(key); - } - - public List getAllArgList(){ - List args = Lists.newArrayList(); - for(Map.Entry one : properties.entrySet()){ - args.add("-" + one.getKey().toString()); - args.add(one.getValue().toString()); - } - - return args; - } - - public List getProgramExeArgList(){ + public List getProgramExeArgList() throws Exception { + Map mapConf = PluginUtil.ObjectToMap(properties); List args = Lists.newArrayList(); - for(Map.Entry one : properties.entrySet()){ - String key = one.getKey().toString(); + for(Map.Entry one : mapConf.entrySet()){ + String key = one.getKey(); if(OPTION_FLINK_CONF_DIR.equalsIgnoreCase(key) || OPTION_YARN_CONF_DIR.equalsIgnoreCase(key)){ continue; } - args.add("-" + key); args.add(one.getValue().toString()); } - return args; } - } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java index 2169bb698..f183957f7 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java @@ -26,23 +26,115 @@ */ public class LauncherOptions { - public static final String OPTION_MODE = "mode"; + private String mode; - public static final String OPTION_NAME = "name"; + private String name; - public static final String OPTION_SQL = "sql"; + private String sql; - public static final String OPTION_FLINK_CONF_DIR = "flinkconf"; + private String flinkconf; - public static final String OPTION_YARN_CONF_DIR = "yarnconf"; + private String yarnconf; - public static final String OPTION_LOCAL_SQL_PLUGIN_PATH = "localSqlPluginPath"; + private String localSqlPluginPath; - public static final String OPTION_REMOTE_SQL_PLUGIN_PATH = "remoteSqlPluginPath"; + private String remoteSqlPluginPath ; - public static final String OPTION_ADDJAR = "addjar"; + private String addjar; - public static final String OPTION_CONF_PROP = "confProp"; + private String confProp; + + private String savePointPath; + + private String allowNonRestoredState = "false"; + + public String getMode() { + return mode; + } + + public void setMode(String mode) { + this.mode = mode; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public String getSql() { + return sql; + } + + public void setSql(String sql) { + this.sql = sql; + } + + public String getFlinkconf() { + return flinkconf; + } + + public void setFlinkconf(String flinkconf) { + this.flinkconf = flinkconf; + } + + public String getYarnconf() { + return yarnconf; + } + + public void setYarnconf(String yarnconf) { + this.yarnconf = yarnconf; + } + + public String getLocalSqlPluginPath() { + return localSqlPluginPath; + } + + public void setLocalSqlPluginPath(String localSqlPluginPath) { + this.localSqlPluginPath = localSqlPluginPath; + } + + public String getRemoteSqlPluginPath() { + return remoteSqlPluginPath; + } + + public void setRemoteSqlPluginPath(String remoteSqlPluginPath) { + this.remoteSqlPluginPath = remoteSqlPluginPath; + } + + public String getAddjar() { + return addjar; + } + + public void setAddjar(String addjar) { + this.addjar = addjar; + } + + public String getConfProp() { + return confProp; + } + + public void setConfProp(String confProp) { + this.confProp = confProp; + } + + public String getSavePointPath() { + return savePointPath; + } + + public void setSavePointPath(String savePointPath) { + this.savePointPath = savePointPath; + } + + public String getAllowNonRestoredState() { + return allowNonRestoredState; + } + + public void setAllowNonRestoredState(String allowNonRestoredState) { + this.allowNonRestoredState = allowNonRestoredState; + } } From 81b9f4b32293d6e4c026b9d2a5b1f209467fe9d4 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 10 Oct 2018 16:04:06 +0800 Subject: [PATCH 047/470] Update README.md --- README.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/README.md b/README.md index 0e4932a6b..a519a8ae7 100644 --- a/README.md +++ b/README.md @@ -111,6 +111,16 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * 描述:Hadoop配置文件(包括hdfs和yarn)所在的目录(单机模式下不需要),如/hadoop/etc/hadoop * 必选:否 * 默认值:无 + +* **savePointPath** + * 描述:任务恢复点的路径 + * 必选:否 + * 默认值:无 + +* **allowNonRestoredState** + * 描述:指示保存点是否允许非还原状态的标志 + * 必选:否 + * 默认值:false ## 2 结构 ### 2.1 源表插件 From 90b9890f560061024c95d8e55878ada740e33d4b Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 10 Oct 2018 16:10:30 +0800 Subject: [PATCH 048/470] Update README.md --- README.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/README.md b/README.md index 0e4932a6b..a519a8ae7 100644 --- a/README.md +++ b/README.md @@ -111,6 +111,16 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * 描述:Hadoop配置文件(包括hdfs和yarn)所在的目录(单机模式下不需要),如/hadoop/etc/hadoop * 必选:否 * 默认值:无 + +* **savePointPath** + * 描述:任务恢复点的路径 + * 必选:否 + * 默认值:无 + +* **allowNonRestoredState** + * 描述:指示保存点是否允许非还原状态的标志 + * 必选:否 + * 默认值:false ## 2 结构 ### 2.1 源表插件 From b35c83fbfeba66afd8eae85250bd736d9f33c62c Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 10 Oct 2018 20:00:26 +0800 Subject: [PATCH 049/470] fix mysql async side fill data exception(IndexOutOfBoundsException) --- core/src/main/java/com/dtstack/flink/sql/Main.java | 3 +++ .../dtstack/flink/sql/launcher/LauncherOptionParser.java | 6 ++++++ .../com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java | 2 +- 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index cc06f2b60..a8eca7333 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -105,6 +105,9 @@ public static void main(String[] args) throws Exception { options.addOption("confProp", true, "env properties"); options.addOption("mode", true, "deploy mode"); + options.addOption("savePointPath", true, "Savepoint restore path"); + options.addOption("allowNonRestoredState", true, "Flag indicating whether non restored state is allowed if the savepoint"); + CommandLineParser parser = new DefaultParser(); CommandLine cl = parser.parse(options, args); String sql = cl.getOptionValue("sql"); diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java index ac83e55cd..b4f244349 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java @@ -105,6 +105,7 @@ public LauncherOptionParser(String[] args) { Preconditions.checkNotNull(remotePlugin); properties.setRemoteSqlPluginPath(remotePlugin); } + String name = Preconditions.checkNotNull(cl.getOptionValue(OPTION_NAME)); properties.setName(name); String addJar = cl.getOptionValue(OPTION_ADDJAR); @@ -153,6 +154,11 @@ public List getProgramExeArgList() throws Exception { || OPTION_YARN_CONF_DIR.equalsIgnoreCase(key)){ continue; } + + if(one.getValue() == null){ + continue; + } + args.add("-" + key); args.add(one.getValue().toString()); } diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java index 979f43a7b..9dca70eaf 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -191,7 +191,7 @@ public Row fillData(Row input, Object line){ row.setField(entry.getKey(), obj); } - for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ + for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ if(jsonArray == null){ row.setField(entry.getKey(), null); }else{ From a29c06ca7ee095eb1409c9804dca91dca4d5777d Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 10 Oct 2018 20:21:25 +0800 Subject: [PATCH 050/470] fix mysql async side fill data exception(IndexOutOfBoundsException) --- core/src/main/java/com/dtstack/flink/sql/Main.java | 3 +++ .../com/dtstack/flink/sql/launcher/LauncherOptionParser.java | 5 +++++ .../com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java | 2 +- 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 4e85c61b5..319e88a97 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -109,6 +109,9 @@ public static void main(String[] args) throws Exception { options.addOption("confProp", true, "env properties"); options.addOption("mode", true, "deploy mode"); + options.addOption("savePointPath", true, "Savepoint restore path"); + options.addOption("allowNonRestoredState", true, "Flag indicating whether non restored state is allowed if the savepoint"); + CommandLineParser parser = new DefaultParser(); CommandLine cl = parser.parse(options, args); String sql = cl.getOptionValue("sql"); diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java index 99ea41489..74589dcc9 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java @@ -155,6 +155,11 @@ public List getProgramExeArgList() throws Exception { || OPTION_YARN_CONF_DIR.equalsIgnoreCase(key)){ continue; } + + if(one.getValue() == null){ + continue; + } + args.add("-" + key); args.add(one.getValue().toString()); } diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java index 979f43a7b..9dca70eaf 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -191,7 +191,7 @@ public Row fillData(Row input, Object line){ row.setField(entry.getKey(), obj); } - for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ + for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ if(jsonArray == null){ row.setField(entry.getKey(), null); }else{ From fe7b3461626b9260ee06d10c5e66d47230c16702 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 11 Oct 2018 09:17:37 +0800 Subject: [PATCH 051/470] ysq --- .gitignore | 1 + .../com/dtstack/flink/sql/launcher/LauncherOptionParser.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/.gitignore b/.gitignore index c88786513..d93fad7de 100644 --- a/.gitignore +++ b/.gitignore @@ -9,3 +9,4 @@ target/ *.iml plugins/ lib/ +.vertx/ diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java index b4f244349..a15f42ded 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java @@ -37,7 +37,7 @@ * The Parser of Launcher commandline options * * Company: www.dtstack.com - * @author huyifan.zju@163.com + * @author sishu.yss */ public class LauncherOptionParser { From 41dfa6a4b165e558fe4c9756ba15e95920955d64 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 11 Oct 2018 09:19:07 +0800 Subject: [PATCH 052/470] Update .gitignore --- launcher/.gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/launcher/.gitignore b/launcher/.gitignore index a4f1c3e53..cecb836d0 100644 --- a/launcher/.gitignore +++ b/launcher/.gitignore @@ -10,3 +10,4 @@ target/ plugins/ lib/ dependency-reduced-pom.xml +.vertx/ From 81bb46070d19393f38b01c8851cdebd07d87fc6e Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 11 Oct 2018 09:19:41 +0800 Subject: [PATCH 053/470] Update LauncherOptionParser.java --- .../com/dtstack/flink/sql/launcher/LauncherOptionParser.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java index 74589dcc9..5052d22e8 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java @@ -39,7 +39,7 @@ * The Parser of Launcher commandline options * * Company: www.dtstack.com - * @author huyifan.zju@163.com + * @author sishu.yss */ public class LauncherOptionParser { From 766d74d3e49aa0a3bd3d1e7033b9e75085b841d8 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 11 Oct 2018 17:40:45 +0800 Subject: [PATCH 054/470] Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. --- .../com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java | 6 +++++- .../com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java | 6 +++++- .../com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java | 7 ++++++- 3 files changed, 16 insertions(+), 3 deletions(-) diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java index ae4be351c..75c65b73b 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java @@ -36,6 +36,7 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.hbase.async.HBaseClient; import org.slf4j.Logger; @@ -159,9 +160,12 @@ protected Row fillData(Row input, Object sideInput){ Row row = new Row(sideInfo.getOutFieldInfoList().size()); for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ Object obj = input.getField(entry.getValue()); - if(obj instanceof Timestamp){ + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ obj = ((Timestamp)obj).getTime(); } + row.setField(entry.getKey(), obj); } diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java index 78a32ce32..ed7f611d5 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java @@ -14,6 +14,7 @@ import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.apache.flink.util.Collector; import org.slf4j.Logger; @@ -66,7 +67,10 @@ protected Row fillData(Row input, Object sideInput) { Row row = new Row(sideInfo.getOutFieldInfoList().size()); for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ Object obj = input.getField(entry.getValue()); - if(obj instanceof Timestamp){ + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. + if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ obj = ((Timestamp)obj).getTime(); } row.setField(entry.getKey(), obj); diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java index 9dca70eaf..f72df5dea 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -35,10 +35,12 @@ import io.vertx.ext.jdbc.JDBCClient; import io.vertx.ext.sql.SQLClient; import io.vertx.ext.sql.SQLConnection; +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -185,9 +187,12 @@ public Row fillData(Row input, Object line){ Row row = new Row(sideInfo.getOutFieldInfoList().size()); for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ Object obj = input.getField(entry.getValue()); - if(obj instanceof Timestamp){ + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ obj = ((Timestamp)obj).getTime(); } + row.setField(entry.getKey(), obj); } From 09eccdda76d09d05266127c586a7ebdf03a5c9ab Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 11 Oct 2018 17:52:46 +0800 Subject: [PATCH 055/470] Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. --- .../flink/sql/side/hbase/HbaseAsyncReqRow.java | 6 +++++- .../flink/sql/side/mysql/MysqlAllReqRow.java | 14 +++++--------- .../flink/sql/side/mysql/MysqlAsyncReqRow.java | 6 +++++- 3 files changed, 15 insertions(+), 11 deletions(-) diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java index ae4be351c..093af463a 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java @@ -36,6 +36,7 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.hbase.async.HBaseClient; import org.slf4j.Logger; @@ -159,7 +160,10 @@ protected Row fillData(Row input, Object sideInput){ Row row = new Row(sideInfo.getOutFieldInfoList().size()); for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ Object obj = input.getField(entry.getValue()); - if(obj instanceof Timestamp){ + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. + if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ obj = ((Timestamp)obj).getTime(); } row.setField(entry.getKey(), obj); diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java index 78a32ce32..078ee589c 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java @@ -3,17 +3,15 @@ import com.dtstack.flink.sql.side.AllReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; -import com.dtstack.flink.sql.threadFactory.DTThreadFactory; import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.calcite.sql.JoinType; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.apache.flink.util.Collector; import org.slf4j.Logger; @@ -28,10 +26,6 @@ import java.util.Calendar; import java.util.List; import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; /** @@ -59,14 +53,16 @@ public MysqlAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List cacheInfo = (Map) sideInput; Row row = new Row(sideInfo.getOutFieldInfoList().size()); for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ Object obj = input.getField(entry.getValue()); - if(obj instanceof Timestamp){ + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. + if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ obj = ((Timestamp)obj).getTime(); } row.setField(entry.getKey(), obj); diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java index 9dca70eaf..7b288531e 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -39,6 +39,7 @@ import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -185,7 +186,10 @@ public Row fillData(Row input, Object line){ Row row = new Row(sideInfo.getOutFieldInfoList().size()); for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ Object obj = input.getField(entry.getValue()); - if(obj instanceof Timestamp){ + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. + if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ obj = ((Timestamp)obj).getTime(); } row.setField(entry.getKey(), obj); From 63f8a885046d0e2b1f3fc5656cf6005cd169b91d Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Tue, 16 Oct 2018 17:06:07 +0800 Subject: [PATCH 056/470] standalone submit --- .../dtstack/flink/sql/launcher/ClusterClientFactory.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 15458b883..2e523eac9 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -22,6 +22,7 @@ import org.apache.flink.client.deployment.ClusterRetrieveException; import org.apache.flink.client.deployment.StandaloneClusterDescriptor; import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.StandaloneClusterClient; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -52,7 +53,7 @@ */ public class ClusterClientFactory { - public static ClusterClient createClusterClient(LauncherOptions launcherOptions) throws ClusterRetrieveException { + public static ClusterClient createClusterClient(LauncherOptions launcherOptions) throws Exception { String mode = launcherOptions.getMode(); if(mode.equals(ClusterMode.standalone.name())) { return createStandaloneClient(launcherOptions); @@ -62,11 +63,10 @@ public static ClusterClient createClusterClient(LauncherOptions launcherOptions) throw new IllegalArgumentException("Unsupported cluster client type: "); } - public static RestClusterClient createStandaloneClient(LauncherOptions launcherOptions) throws ClusterRetrieveException { + public static ClusterClient createStandaloneClient(LauncherOptions launcherOptions) throws Exception { String flinkConfDir = launcherOptions.getFlinkconf(); Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); - StandaloneClusterDescriptor descriptor = new StandaloneClusterDescriptor(config); - RestClusterClient clusterClient = descriptor.retrieve(null); + StandaloneClusterClient clusterClient = new StandaloneClusterClient(config); clusterClient.setDetached(true); return clusterClient; } From ca52e90bc238fa8050835e4a6d486cb5b13e8a71 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Tue, 16 Oct 2018 17:20:11 +0800 Subject: [PATCH 057/470] bugfix --- .../dtstack/flink/sql/launcher/ClusterClientFactory.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 2e523eac9..aa8b5db5f 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -27,7 +27,10 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -37,6 +40,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import java.io.File; import java.io.FilenameFilter; +import java.net.InetSocketAddress; import java.util.EnumSet; import java.util.HashSet; import java.util.Iterator; @@ -67,6 +71,10 @@ public static ClusterClient createStandaloneClient(LauncherOptions launcherOptio String flinkConfDir = launcherOptions.getFlinkconf(); Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); StandaloneClusterClient clusterClient = new StandaloneClusterClient(config); + LeaderConnectionInfo connectionInfo = clusterClient.getClusterConnectionInfo(); + InetSocketAddress address = AkkaUtils.getInetSocketAddressFromAkkaURL(connectionInfo.getAddress()); + config.setString(JobManagerOptions.ADDRESS, address.getAddress().getHostName()); + config.setInteger(JobManagerOptions.PORT, address.getPort()); clusterClient.setDetached(true); return clusterClient; } From d68ffd9dce90250ac39197cc42f2bdbeda866f0f Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 18 Oct 2018 13:55:42 +0800 Subject: [PATCH 058/470] update comment --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index a519a8ae7..719f0ca38 100644 --- a/README.md +++ b/README.md @@ -52,7 +52,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack #### 1.4.2 命令行参数选项 -* **model** +* **mode** * 描述:执行模式,也就是flink集群的工作模式 * local: 本地模式 * standalone: 独立部署模式的flink集群 From 5f3d5ac365d09fea64d0e105d49d4eb1f0c4ecef Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 18 Oct 2018 16:12:18 +0800 Subject: [PATCH 059/470] =?UTF-8?q?=E5=88=9D=E6=AD=A5=E6=B7=BB=E5=8A=A0kaf?= =?UTF-8?q?ka=E8=BE=93=E5=85=A5bps,rps,tps,=20=E8=84=8F=E6=95=B0=E6=8D=AE?= =?UTF-8?q?=EF=BC=8C=E4=B8=9A=E5=8A=A1=E5=BB=B6=E8=BF=9F=20metric.?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/Main.java | 3 +- .../flink/sql/metric/EventDelayGauge.java | 24 ++++++ .../com/dtstack/flink/sql/util/MathUtil.java | 10 ++- .../watermarker/AbsCustomerWaterMarker.java | 74 +++++++++++++++++++ .../CustomerWaterMarkerForLong.java | 3 +- .../CustomerWaterMarkerForTimeStamp.java | 11 ++- .../sql/watermarker/WaterMarkerAssigner.java | 14 +++- .../kafka/CustomerJsonDeserialization.java | 72 +++++++++++++++++- .../source/kafka/CustomerKafka09Consumer.java | 30 ++++++++ .../flink/sql/source/kafka/KafkaSource.java | 8 +- 10 files changed, 235 insertions(+), 14 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/metric/EventDelayGauge.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/watermarker/AbsCustomerWaterMarker.java create mode 100644 kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 319e88a97..46f1aebc5 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -58,6 +58,7 @@ import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -260,7 +261,7 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en String fields = String.join(",", typeInfo.getFieldNames()); if(waterMarkerAssigner.checkNeedAssignWaterMarker(sourceTableInfo)){ - adaptStream = waterMarkerAssigner.assignWaterMarker(adaptStream, typeInfo, sourceTableInfo.getEventTimeField(), sourceTableInfo.getMaxOutOrderness()); + adaptStream = waterMarkerAssigner.assignWaterMarker(adaptStream, typeInfo, sourceTableInfo); fields += ",ROWTIME.ROWTIME"; }else{ fields += ",PROCTIME.PROCTIME"; diff --git a/core/src/main/java/com/dtstack/flink/sql/metric/EventDelayGauge.java b/core/src/main/java/com/dtstack/flink/sql/metric/EventDelayGauge.java new file mode 100644 index 000000000..3e3d558ec --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/metric/EventDelayGauge.java @@ -0,0 +1,24 @@ +package com.dtstack.flink.sql.metric; + +import org.apache.flink.metrics.Gauge; + +/** + * 数据延迟时间 单位 s + * Date: 2018/10/18 + * Company: www.dtstack.com + * @author xuchao + */ + +public class EventDelayGauge implements Gauge { + + private volatile int delayTime = 0; + + public void setDelayTime(int delayTime) { + this.delayTime = delayTime; + } + + @Override + public Integer getValue() { + return delayTime; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/util/MathUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/MathUtil.java index 829cea461..37b06c8b8 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/MathUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/MathUtil.java @@ -20,6 +20,8 @@ package com.dtstack.flink.sql.util; +import java.math.BigDecimal; + /** * Date: 2017/4/21 * Company: www.dtstack.com @@ -51,8 +53,14 @@ public static Integer getIntegerVal(Object obj){ if(obj instanceof String){ return Integer.valueOf((String) obj); - }else if(obj instanceof Integer){ + } else if (obj instanceof Integer){ return (Integer) obj; + } else if (obj instanceof Long){ + return ((Long)obj).intValue(); + } else if(obj instanceof Double){ + return ((Double)obj).intValue(); + } else if(obj instanceof BigDecimal){ + return ((BigDecimal)obj).intValue(); } throw new RuntimeException("not support type of " + obj.getClass() + " convert to Integer." ); diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/AbsCustomerWaterMarker.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/AbsCustomerWaterMarker.java new file mode 100644 index 000000000..872c47d23 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/AbsCustomerWaterMarker.java @@ -0,0 +1,74 @@ +package com.dtstack.flink.sql.watermarker; + +import com.dtstack.flink.sql.metric.EventDelayGauge; +import org.apache.flink.api.common.functions.IterationRuntimeContext; +import org.apache.flink.api.common.functions.RichFunction; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor; +import org.apache.flink.streaming.api.windowing.time.Time; + +/** + * Reason: + * Date: 2018/10/18 + * Company: www.dtstack.com + * @author xuchao + */ + +public abstract class AbsCustomerWaterMarker extends BoundedOutOfOrdernessTimestampExtractor implements RichFunction { + + private static final String EVENT_DELAY_GAUGE = "eventDelay"; + + private static final long serialVersionUID = 1L; + + private String fromSourceTag = "NONE"; + + private transient RuntimeContext runtimeContext; + + protected transient EventDelayGauge eventDelayGauge; + + public AbsCustomerWaterMarker(Time maxOutOfOrderness) { + super(maxOutOfOrderness); + } + + @Override + public void open(Configuration parameters) throws Exception { + } + + @Override + public void close() throws Exception { + //do nothing + } + + @Override + public RuntimeContext getRuntimeContext() { + if (this.runtimeContext != null) { + return this.runtimeContext; + } else { + throw new IllegalStateException("The runtime context has not been initialized."); + } + } + + @Override + public IterationRuntimeContext getIterationRuntimeContext() { + if (this.runtimeContext == null) { + throw new IllegalStateException("The runtime context has not been initialized."); + } else if (this.runtimeContext instanceof IterationRuntimeContext) { + return (IterationRuntimeContext) this.runtimeContext; + } else { + throw new IllegalStateException("This stub is not part of an iteration step function."); + } + } + + @Override + public void setRuntimeContext(RuntimeContext t) { + this.runtimeContext = t; + eventDelayGauge = new EventDelayGauge(); + t.getMetricGroup().getAllVariables().put("", fromSourceTag); + t.getMetricGroup().gauge(EVENT_DELAY_GAUGE, eventDelayGauge); + } + + public void setFromSourceTag(String fromSourceTag) { + this.fromSourceTag = fromSourceTag; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java index fe8904efd..0a6874a81 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java @@ -34,7 +34,7 @@ * @author xuchao */ -public class CustomerWaterMarkerForLong extends BoundedOutOfOrdernessTimestampExtractor { +public class CustomerWaterMarkerForLong extends AbsCustomerWaterMarker { private static final Logger logger = LoggerFactory.getLogger(CustomerWaterMarkerForLong.class); @@ -55,6 +55,7 @@ public long extractTimestamp(Row row) { try{ Long eveTime = MathUtil.getLongVal(row.getField(pos)); lastTime = eveTime; + eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - eveTime)/1000)); return eveTime; }catch (Exception e){ logger.error("", e); diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java index aa6810066..b4aee0302 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java @@ -20,15 +20,13 @@ package com.dtstack.flink.sql.watermarker; -import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor; +import com.dtstack.flink.sql.util.MathUtil; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.sql.Timestamp; -import java.text.ParseException; -import java.text.SimpleDateFormat; /** * Custom watermark --- for eventtime @@ -37,7 +35,7 @@ * @author xuchao */ -public class CustomerWaterMarkerForTimeStamp extends BoundedOutOfOrdernessTimestampExtractor { +public class CustomerWaterMarkerForTimeStamp extends AbsCustomerWaterMarker { private static final Logger logger = LoggerFactory.getLogger(CustomerWaterMarkerForTimeStamp.class); @@ -47,6 +45,7 @@ public class CustomerWaterMarkerForTimeStamp extends BoundedOutOfOrdernessTimest private long lastTime = 0; + public CustomerWaterMarkerForTimeStamp(Time maxOutOfOrderness, int pos) { super(maxOutOfOrderness); this.pos = pos; @@ -57,10 +56,14 @@ public long extractTimestamp(Row row) { try { Timestamp time = (Timestamp) row.getField(pos); lastTime = time.getTime(); + + eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - time.getTime())/1000)); return time.getTime(); } catch (RuntimeException e) { logger.error("", e); } return lastTime; } + + } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java index b095ea196..a29e8391b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java @@ -48,7 +48,11 @@ public boolean checkNeedAssignWaterMarker(SourceTableInfo tableInfo){ return true; } - public DataStream assignWaterMarker(DataStream dataStream, RowTypeInfo typeInfo, String eventTimeFieldName, int maxOutOfOrderness){ + public DataStream assignWaterMarker(DataStream dataStream, RowTypeInfo typeInfo, SourceTableInfo sourceTableInfo){ + + String eventTimeFieldName = sourceTableInfo.getEventTimeField(); + + int maxOutOrderness = sourceTableInfo.getMaxOutOrderness(); String[] fieldNames = typeInfo.getFieldNames(); TypeInformation[] fieldTypes = typeInfo.getFieldTypes(); @@ -69,15 +73,17 @@ public DataStream assignWaterMarker(DataStream dataStream, RowTypeInfo type TypeInformation fieldType = fieldTypes[pos]; - BoundedOutOfOrdernessTimestampExtractor waterMarker = null; + AbsCustomerWaterMarker waterMarker = null; if(fieldType.getTypeClass().getTypeName().equalsIgnoreCase("java.sql.Timestamp")){ - waterMarker = new CustomerWaterMarkerForTimeStamp(Time.milliseconds(maxOutOfOrderness), pos); + waterMarker = new CustomerWaterMarkerForTimeStamp(Time.milliseconds(maxOutOrderness), pos); }else if(fieldType.getTypeClass().getTypeName().equalsIgnoreCase("java.lang.Long")){ - waterMarker = new CustomerWaterMarkerForLong(Time.milliseconds(maxOutOfOrderness), pos); + waterMarker = new CustomerWaterMarkerForLong(Time.milliseconds(maxOutOrderness), pos); }else{ throw new IllegalArgumentException("not support type of " + fieldType + ", current only support(timestamp, long)."); } + String fromTag = "Source:" + sourceTableInfo.getName(); + waterMarker.setFromSourceTag(fromTag); return dataStream.assignTimestampsAndWatermarks(waterMarker); } } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 6c176badc..9b1637696 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -21,9 +21,14 @@ package com.dtstack.flink.sql.source.kafka; +import org.apache.flink.api.common.accumulators.LongCounter; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.types.Row; @@ -46,9 +51,25 @@ public class CustomerJsonDeserialization extends AbstractDeserializationSchema typeInfo; + private transient RuntimeContext runtimeContext; + /** Field names to parse. Indices match fieldTypes indices. */ private final String[] fieldNames; @@ -58,6 +79,25 @@ public class CustomerJsonDeserialization extends AbstractDeserializationSchema typeInfo){ this.typeInfo = typeInfo; @@ -69,6 +109,9 @@ public CustomerJsonDeserialization(TypeInformation typeInfo){ @Override public Row deserialize(byte[] message) throws IOException { try { + myCounter.add(1); + kafkaInRecord.inc(); + kafkaInBytes.inc(message.length); JsonNode root = objectMapper.readTree(message); Row row = new Row(fieldNames.length); for (int i = 0; i < fieldNames.length; i++) { @@ -88,9 +131,12 @@ public Row deserialize(byte[] message) throws IOException { } } + kafkaInResolveRecord.inc(); return row; } catch (Throwable t) { - throw new IOException("Failed to deserialize JSON object.", t); + //add metric of dirty data + dirtyDataCounter.inc(); + return new Row(fieldNames.length); } } @@ -111,4 +157,28 @@ public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { return null; } + + public RuntimeContext getRuntimeContext() { + return runtimeContext; + } + + public void setRuntimeContext(RuntimeContext runtimeContext) { + this.runtimeContext = runtimeContext; + } + + public void initMetric(){ + dirtyDataCounter = runtimeContext.getMetricGroup().counter(DIRTY_DATA_METRICS_COUNTER); + + kafkaInRecord = runtimeContext.getMetricGroup().counter(KAFKA_SOURCE_IN_METRIC_COUNTER); + kafkaInRate = runtimeContext.getMetricGroup().meter( KAFKA_SOURCE_IN_RATE_METRIC_METRE, new MeterView(kafkaInRecord, 20)); + + kafkaInBytes = runtimeContext.getMetricGroup().counter(KAFKA_SOURCE_IN_BYTES_METRIC_COUNTER); + kafkaInBytesRate = runtimeContext.getMetricGroup().meter( KAFKA_SOURCE_IN_BYTES_RATE_METRIC_METRE, new MeterView(kafkaInBytes, 20)); + + kafkaInResolveRecord = runtimeContext.getMetricGroup().counter(KAFKA_SOURCE_IN_RESOLVE_METRIC_COUNTER); + kafkaInResolveRate = runtimeContext.getMetricGroup().meter(KAFKA_SOURCE_IN_RESOLVE_RATE_METRIC_METRE, new MeterView(kafkaInResolveRecord, 20)); + + //FIXME + myCounter = runtimeContext.getLongCounter("kafkaSourceTotalIn"); + } } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java new file mode 100644 index 000000000..dfca281a3 --- /dev/null +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java @@ -0,0 +1,30 @@ +package com.dtstack.flink.sql.source.kafka; + +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; +import org.apache.flink.types.Row; + +import java.util.Properties; + +/** + * Reason: + * Date: 2018/10/12 + * Company: www.dtstack.com + * @author xuchao + */ + +public class CustomerKafka09Consumer extends FlinkKafkaConsumer09 { + + private CustomerJsonDeserialization customerJsonDeserialization; + + public CustomerKafka09Consumer(String topic, CustomerJsonDeserialization valueDeserializer, Properties props) { + super(topic, valueDeserializer, props); + this.customerJsonDeserialization = valueDeserializer; + } + + @Override + public void run(SourceContext sourceContext) throws Exception { + customerJsonDeserialization.setRuntimeContext(getRuntimeContext()); + customerJsonDeserialization.initMetric(); + super.run(sourceContext); + } +} diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 8e305e06d..8486b9883 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -24,6 +24,7 @@ import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -43,6 +44,8 @@ public class KafkaSource implements IStreamSourceGener
{ + private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; + /** * Get kafka data source, you need to provide the data field names, data types * If you do not specify auto.offset.reset, the default use groupoffset @@ -67,7 +70,7 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } TypeInformation typeInformation = new RowTypeInfo(types, kafka09SourceTableInfo.getFields()); - FlinkKafkaConsumer09 kafkaSrc = new FlinkKafkaConsumer09(topicName, + FlinkKafkaConsumer09 kafkaSrc = new CustomerKafka09Consumer(topicName, new CustomerJsonDeserialization(typeInformation), props); //earliest,latest @@ -78,6 +81,7 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } String fields = StringUtils.join(kafka09SourceTableInfo.getFields(), ","); - return tableEnv.fromDataStream(env.addSource(kafkaSrc, typeInformation), fields); + String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); + return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); } } From d6afa710702cefc03ab1d0049b326fbed7142b2f Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 18 Oct 2018 21:24:56 +0800 Subject: [PATCH 060/470] modify input metric name --- .../flink/sql/metric/MetricConstant.java | 31 +++++++++++ .../kafka/CustomerJsonDeserialization.java | 55 ++++++------------- .../dtstack/flink/sql/sink/mysql/DBSink.java | 1 + .../sink/mysql/RetractJDBCOutputFormat.java | 16 ++++++ 4 files changed, 65 insertions(+), 38 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java diff --git a/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java b/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java new file mode 100644 index 000000000..e829a892e --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java @@ -0,0 +1,31 @@ +package com.dtstack.flink.sql.metric; + +/** + * Reason: + * Date: 2018/10/18 + * Company: www.dtstack.com + * @author xuchao + */ + +public class MetricConstant { + + /**metric name of dirty data*/ + public static final String DT_DIRTY_DATA_COUNTER = "dtDirtyData"; + + public static final String DT_NUM_RECORDS_IN_COUNTER = "dtNumRecordsIn"; + + public static final String DT_NUM_RECORDS_IN_RATE = "dtNumRecordsInRate"; + + public static final String DT_NUM_BYTES_IN_COUNTER = "dtNumBytesIn"; + + public static final String DT_NUM_BYTES_IN_RATE = "dtNumBytesInRate"; + + /**diff of DT_NUM_RECORD_IN_COUNTER ,this metric is desc record num after of deserialization*/ + public static final String DT_NUM_RECORDS_RESOVED_IN_COUNTER = "dtNumRecordsInResolve"; + + public static final String DT_NUM_RECORDS_RESOVED_IN_RATE = "dtNumRecordsInResolveRate"; + + public static final String DT_NUM_RECORDS_OUT = "dtNumRecordsOut"; + + public static final String DT_NUM_RECORDS_OUT_RATE = "dtNumRecordsOutRate"; +} diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 9b1637696..c0fc40dfc 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -21,7 +21,7 @@ package com.dtstack.flink.sql.source.kafka; -import org.apache.flink.api.common.accumulators.LongCounter; +import com.dtstack.flink.sql.metric.MetricConstant; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -51,20 +51,6 @@ public class CustomerJsonDeserialization extends AbstractDeserializationSchema typeInfo; @@ -82,21 +68,18 @@ public class CustomerJsonDeserialization extends AbstractDeserializationSchema typeInfo){ this.typeInfo = typeInfo; @@ -109,9 +92,8 @@ public CustomerJsonDeserialization(TypeInformation typeInfo){ @Override public Row deserialize(byte[] message) throws IOException { try { - myCounter.add(1); - kafkaInRecord.inc(); - kafkaInBytes.inc(message.length); + numInRecord.inc(); + numInBytes.inc(message.length); JsonNode root = objectMapper.readTree(message); Row row = new Row(fieldNames.length); for (int i = 0; i < fieldNames.length; i++) { @@ -131,7 +113,7 @@ public Row deserialize(byte[] message) throws IOException { } } - kafkaInResolveRecord.inc(); + numInResolveRecord.inc(); return row; } catch (Throwable t) { //add metric of dirty data @@ -167,18 +149,15 @@ public void setRuntimeContext(RuntimeContext runtimeContext) { } public void initMetric(){ - dirtyDataCounter = runtimeContext.getMetricGroup().counter(DIRTY_DATA_METRICS_COUNTER); - - kafkaInRecord = runtimeContext.getMetricGroup().counter(KAFKA_SOURCE_IN_METRIC_COUNTER); - kafkaInRate = runtimeContext.getMetricGroup().meter( KAFKA_SOURCE_IN_RATE_METRIC_METRE, new MeterView(kafkaInRecord, 20)); + dirtyDataCounter = runtimeContext.getMetricGroup().counter(MetricConstant.DT_DIRTY_DATA_COUNTER); - kafkaInBytes = runtimeContext.getMetricGroup().counter(KAFKA_SOURCE_IN_BYTES_METRIC_COUNTER); - kafkaInBytesRate = runtimeContext.getMetricGroup().meter( KAFKA_SOURCE_IN_BYTES_RATE_METRIC_METRE, new MeterView(kafkaInBytes, 20)); + numInRecord = runtimeContext.getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_IN_COUNTER); + numInRate = runtimeContext.getMetricGroup().meter( MetricConstant.DT_NUM_RECORDS_IN_RATE, new MeterView(numInRecord, 20)); - kafkaInResolveRecord = runtimeContext.getMetricGroup().counter(KAFKA_SOURCE_IN_RESOLVE_METRIC_COUNTER); - kafkaInResolveRate = runtimeContext.getMetricGroup().meter(KAFKA_SOURCE_IN_RESOLVE_RATE_METRIC_METRE, new MeterView(kafkaInResolveRecord, 20)); + numInBytes = runtimeContext.getMetricGroup().counter(MetricConstant.DT_NUM_BYTES_IN_COUNTER); + numInBytesRate = runtimeContext.getMetricGroup().meter(MetricConstant.DT_NUM_BYTES_IN_RATE , new MeterView(numInBytes, 20)); - //FIXME - myCounter = runtimeContext.getLongCounter("kafkaSourceTotalIn"); + numInResolveRecord = runtimeContext.getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_RESOVED_IN_COUNTER); + numInResolveRate = runtimeContext.getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_RESOVED_IN_RATE, new MeterView(numInResolveRecord, 20)); } } diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java index 26ab2da73..e6d6ca9d2 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java @@ -133,6 +133,7 @@ public void setBatchInterval(int batchInterval) { public void emitDataStream(DataStream> dataStream) { RichSinkFunction richSinkFunction = createJdbcSinkFunc(); DataStreamSink streamSink = dataStream.addSink(richSinkFunction); + streamSink.name(tableName); if(parallelism > 0){ streamSink.setParallelism(parallelism); } diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java index 3a6d76364..fe3751aee 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java @@ -38,10 +38,14 @@ package com.dtstack.flink.sql.sink.mysql; +import com.dtstack.flink.sql.metric.MetricConstant; import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -77,6 +81,10 @@ public class RetractJDBCOutputFormat extends RichOutputFormat { private int batchCount = 0; public int[] typesArray; + + private transient Counter outRecords; + + private transient Meter outRecordsRate; public RetractJDBCOutputFormat() { } @@ -97,12 +105,18 @@ public void open(int taskNumber, int numTasks) throws IOException { try { establishConnection(); upload = dbConn.prepareStatement(insertQuery); + initMetric(); } catch (SQLException sqe) { throw new IllegalArgumentException("open() failed.", sqe); } catch (ClassNotFoundException cnfe) { throw new IllegalArgumentException("JDBC driver class not found.", cnfe); } } + + private void initMetric(){ + outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); + } private void establishConnection() throws SQLException, ClassNotFoundException { Class.forName(drivername); @@ -140,6 +154,7 @@ public void writeRecord(Tuple2 tuple2) throws IOException { try { if(retract){ insertWrite(row); + outRecords.inc(); }else{ //do nothing } @@ -150,6 +165,7 @@ public void writeRecord(Tuple2 tuple2) throws IOException { private void insertWrite(Row row) throws SQLException { + updatePreparedStmt(row, upload); upload.addBatch(); batchCount++; From 4d44b0c0ae66d982468f247bcada95bfa3970499 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 18 Oct 2018 21:39:34 +0800 Subject: [PATCH 061/470] comment --- .gitignore | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.gitignore b/.gitignore index c88786513..7b6a61cc8 100644 --- a/.gitignore +++ b/.gitignore @@ -9,3 +9,6 @@ target/ *.iml plugins/ lib/ +.vertx/ +bin/nohup.out +.vertx/ From 9570d4553831aa36f5f5f1ff235b1d1ed70eb89a Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 18 Oct 2018 21:40:29 +0800 Subject: [PATCH 062/470] comment --- .gitignore | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.gitignore b/.gitignore index 7b6a61cc8..cedc8fd14 100644 --- a/.gitignore +++ b/.gitignore @@ -11,4 +11,5 @@ plugins/ lib/ .vertx/ bin/nohup.out -.vertx/ + +bin/sideSql.txt From 8859ccc2e377262a6e3825e5ffebb10e20bf6d14 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 18 Oct 2018 21:43:17 +0800 Subject: [PATCH 063/470] comment update --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 719f0ca38..315a83d2c 100644 --- a/README.md +++ b/README.md @@ -47,7 +47,7 @@ mvn clean package -Dmaven.test.skip #### 1.4.1 启动命令 ``` -sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack/150_flinkplugin/sqlplugin -localSqlPluginPath D:\gitspace\flinkStreamSQL\plugins -mode yarn -flinkconf D:\flink_home\kudu150etc -yarnconf D:\hadoop\etc\hadoopkudu -confProp {\"time.characteristic\":\"EventTime\",\"sql.checkpoint.interval\":10000} +sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack/150_flinkplugin/sqlplugin -localSqlPluginPath D:\gitspace\flinkStreamSQL\plugins -mode yarn -flinkconf D:\flink_home\kudu150etc -yarnconf D:\hadoop\etc\hadoopkudu -confProp {"time.characteristic":"EventTime","sql.checkpoint.interval":10000} ``` #### 1.4.2 命令行参数选项 From 73d3b09430d03ce3bb4afb2e6dd884c157b8970b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Thu, 18 Oct 2018 21:54:44 +0800 Subject: [PATCH 064/470] implement hbase table cache --- hbase/hbase-side/hbase-all-side/pom.xml | 20 +++ .../flink/sql/side/hbase/HbaseAllReqRow.java | 154 ++++++++++++++++++ .../sql/side/hbase/HbaseAllSideInfo.java | 75 +++++++++ 3 files changed, 249 insertions(+) create mode 100644 hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java create mode 100644 hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java diff --git a/hbase/hbase-side/hbase-all-side/pom.xml b/hbase/hbase-side/hbase-all-side/pom.xml index f0f08689d..ca51efab8 100644 --- a/hbase/hbase-side/hbase-all-side/pom.xml +++ b/hbase/hbase-side/hbase-all-side/pom.xml @@ -13,6 +13,26 @@ sql.side.all.hbase hbase-all-side + + + com.dtstack.flink + sql.side.hbase.core + 1.0-SNAPSHOT + + + + org.hbase + asynchbase + 1.8.2 + + + + org.apache.hbase + hbase-client + 1.3.1 + + + diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java new file mode 100644 index 000000000..c35dea5b4 --- /dev/null +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java @@ -0,0 +1,154 @@ +package com.dtstack.flink.sql.side.hbase; + +import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.util.Bytes; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.sql.SQLException; +import java.sql.Timestamp; +import java.util.*; +import java.util.concurrent.atomic.AtomicReference; + +public class HbaseAllReqRow extends AllReqRow { + + private static final Logger LOG = LoggerFactory.getLogger(HbaseAllReqRow.class); + + private String tableName; + + private AtomicReference>> cacheRef = new AtomicReference<>(); + + public HbaseAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new HbaseAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + tableName = ((HbaseSideTableInfo)sideTableInfo).getTableName(); + } + + @Override + protected Row fillData(Row input, Object sideInput) { + Map sideInputList = (Map) sideInput; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. + if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ + obj = ((Timestamp)obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ + if(sideInputList == null){ + row.setField(entry.getKey(), null); + }else{ + row.setField(entry.getKey(), sideInputList.get(entry.getValue())); + } + } + + return row; + } + + @Override + protected void initCache() throws SQLException { + Map> newCache = Maps.newConcurrentMap(); + cacheRef.set(newCache); + loadData(newCache); + } + + @Override + protected void reloadCache() { + Map> newCache = Maps.newConcurrentMap(); + try { + loadData(newCache); + } catch (SQLException e) { + LOG.error("", e); + } + + cacheRef.set(newCache); + LOG.info("----- HBase all cacheRef reload end:{}", Calendar.getInstance()); + } + + @Override + public void flatMap(Row value, Collector out) throws Exception { + Map refData = Maps.newHashMap(); + for (int i = 0; i < sideInfo.getEqualValIndex().size(); i++) { + Integer conValIndex = sideInfo.getEqualValIndex().get(i); + Object equalObj = value.getField(conValIndex); + if(equalObj == null){ + out.collect(null); + } + refData.put(sideInfo.getEqualFieldList().get(i), equalObj); + } + + String rowKeyStr = ((HbaseAllSideInfo)sideInfo).getRowKeyBuilder().getRowKey(refData); + + Object cacheList = cacheRef.get().get(rowKeyStr); + Row row = fillData(value, cacheList); + out.collect(row); + } + + private void loadData(Map> tmpCache) throws SQLException { + SideTableInfo sideTableInfo = sideInfo.getSideTableInfo(); + HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo; + Configuration conf = new Configuration(); + conf.set("hbase.zookeeper.quorum", hbaseSideTableInfo.getHost()); + Connection conn = null; + Table table = null; + ResultScanner resultScanner = null; + try { + conn = ConnectionFactory.createConnection(conf); + table = conn.getTable(TableName.valueOf(tableName)); + resultScanner = table.getScanner(new Scan()); + List rows = new LinkedList<>(); + for (Result r : resultScanner) { + for (Cell cell : r.listCells()){ + rows.add(cell.getRow().toString()); + } + } + //根据表,rowkey查询值 + for (int i=0; i < rows.size(); i++){ + Get get = new Get(Bytes.toBytes(rows.get(i))); + Result result = table.get(get); + tmpCache.put(rows.get(i), result2Map(result)); + } + } catch (IOException e) { + e.printStackTrace(); + } finally { + try { + conn.close(); + table.close(); + resultScanner.close(); + } catch (IOException e) { + e.printStackTrace(); + } + } + } + + private static Map result2Map(Result result) { + Map ret = new HashMap(); + if (result != null && result.listCells() != null) { + for (Cell cell : result.listCells()) { + String family = Bytes.toString(CellUtil.cloneFamily(cell)); + String qualifier = Bytes.toString(CellUtil.cloneQualifier(cell)); + String value = Bytes.toString(CellUtil.cloneValue(cell)); + StringBuilder key = new StringBuilder(); + key.append(family).append(":").append(qualifier); + ret.put(key.toString(), value); + } + } + return ret; + } +} diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java new file mode 100644 index 000000000..08c426deb --- /dev/null +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java @@ -0,0 +1,75 @@ +package com.dtstack.flink.sql.side.hbase; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; + +public class HbaseAllSideInfo extends SideInfo { + + private RowKeyBuilder rowKeyBuilder; + + private Map colRefType; + + public HbaseAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + rowKeyBuilder = new RowKeyBuilder(); + if(sideTableInfo.getPrimaryKeys().size() < 1){ + throw new RuntimeException("Primary key dimension table must be filled"); + } + + HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo; + rowKeyBuilder.init(sideTableInfo.getPrimaryKeys().get(0)); + + colRefType = Maps.newHashMap(); + for(int i=0; i sqlNodeList = Lists.newArrayList(); + if(conditionNode.getKind() == SqlKind.AND){ + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); + }else{ + sqlNodeList.add(conditionNode); + } + + for(SqlNode sqlNode : sqlNodeList){ + dealOneEqualCon(sqlNode, sideTableName); + } + } + + public RowKeyBuilder getRowKeyBuilder() { + return rowKeyBuilder; + } + + public void setRowKeyBuilder(RowKeyBuilder rowKeyBuilder) { + this.rowKeyBuilder = rowKeyBuilder; + } + + public Map getColRefType() { + return colRefType; + } + + public void setColRefType(Map colRefType) { + this.colRefType = colRefType; + } +} From eecb4a72ea0ceff3b087cd64f43519a0119b37e7 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 19 Oct 2018 10:13:20 +0800 Subject: [PATCH 065/470] abstract class of source metric --- .../flink/sql/metric/EventDelayGauge.java | 20 ++++- .../flink/sql/metric/MetricConstant.java | 20 ++++- .../flink/sql/source/AbsDeserialization.java | 78 +++++++++++++++++++ .../kafka/CustomerJsonDeserialization.java | 49 +----------- .../source/kafka/CustomerKafka09Consumer.java | 25 +++++- .../kafka/CustomerJsonDeserialization.java | 11 ++- .../kafka/CustomerKafka010Consumer.java | 51 ++++++++++++ .../flink/sql/source/kafka/KafkaSource.java | 2 +- .../kafka/CustomerJsonDeserialization.java | 3 +- .../kafka/CustomerKafka011Consumer.java | 53 +++++++++++++ .../flink/sql/source/kafka/KafkaSource.java | 2 +- 11 files changed, 260 insertions(+), 54 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/source/AbsDeserialization.java create mode 100644 kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka010Consumer.java create mode 100644 kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka011Consumer.java diff --git a/core/src/main/java/com/dtstack/flink/sql/metric/EventDelayGauge.java b/core/src/main/java/com/dtstack/flink/sql/metric/EventDelayGauge.java index 3e3d558ec..fd2b3351f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/metric/EventDelayGauge.java +++ b/core/src/main/java/com/dtstack/flink/sql/metric/EventDelayGauge.java @@ -1,9 +1,27 @@ +/* + * 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 com.dtstack.flink.sql.metric; import org.apache.flink.metrics.Gauge; /** - * 数据延迟时间 单位 s + * event data delay time ,unit seconds * Date: 2018/10/18 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java b/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java index e829a892e..6f6a40bc9 100644 --- a/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java +++ b/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java @@ -1,7 +1,25 @@ +/* + * 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 com.dtstack.flink.sql.metric; /** - * Reason: + * defined customer metric name * Date: 2018/10/18 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/source/AbsDeserialization.java b/core/src/main/java/com/dtstack/flink/sql/source/AbsDeserialization.java new file mode 100644 index 000000000..d8b2e4d7e --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/source/AbsDeserialization.java @@ -0,0 +1,78 @@ +/* + * 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 com.dtstack.flink.sql.source; + +import com.dtstack.flink.sql.metric.MetricConstant; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; + +/** + * add metric for source, customer Deserialization which want add metric need to extends this abs class + * Date: 2018/10/19 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public abstract class AbsDeserialization extends AbstractDeserializationSchema { + + private static final long serialVersionUID = 2176278128811784415L; + + private transient RuntimeContext runtimeContext; + + protected transient Counter dirtyDataCounter; + + //tps ransactions Per Second + protected transient Counter numInRecord; + + protected transient Meter numInRate; + + //rps Record Per Second: deserialize data and out record num + protected transient Counter numInResolveRecord; + + protected transient Meter numInResolveRate; + + protected transient Counter numInBytes; + + protected transient Meter numInBytesRate; + + public RuntimeContext getRuntimeContext() { + return runtimeContext; + } + + public void setRuntimeContext(RuntimeContext runtimeContext) { + this.runtimeContext = runtimeContext; + } + + public void initMetric(){ + dirtyDataCounter = runtimeContext.getMetricGroup().counter(MetricConstant.DT_DIRTY_DATA_COUNTER); + + numInRecord = runtimeContext.getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_IN_COUNTER); + numInRate = runtimeContext.getMetricGroup().meter( MetricConstant.DT_NUM_RECORDS_IN_RATE, new MeterView(numInRecord, 20)); + + numInBytes = runtimeContext.getMetricGroup().counter(MetricConstant.DT_NUM_BYTES_IN_COUNTER); + numInBytesRate = runtimeContext.getMetricGroup().meter(MetricConstant.DT_NUM_BYTES_IN_RATE , new MeterView(numInBytes, 20)); + + numInResolveRecord = runtimeContext.getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_RESOVED_IN_COUNTER); + numInResolveRate = runtimeContext.getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_RESOVED_IN_RATE, new MeterView(numInResolveRecord, 20)); + } +} diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index c0fc40dfc..dd058a25b 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -21,14 +21,9 @@ package com.dtstack.flink.sql.source.kafka; -import com.dtstack.flink.sql.metric.MetricConstant; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; +import com.dtstack.flink.sql.source.AbsDeserialization; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.MeterView; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.types.Row; @@ -45,17 +40,17 @@ * @author xuchao */ -public class CustomerJsonDeserialization extends AbstractDeserializationSchema { +public class CustomerJsonDeserialization extends AbsDeserialization { private static final Logger logger = LoggerFactory.getLogger(CustomerJsonDeserialization.class); + private static final long serialVersionUID = -2706012724306826506L; + private final ObjectMapper objectMapper = new ObjectMapper(); /** Type information describing the result type. */ private final TypeInformation typeInfo; - private transient RuntimeContext runtimeContext; - /** Field names to parse. Indices match fieldTypes indices. */ private final String[] fieldNames; @@ -65,22 +60,6 @@ public class CustomerJsonDeserialization extends AbstractDeserializationSchema typeInfo){ this.typeInfo = typeInfo; @@ -140,24 +119,4 @@ public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { } - public RuntimeContext getRuntimeContext() { - return runtimeContext; - } - - public void setRuntimeContext(RuntimeContext runtimeContext) { - this.runtimeContext = runtimeContext; - } - - public void initMetric(){ - dirtyDataCounter = runtimeContext.getMetricGroup().counter(MetricConstant.DT_DIRTY_DATA_COUNTER); - - numInRecord = runtimeContext.getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_IN_COUNTER); - numInRate = runtimeContext.getMetricGroup().meter( MetricConstant.DT_NUM_RECORDS_IN_RATE, new MeterView(numInRecord, 20)); - - numInBytes = runtimeContext.getMetricGroup().counter(MetricConstant.DT_NUM_BYTES_IN_COUNTER); - numInBytesRate = runtimeContext.getMetricGroup().meter(MetricConstant.DT_NUM_BYTES_IN_RATE , new MeterView(numInBytes, 20)); - - numInResolveRecord = runtimeContext.getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_RESOVED_IN_COUNTER); - numInResolveRate = runtimeContext.getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_RESOVED_IN_RATE, new MeterView(numInResolveRecord, 20)); - } } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java index dfca281a3..870cabfea 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java @@ -1,5 +1,24 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka; +import com.dtstack.flink.sql.source.AbsDeserialization; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; import org.apache.flink.types.Row; @@ -14,9 +33,11 @@ public class CustomerKafka09Consumer extends FlinkKafkaConsumer09 { - private CustomerJsonDeserialization customerJsonDeserialization; + private static final long serialVersionUID = 4451177393982291909L; + + private AbsDeserialization customerJsonDeserialization; - public CustomerKafka09Consumer(String topic, CustomerJsonDeserialization valueDeserializer, Properties props) { + public CustomerKafka09Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { super(topic, valueDeserializer, props); this.customerJsonDeserialization = valueDeserializer; } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 9225464ab..e0ef3ff05 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.source.kafka; +import com.dtstack.flink.sql.source.AbsDeserialization; import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -40,7 +41,7 @@ * @author sishu.yss */ -public class CustomerJsonDeserialization extends AbstractDeserializationSchema { +public class CustomerJsonDeserialization extends AbsDeserialization { private static final Logger logger = LoggerFactory.getLogger(CustomerJsonDeserialization.class); @@ -69,6 +70,9 @@ public CustomerJsonDeserialization(TypeInformation typeInfo){ @Override public Row deserialize(byte[] message) throws IOException { try { + numInRecord.inc(); + numInBytes.inc(message.length); + JsonNode root = objectMapper.readTree(message); Row row = new Row(fieldNames.length); for (int i = 0; i < fieldNames.length; i++) { @@ -88,9 +92,12 @@ public Row deserialize(byte[] message) throws IOException { } } + numInResolveRecord.inc(); return row; } catch (Throwable t) { - throw new IOException("Failed to deserialize JSON object.", t); + //add metric of dirty data + dirtyDataCounter.inc(); + return new Row(fieldNames.length); } } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka010Consumer.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka010Consumer.java new file mode 100644 index 000000000..862e3530f --- /dev/null +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka010Consumer.java @@ -0,0 +1,51 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; +import org.apache.flink.types.Row; + +import java.util.Properties; + +/** + * Reason: + * Date: 2018/10/19 + * Company: www.dtstack.com + * @author xuchao + */ + +public class CustomerKafka010Consumer extends FlinkKafkaConsumer010 { + + private static final long serialVersionUID = 4873757508981691375L; + + private AbsDeserialization customerJsonDeserialization; + + public CustomerKafka010Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + super(topic, valueDeserializer, props); + this.customerJsonDeserialization = valueDeserializer; + } + + @Override + public void run(SourceContext sourceContext) throws Exception { + customerJsonDeserialization.setRuntimeContext(getRuntimeContext()); + customerJsonDeserialization.initMetric(); + super.run(sourceContext); + } +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index a3ea8693b..e884dcc37 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -67,7 +67,7 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } TypeInformation typeInformation = new RowTypeInfo(types, kafka010SourceTableInfo.getFields()); - FlinkKafkaConsumer010 kafkaSrc = new FlinkKafkaConsumer010(topicName, + FlinkKafkaConsumer010 kafkaSrc = new CustomerKafka010Consumer(topicName, new CustomerJsonDeserialization(typeInformation), props); //earliest,latest diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 9225464ab..aa0a5fe2c 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.source.kafka; +import com.dtstack.flink.sql.source.AbsDeserialization; import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -40,7 +41,7 @@ * @author sishu.yss */ -public class CustomerJsonDeserialization extends AbstractDeserializationSchema { +public class CustomerJsonDeserialization extends AbsDeserialization { private static final Logger logger = LoggerFactory.getLogger(CustomerJsonDeserialization.class); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka011Consumer.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka011Consumer.java new file mode 100644 index 000000000..390028d20 --- /dev/null +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka011Consumer.java @@ -0,0 +1,53 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; +import org.apache.flink.types.Row; + + +import java.util.Properties; + +/** + * Reason: + * Date: 2018/10/19 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class CustomerKafka011Consumer extends FlinkKafkaConsumer011 { + + private static final long serialVersionUID = -2265366268827807739L; + + private AbsDeserialization customerJsonDeserialization; + + public CustomerKafka011Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + super(topic, valueDeserializer, props); + this.customerJsonDeserialization = valueDeserializer; + } + + @Override + public void run(SourceContext sourceContext) throws Exception { + customerJsonDeserialization.setRuntimeContext(getRuntimeContext()); + customerJsonDeserialization.initMetric(); + super.run(sourceContext); + } +} diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index c813c21eb..4c9acd1de 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -67,7 +67,7 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } TypeInformation typeInformation = new RowTypeInfo(types, kafka011SourceTableInfo.getFields()); - FlinkKafkaConsumer011 kafkaSrc = new FlinkKafkaConsumer011(topicName, + FlinkKafkaConsumer011 kafkaSrc = new CustomerKafka011Consumer(topicName, new CustomerJsonDeserialization(typeInformation), props); //earliest,latest From 26b548f4ac06178e32e05fbb730bbf301e6f234c Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 19 Oct 2018 16:41:35 +0800 Subject: [PATCH 066/470] pom --- hbase/hbase-sink/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/hbase/hbase-sink/pom.xml b/hbase/hbase-sink/pom.xml index 44a698f60..da273d496 100644 --- a/hbase/hbase-sink/pom.xml +++ b/hbase/hbase-sink/pom.xml @@ -28,6 +28,7 @@ org.apache.hadoop:hadoop-common + org.slf4j:slf4j-log4j12 From 2f27f01e48d031803e59c94eae6e6c5883e17b4e Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 19 Oct 2018 20:20:17 +0800 Subject: [PATCH 067/470] modify event delay time metric --- .../java/com/dtstack/flink/sql/metric/MetricConstant.java | 2 ++ .../dtstack/flink/sql/watermarker/AbsCustomerWaterMarker.java | 4 ++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java b/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java index 6f6a40bc9..68e4c0a89 100644 --- a/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java +++ b/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java @@ -46,4 +46,6 @@ public class MetricConstant { public static final String DT_NUM_RECORDS_OUT = "dtNumRecordsOut"; public static final String DT_NUM_RECORDS_OUT_RATE = "dtNumRecordsOutRate"; + + public static final String DT_EVENT_DELAY_GAUGE = "dtEventDelay"; } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/AbsCustomerWaterMarker.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/AbsCustomerWaterMarker.java index 872c47d23..b990bdd82 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/AbsCustomerWaterMarker.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/AbsCustomerWaterMarker.java @@ -1,6 +1,7 @@ package com.dtstack.flink.sql.watermarker; import com.dtstack.flink.sql.metric.EventDelayGauge; +import com.dtstack.flink.sql.metric.MetricConstant; import org.apache.flink.api.common.functions.IterationRuntimeContext; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.RuntimeContext; @@ -17,7 +18,6 @@ public abstract class AbsCustomerWaterMarker extends BoundedOutOfOrdernessTimestampExtractor implements RichFunction { - private static final String EVENT_DELAY_GAUGE = "eventDelay"; private static final long serialVersionUID = 1L; @@ -65,7 +65,7 @@ public void setRuntimeContext(RuntimeContext t) { this.runtimeContext = t; eventDelayGauge = new EventDelayGauge(); t.getMetricGroup().getAllVariables().put("", fromSourceTag); - t.getMetricGroup().gauge(EVENT_DELAY_GAUGE, eventDelayGauge); + t.getMetricGroup().gauge(MetricConstant.DT_EVENT_DELAY_GAUGE, eventDelayGauge); } public void setFromSourceTag(String fromSourceTag) { From 1f1e50db3d067306dcdb99168250cb80f31a012d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Fri, 19 Oct 2018 20:21:41 +0800 Subject: [PATCH 068/470] cache all data in hbase table --- hbase/hbase-side/hbase-all-side/pom.xml | 6 ------ .../flink/sql/side/hbase/HbaseAllSideInfo.java | 17 ----------------- .../sql/side/hbase/table/HbaseSideParser.java | 3 +++ 3 files changed, 3 insertions(+), 23 deletions(-) diff --git a/hbase/hbase-side/hbase-all-side/pom.xml b/hbase/hbase-side/hbase-all-side/pom.xml index ca51efab8..f92fc6dd7 100644 --- a/hbase/hbase-side/hbase-all-side/pom.xml +++ b/hbase/hbase-side/hbase-all-side/pom.xml @@ -20,12 +20,6 @@ 1.0-SNAPSHOT - - org.hbase - asynchbase - 1.8.2 - - org.apache.hbase hbase-client diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java index 08c426deb..3265e15a0 100644 --- a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java @@ -19,8 +19,6 @@ public class HbaseAllSideInfo extends SideInfo { private RowKeyBuilder rowKeyBuilder; - private Map colRefType; - public HbaseAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @@ -32,16 +30,8 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { throw new RuntimeException("Primary key dimension table must be filled"); } - HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo; rowKeyBuilder.init(sideTableInfo.getPrimaryKeys().get(0)); - colRefType = Maps.newHashMap(); - for(int i=0; i getColRefType() { - return colRefType; - } - - public void setColRefType(Map colRefType) { - this.colRefType = colRefType; - } } diff --git a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java index 48bb9d66a..299db961d 100644 --- a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java +++ b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java @@ -56,6 +56,8 @@ public class HbaseSideParser extends AbsSideTableParser { public static final String PRE_ROW_KEY = "preRowKey"; + public static final String CACHE = "cache"; + static { keyPatternMap.put(SIDE_SIGN_KEY, SIDE_TABLE_SIGN); @@ -76,6 +78,7 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Sat, 20 Oct 2018 14:15:10 +0800 Subject: [PATCH 069/470] modify pom, exclude hadoop and log4j --- hbase/hbase-side/hbase-async-side/pom.xml | 3 +++ hbase/hbase-sink/pom.xml | 3 +++ 2 files changed, 6 insertions(+) diff --git a/hbase/hbase-side/hbase-async-side/pom.xml b/hbase/hbase-side/hbase-async-side/pom.xml index 522d503fd..2a18eeea6 100644 --- a/hbase/hbase-side/hbase-async-side/pom.xml +++ b/hbase/hbase-side/hbase-async-side/pom.xml @@ -44,6 +44,9 @@ org.slf4j:slf4j-log4j12 + org.apache.hadoop:hadoop-common + org.apache.hadoop:hadoop-auth + org.apache.hadoop:hadoop-mapreduce-client-core diff --git a/hbase/hbase-sink/pom.xml b/hbase/hbase-sink/pom.xml index 44a698f60..9ec6d161c 100644 --- a/hbase/hbase-sink/pom.xml +++ b/hbase/hbase-sink/pom.xml @@ -27,7 +27,10 @@ + org.slf4j:slf4j-log4j12 org.apache.hadoop:hadoop-common + org.apache.hadoop:hadoop-auth + org.apache.hadoop:hadoop-mapreduce-client-core From 7326b30b7be940c20c38e2044ac2730c11a26cc2 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 20 Oct 2018 14:18:31 +0800 Subject: [PATCH 070/470] modify pom, exclude hadoop and log4j --- hbase/hbase-side/hbase-all-side/pom.xml | 5 ++++- hbase/hbase-side/hbase-async-side/pom.xml | 3 +++ hbase/hbase-sink/pom.xml | 4 +++- 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/hbase/hbase-side/hbase-all-side/pom.xml b/hbase/hbase-side/hbase-all-side/pom.xml index f0f08689d..f70bdacdc 100644 --- a/hbase/hbase-side/hbase-all-side/pom.xml +++ b/hbase/hbase-side/hbase-all-side/pom.xml @@ -28,7 +28,10 @@ - + org.slf4j:slf4j-log4j12 + org.apache.hadoop:hadoop-common + org.apache.hadoop:hadoop-auth + org.apache.hadoop:hadoop-mapreduce-client-core diff --git a/hbase/hbase-side/hbase-async-side/pom.xml b/hbase/hbase-side/hbase-async-side/pom.xml index 522d503fd..2a18eeea6 100644 --- a/hbase/hbase-side/hbase-async-side/pom.xml +++ b/hbase/hbase-side/hbase-async-side/pom.xml @@ -44,6 +44,9 @@ org.slf4j:slf4j-log4j12 + org.apache.hadoop:hadoop-common + org.apache.hadoop:hadoop-auth + org.apache.hadoop:hadoop-mapreduce-client-core diff --git a/hbase/hbase-sink/pom.xml b/hbase/hbase-sink/pom.xml index da273d496..9ec6d161c 100644 --- a/hbase/hbase-sink/pom.xml +++ b/hbase/hbase-sink/pom.xml @@ -27,8 +27,10 @@ - org.apache.hadoop:hadoop-common org.slf4j:slf4j-log4j12 + org.apache.hadoop:hadoop-common + org.apache.hadoop:hadoop-auth + org.apache.hadoop:hadoop-mapreduce-client-core From 5479d822000009ee9d3aee156b38d4359f2be4e3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Sat, 20 Oct 2018 20:57:20 +0800 Subject: [PATCH 071/470] load data from hbase --- .../flink/sql/side/hbase/HbaseAllReqRow.java | 38 +++++++------------ 1 file changed, 13 insertions(+), 25 deletions(-) diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java index c35dea5b4..2b99a1325 100644 --- a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java @@ -2,6 +2,7 @@ import com.dtstack.flink.sql.side.*; import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo; +import org.apache.commons.collections.map.HashedMap; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; @@ -21,6 +22,7 @@ import java.sql.Timestamp; import java.util.*; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; public class HbaseAllReqRow extends AllReqRow { @@ -50,7 +52,7 @@ protected Row fillData(Row input, Object sideInput) { row.setField(entry.getKey(), obj); } - for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ + for(Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()){ if(sideInputList == null){ row.setField(entry.getKey(), null); }else{ @@ -112,17 +114,18 @@ private void loadData(Map> tmpCache) throws SQLExcep conn = ConnectionFactory.createConnection(conf); table = conn.getTable(TableName.valueOf(tableName)); resultScanner = table.getScanner(new Scan()); - List rows = new LinkedList<>(); for (Result r : resultScanner) { - for (Cell cell : r.listCells()){ - rows.add(cell.getRow().toString()); + Map kv = new HashedMap(); + for (Cell cell : r.listCells()) + { + String family = Bytes.toString(CellUtil.cloneFamily(cell)); + String qualifier = Bytes.toString(CellUtil.cloneQualifier(cell)); + String value = Bytes.toString(CellUtil.cloneValue(cell)); + StringBuilder key = new StringBuilder(); + key.append(family).append(":").append(qualifier); + kv.put(key.toString(), value); } - } - //根据表,rowkey查询值 - for (int i=0; i < rows.size(); i++){ - Get get = new Get(Bytes.toBytes(rows.get(i))); - Result result = table.get(get); - tmpCache.put(rows.get(i), result2Map(result)); + tmpCache.put(new String(r.getRow()), kv); } } catch (IOException e) { e.printStackTrace(); @@ -136,19 +139,4 @@ private void loadData(Map> tmpCache) throws SQLExcep } } } - - private static Map result2Map(Result result) { - Map ret = new HashMap(); - if (result != null && result.listCells() != null) { - for (Cell cell : result.listCells()) { - String family = Bytes.toString(CellUtil.cloneFamily(cell)); - String qualifier = Bytes.toString(CellUtil.cloneQualifier(cell)); - String value = Bytes.toString(CellUtil.cloneValue(cell)); - StringBuilder key = new StringBuilder(); - key.append(family).append(":").append(qualifier); - ret.put(key.toString(), value); - } - } - return ret; - } } From 8b377b5b8ebe72f2e583af47287efe503769231e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Sat, 20 Oct 2018 21:04:24 +0800 Subject: [PATCH 072/470] exclude hdfs jars --- hbase/hbase-side/hbase-all-side/pom.xml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/hbase/hbase-side/hbase-all-side/pom.xml b/hbase/hbase-side/hbase-all-side/pom.xml index f92fc6dd7..9b461ce94 100644 --- a/hbase/hbase-side/hbase-all-side/pom.xml +++ b/hbase/hbase-side/hbase-all-side/pom.xml @@ -42,7 +42,9 @@ - + org.apache.hadoop:hadoop-common + org.apache.hadoop:hadoop-auth + org.apache.hadoop:hadoop-mapreduce-client-core From 05e7caabbf7c904069dbe325257a4e2e07986439 Mon Sep 17 00:00:00 2001 From: chunguangli3 Date: Sun, 21 Oct 2018 22:16:25 +0800 Subject: [PATCH 073/470] on yarn job mode --- .../sql/launcher/ClusterClientFactory.java | 89 +++++++++++-------- 1 file changed, 53 insertions(+), 36 deletions(-) diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index aa8b5db5f..39bc01bbb 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -19,11 +19,8 @@ package com.dtstack.flink.sql.launcher; import org.apache.commons.lang.StringUtils; -import org.apache.flink.client.deployment.ClusterRetrieveException; -import org.apache.flink.client.deployment.StandaloneClusterDescriptor; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.StandaloneClusterClient; -import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; @@ -37,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.client.api.YarnClientApplication; import org.apache.hadoop.yarn.conf.YarnConfiguration; import java.io.File; import java.io.FilenameFilter; @@ -48,6 +46,8 @@ import java.util.Map; import java.util.Set; import com.dtstack.flink.sql.ClusterMode; +import org.apache.hadoop.yarn.exceptions.YarnException; +import java.io.IOException; /** * The Factory of ClusterClient @@ -61,8 +61,8 @@ public static ClusterClient createClusterClient(LauncherOptions launcherOptions) String mode = launcherOptions.getMode(); if(mode.equals(ClusterMode.standalone.name())) { return createStandaloneClient(launcherOptions); - } else if(mode.equals(ClusterMode.yarn.name())) { - return createYarnClient(launcherOptions); + } else if(mode.equals(ClusterMode.yarn.name()) || mode.equals(ClusterMode.yarnPer.name())) { + return createYarnClient(launcherOptions,mode); } throw new IllegalArgumentException("Unsupported cluster client type: "); } @@ -79,7 +79,7 @@ public static ClusterClient createStandaloneClient(LauncherOptions launcherOptio return clusterClient; } - public static ClusterClient createYarnClient(LauncherOptions launcherOptions) { + public static ClusterClient createYarnClient(LauncherOptions launcherOptions,String mode) { String flinkConfDir = launcherOptions.getFlinkconf(); Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); String yarnConfDir = launcherOptions.getYarnconf(); @@ -112,39 +112,16 @@ public boolean accept(File dir, String name) { haYarnConf(yarnConf); yarnClient.init(yarnConf); yarnClient.start(); - ApplicationId applicationId = null; - - Set set = new HashSet<>(); - set.add("Apache Flink"); - EnumSet enumSet = EnumSet.noneOf(YarnApplicationState.class); - enumSet.add(YarnApplicationState.RUNNING); - List reportList = yarnClient.getApplications(set, enumSet); - - int maxMemory = -1; - int maxCores = -1; - for(ApplicationReport report : reportList) { - if(!report.getName().startsWith("Flink session")){ - continue; - } - if(!report.getYarnApplicationState().equals(YarnApplicationState.RUNNING)) { - continue; - } - - int thisMemory = report.getApplicationResourceUsageReport().getNeededResources().getMemory(); - int thisCores = report.getApplicationResourceUsageReport().getNeededResources().getVirtualCores(); - if(thisMemory > maxMemory || thisMemory == maxMemory && thisCores > maxCores) { - maxMemory = thisMemory; - maxCores = thisCores; - applicationId = report.getApplicationId(); - } - - } - - if(StringUtils.isEmpty(applicationId.toString())) { - throw new RuntimeException("No flink session found on yarn cluster."); + ApplicationId applicationId = null; + if(mode.equals(ClusterMode.yarn.name())) {//on yarn cluster mode + applicationId = getYarnClusterApplicationId(yarnClient); + } else {//on yarn job mode + applicationId = createApplication(yarnClient); } + System.out.println("applicationId="+applicationId.toString()); + yarnClient.stop(); AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor(config, yarnConf, ".", yarnClient, false); ClusterClient clusterClient = clusterDescriptor.retrieve(applicationId); @@ -161,6 +138,46 @@ public boolean accept(File dir, String name) { throw new UnsupportedOperationException("Haven't been developed yet!"); } + private static ApplicationId createApplication(YarnClient yarnClient)throws IOException, YarnException { + YarnClientApplication app = yarnClient.createApplication(); + return app.getApplicationSubmissionContext().getApplicationId(); + } + private static ApplicationId getYarnClusterApplicationId(YarnClient yarnClient) throws Exception{ + ApplicationId applicationId = null; + + Set set = new HashSet<>(); + set.add("Apache Flink"); + EnumSet enumSet = EnumSet.noneOf(YarnApplicationState.class); + enumSet.add(YarnApplicationState.RUNNING); + List reportList = yarnClient.getApplications(set, enumSet); + + int maxMemory = -1; + int maxCores = -1; + for(ApplicationReport report : reportList) { + if(!report.getName().startsWith("Flink session")){ + continue; + } + + if(!report.getYarnApplicationState().equals(YarnApplicationState.RUNNING)) { + continue; + } + + int thisMemory = report.getApplicationResourceUsageReport().getNeededResources().getMemory(); + int thisCores = report.getApplicationResourceUsageReport().getNeededResources().getVirtualCores(); + if(thisMemory > maxMemory || thisMemory == maxMemory && thisCores > maxCores) { + maxMemory = thisMemory; + maxCores = thisCores; + applicationId = report.getApplicationId(); + } + + } + + if(StringUtils.isEmpty(applicationId.toString())) { + throw new RuntimeException("No flink session found on yarn cluster."); + } + return applicationId; + } + /** * 处理yarn HA的配置项 */ From 56a40812b7bc8fea65304edbb9b2f37817bc34df Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Sun, 21 Oct 2018 22:50:25 +0800 Subject: [PATCH 074/470] add function demo --- README.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/README.md b/README.md index 315a83d2c..8e552257c 100644 --- a/README.md +++ b/README.md @@ -138,6 +138,10 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack ## 3 样例 ``` + +CREATE (scala|table) FUNCTION CHARACTER_LENGTH WITH com.dtstack.Kun + + CREATE TABLE MyTable( name string, channel STRING, From 6728f98755bc46f6258748878b51234c5d2e9b24 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 22 Oct 2018 16:53:30 +0800 Subject: [PATCH 075/470] remove unused import --- .../com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java index ed7f611d5..b87e9eba7 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java @@ -3,17 +3,14 @@ import com.dtstack.flink.sql.side.AllReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; -import com.dtstack.flink.sql.threadFactory.DTThreadFactory; import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.calcite.sql.JoinType; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; -import org.apache.flink.configuration.Configuration; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.apache.flink.util.Collector; @@ -29,10 +26,6 @@ import java.util.Calendar; import java.util.List; import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; /** From 4cf8084d61bfdad016c9da62bdca73330767bdf2 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 22 Oct 2018 17:22:56 +0800 Subject: [PATCH 076/470] modify support create table field type(for match standard SQL) --- .../java/com/dtstack/flink/sql/util/ClassUtil.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java index 008913f8f..cac889ba3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.util; +import java.math.BigDecimal; import java.sql.Date; import java.sql.Timestamp; @@ -35,6 +36,8 @@ public static Class stringConvertClass(String str) { switch (str.toLowerCase()) { case "boolean": return Boolean.class; + + case "integer": case "int": return Integer.class; @@ -42,18 +45,15 @@ public static Class stringConvertClass(String str) { return Long.class; case "tinyint": - case "byte": return Byte.class; - case "short": case "smallint": return Short.class; - case "char": case "varchar": - case "string": return String.class; + case "real": case "float": return Float.class; @@ -66,6 +66,9 @@ public static Class stringConvertClass(String str) { case "timestamp": return Timestamp.class; + case "DECIMAL": + return BigDecimal.class; + } throw new RuntimeException("不支持 " + str + " 类型"); From ab84316efdd503e2a01d9c198e1bccbccdc34576 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 22 Oct 2018 17:30:35 +0800 Subject: [PATCH 077/470] modify support create table field type(for match standard SQL) --- README.md | 18 +++++++++--------- .../com/dtstack/flink/sql/util/ClassUtil.java | 2 +- docs/colType.md | 9 ++++----- 3 files changed, 14 insertions(+), 15 deletions(-) diff --git a/README.md b/README.md index a519a8ae7..9a9ab8ad3 100644 --- a/README.md +++ b/README.md @@ -139,9 +139,9 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack ``` CREATE TABLE MyTable( - name string, - channel STRING, - pv INT, + name varchar, + channel varchar, + pv int, xctime bigint, CHARACTER_LENGTH(channel) AS timeLeng )WITH( @@ -154,8 +154,8 @@ CREATE TABLE MyTable( ); CREATE TABLE MyResult( - channel VARCHAR, - pv VARCHAR + channel varchar, + pv varchar )WITH( type ='mysql', url ='jdbc:mysql://172.16.8.104:3306/test?charset=utf8', @@ -166,8 +166,8 @@ CREATE TABLE MyResult( ); CREATE TABLE workerinfo( - cast(logtime as TIMESTAMP)AS rtime, - cast(logtime)AS rtime + cast(logtime as TIMESTAMP) AS rtime, + cast(logtime) AS rtime )WITH( type ='hbase', zookeeperQuorum ='rdos1:2181', @@ -178,8 +178,8 @@ CREATE TABLE workerinfo( ); CREATE TABLE sideTable( - cf:name String as name, - cf:info String as info, + cf:name varchar as name, + cf:info varchar as info, PRIMARY KEY(name), PERIOD FOR SYSTEM_TIME )WITH( diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java index cac889ba3..4012e5c3b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java @@ -66,7 +66,7 @@ public static Class stringConvertClass(String str) { case "timestamp": return Timestamp.class; - case "DECIMAL": + case "decimal": return BigDecimal.class; } diff --git a/docs/colType.md b/docs/colType.md index 9a9593e7a..5819d3faf 100644 --- a/docs/colType.md +++ b/docs/colType.md @@ -2,15 +2,14 @@ | ------ | ----- | | boolean | Boolean | | int | Integer | +| integer| Integer | | bigint | Long | | tinyint | Byte | -| byte | Byte | -| short | Short | | smallint | Short| -| char | String| | varchar | String | -| string | String| +| real | Float | | float | Float| | double | Double| | date | Date | -| timestamp | Timestamp | \ No newline at end of file +| timestamp | Timestamp | +| decimal |BigDecimal| \ No newline at end of file From af379c70bb384b53e306b2baef6c54326f0b4d3d Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 22 Oct 2018 17:39:04 +0800 Subject: [PATCH 078/470] modify support create table field type(for match standard SQL) --- README.md | 18 +++++++++--------- .../com/dtstack/flink/sql/util/ClassUtil.java | 11 +++++++---- docs/colType.md | 9 ++++----- 3 files changed, 20 insertions(+), 18 deletions(-) diff --git a/README.md b/README.md index 315a83d2c..a8e23524a 100644 --- a/README.md +++ b/README.md @@ -139,9 +139,9 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack ``` CREATE TABLE MyTable( - name string, - channel STRING, - pv INT, + name varchar, + channel varchar, + pv int, xctime bigint, CHARACTER_LENGTH(channel) AS timeLeng )WITH( @@ -154,8 +154,8 @@ CREATE TABLE MyTable( ); CREATE TABLE MyResult( - channel VARCHAR, - pv VARCHAR + channel varchar, + pv varchar )WITH( type ='mysql', url ='jdbc:mysql://172.16.8.104:3306/test?charset=utf8', @@ -166,8 +166,8 @@ CREATE TABLE MyResult( ); CREATE TABLE workerinfo( - cast(logtime as TIMESTAMP)AS rtime, - cast(logtime)AS rtime + cast(logtime as TIMESTAMP) AS rtime, + cast(logtime) AS rtime )WITH( type ='hbase', zookeeperQuorum ='rdos1:2181', @@ -178,8 +178,8 @@ CREATE TABLE workerinfo( ); CREATE TABLE sideTable( - cf:name String as name, - cf:info String as info, + cf:name varchar as name, + cf:info varchar as info, PRIMARY KEY(name), PERIOD FOR SYSTEM_TIME )WITH( diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java index 008913f8f..4012e5c3b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.util; +import java.math.BigDecimal; import java.sql.Date; import java.sql.Timestamp; @@ -35,6 +36,8 @@ public static Class stringConvertClass(String str) { switch (str.toLowerCase()) { case "boolean": return Boolean.class; + + case "integer": case "int": return Integer.class; @@ -42,18 +45,15 @@ public static Class stringConvertClass(String str) { return Long.class; case "tinyint": - case "byte": return Byte.class; - case "short": case "smallint": return Short.class; - case "char": case "varchar": - case "string": return String.class; + case "real": case "float": return Float.class; @@ -66,6 +66,9 @@ public static Class stringConvertClass(String str) { case "timestamp": return Timestamp.class; + case "decimal": + return BigDecimal.class; + } throw new RuntimeException("不支持 " + str + " 类型"); diff --git a/docs/colType.md b/docs/colType.md index 9a9593e7a..5819d3faf 100644 --- a/docs/colType.md +++ b/docs/colType.md @@ -2,15 +2,14 @@ | ------ | ----- | | boolean | Boolean | | int | Integer | +| integer| Integer | | bigint | Long | | tinyint | Byte | -| byte | Byte | -| short | Short | | smallint | Short| -| char | String| | varchar | String | -| string | String| +| real | Float | | float | Float| | double | Double| | date | Date | -| timestamp | Timestamp | \ No newline at end of file +| timestamp | Timestamp | +| decimal |BigDecimal| \ No newline at end of file From 93c200cdf230dde6c963dec2166cf2bfb53f1288 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Mon, 22 Oct 2018 19:25:52 +0800 Subject: [PATCH 079/470] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 07c03899d..b24ef96e8 100644 --- a/README.md +++ b/README.md @@ -29,7 +29,7 @@ ### 1.2 执行环境 * Java: JDK8及以上 -* Flink集群: 1.4(单机模式不需要安装Flink集群) +* Flink集群: 1.4,1.5(单机模式不需要安装Flink集群) * 操作系统:理论上不限 ### 1.3 打包 From 84bdbe8a986ec34e090ad9f2b12731fb2290435a Mon Sep 17 00:00:00 2001 From: ideal Date: Mon, 22 Oct 2018 20:08:35 +0800 Subject: [PATCH 080/470] Add `with` support --- .../com/dtstack/flink/sql/side/SideSQLParser.java | 14 ++++++++++++++ .../com/dtstack/flink/sql/side/SideSqlExec.java | 6 ++++++ 2 files changed, 20 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 7f165ac30..076ce4006 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -29,8 +29,11 @@ import org.apache.calcite.sql.SqlJoin; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlNodeList; import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlSelect; +import org.apache.calcite.sql.SqlWith; +import org.apache.calcite.sql.SqlWithItem; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; import org.apache.calcite.sql.parser.SqlParserPos; @@ -66,6 +69,17 @@ public Queue getExeQueue(String exeSql, Set sideTableSet) throws private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue queueInfo){ SqlKind sqlKind = sqlNode.getKind(); switch (sqlKind){ + case WITH: { + SqlWith sqlWith = (SqlWith) sqlNode; + SqlNodeList sqlNodeList = sqlWith.withList; + for (SqlNode withAsTable : sqlNodeList) { + SqlWithItem sqlWithItem = (SqlWithItem) withAsTable; + parseSql(sqlWithItem.query, sideTableSet, queueInfo); + queueInfo.add(sqlWithItem); + } + parseSql(sqlWith.body, sideTableSet, queueInfo); + break; + } case INSERT: SqlNode sqlSource = ((SqlInsert)sqlNode).getSource(); return parseSql(sqlSource, sideTableSet, queueInfo); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 3aed54554..024da0f27 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -33,6 +33,7 @@ import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; import org.apache.calcite.sql.SqlSelect; +import org.apache.calcite.sql.SqlWithItem; import org.apache.calcite.sql.fun.SqlCase; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParserPos; @@ -101,6 +102,11 @@ public void exec(String sql, Map sideTableMap, StreamTabl Table table = tableEnv.sql(aliasInfo.getName()); tableEnv.registerTable(aliasInfo.getAlias(), table); localTableCache.put(aliasInfo.getAlias(), table); + } else if (pollSqlNode.getKind() == WITH_ITEM) { + SqlWithItem sqlWithItem = (SqlWithItem) pollSqlNode; + String TableAlias = sqlWithItem.name.toString(); + Table table = tableEnv.sqlQuery(sqlWithItem.query.toString()); + tableEnv.registerTable(TableAlias, table); } }else if (pollObj instanceof JoinInfo){ From 612ac9eceae1d0fa086deca6f9c21a62075e43a8 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Mon, 22 Oct 2018 21:08:21 +0800 Subject: [PATCH 081/470] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index b24ef96e8..3d106f781 100644 --- a/README.md +++ b/README.md @@ -47,7 +47,7 @@ mvn clean package -Dmaven.test.skip #### 1.4.1 启动命令 ``` -sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack/150_flinkplugin/sqlplugin -localSqlPluginPath D:\gitspace\flinkStreamSQL\plugins -mode yarn -flinkconf D:\flink_home\kudu150etc -yarnconf D:\hadoop\etc\hadoopkudu -confProp {"time.characteristic":"EventTime","sql.checkpoint.interval":10000} +sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack/150_flinkplugin/sqlplugin -localSqlPluginPath D:\gitspace\flinkStreamSQL\plugins -mode yarn -flinkconf D:\flink_home\kudu150etc -yarnconf D:\hadoop\etc\hadoopkudu -confProp \{\"time.characteristic\":\"EventTime\",\"sql.checkpoint.interval\":10000\} ``` #### 1.4.2 命令行参数选项 From 9ab0339380e3954373060afd33d17cb76024a759 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Mon, 22 Oct 2018 22:01:52 +0800 Subject: [PATCH 082/470] add prefix query function --- .../flink/sql/side/hbase/HbaseAllReqRow.java | 40 ++++++++++++++++--- 1 file changed, 34 insertions(+), 6 deletions(-) diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java index 2b99a1325..b5221d467 100644 --- a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java @@ -30,11 +30,20 @@ public class HbaseAllReqRow extends AllReqRow { private String tableName; + private Map aliasNameInversion; + private AtomicReference>> cacheRef = new AtomicReference<>(); public HbaseAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(new HbaseAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); tableName = ((HbaseSideTableInfo)sideTableInfo).getTableName(); + + HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo; + Map aliasNameRef = hbaseSideTableInfo.getAliasNameRef(); + aliasNameInversion = new HashMap<>(); + for(Map.Entry entry : aliasNameRef.entrySet()){ + aliasNameInversion.put(entry.getValue(), entry.getKey()); + } } @Override @@ -52,11 +61,12 @@ protected Row fillData(Row input, Object sideInput) { row.setField(entry.getKey(), obj); } - for(Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()){ + for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ if(sideInputList == null){ row.setField(entry.getKey(), null); }else{ - row.setField(entry.getKey(), sideInputList.get(entry.getValue())); + String key = sideInfo.getSideFieldNameIndex().get(entry.getKey()); + row.setField(entry.getKey(), sideInputList.get(key)); } } @@ -97,9 +107,26 @@ public void flatMap(Row value, Collector out) throws Exception { String rowKeyStr = ((HbaseAllSideInfo)sideInfo).getRowKeyBuilder().getRowKey(refData); - Object cacheList = cacheRef.get().get(rowKeyStr); - Row row = fillData(value, cacheList); - out.collect(row); + Map cacheList = null; + + SideTableInfo sideTableInfo = sideInfo.getSideTableInfo(); + HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo; + if (hbaseSideTableInfo.isPreRowKey()) + { + for (Map.Entry> entry : cacheRef.get().entrySet()){ + if (entry.getKey().startsWith(rowKeyStr)) + { + cacheList = cacheRef.get().get(entry.getKey()); + Row row = fillData(value, cacheList); + out.collect(row); + } + } + } else { + cacheList = cacheRef.get().get(rowKeyStr); + Row row = fillData(value, cacheList); + out.collect(row); + } + } private void loadData(Map> tmpCache) throws SQLException { @@ -123,7 +150,8 @@ private void loadData(Map> tmpCache) throws SQLExcep String value = Bytes.toString(CellUtil.cloneValue(cell)); StringBuilder key = new StringBuilder(); key.append(family).append(":").append(qualifier); - kv.put(key.toString(), value); + + kv.put(aliasNameInversion.get(key.toString().toUpperCase()), value); } tmpCache.put(new String(r.getRow()), kv); } From 273e1c8326fd0d08cb34b08a05edf52d6019a20a Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 24 Oct 2018 09:58:23 +0800 Subject: [PATCH 083/470] Update README.md --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 3d106f781..a26667b62 100644 --- a/README.md +++ b/README.md @@ -3,6 +3,7 @@ > > * 自定义create table 语法(包括源表,输出表,维表) > > * 自定义create function 语法 > > * 实现了流与维表的join +> > * 支持原生FLinkSQL所有的语法 # 已支持 * 源表:kafka 0.9,1.x版本 From 966b09b30f3752c8e3479b655966357bdf5ac036 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 24 Oct 2018 14:43:05 +0800 Subject: [PATCH 084/470] Update README.md --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index a26667b62..9e7cb736b 100644 --- a/README.md +++ b/README.md @@ -83,6 +83,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * **addjar** * 描述:扩展jar路径,当前主要是UDF定义的jar; + * 格式:json \[\"udf.jar\"\] * 必选:否 * 默认值:无 From 6ad1b50c84e801d8cad9e9ae64a549066a1e9b9e Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 24 Oct 2018 14:46:01 +0800 Subject: [PATCH 085/470] Update README.md --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 9e7cb736b..c34f8a6fb 100644 --- a/README.md +++ b/README.md @@ -48,7 +48,7 @@ mvn clean package -Dmaven.test.skip #### 1.4.1 启动命令 ``` -sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack/150_flinkplugin/sqlplugin -localSqlPluginPath D:\gitspace\flinkStreamSQL\plugins -mode yarn -flinkconf D:\flink_home\kudu150etc -yarnconf D:\hadoop\etc\hadoopkudu -confProp \{\"time.characteristic\":\"EventTime\",\"sql.checkpoint.interval\":10000\} +sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack/150_flinkplugin/sqlplugin -localSqlPluginPath D:\gitspace\flinkStreamSQL\plugins -addjar \["udf.jar\"\] -mode yarn -flinkconf D:\flink_home\kudu150etc -yarnconf D:\hadoop\etc\hadoopkudu -confProp \{\"time.characteristic\":\"EventTime\",\"sql.checkpoint.interval\":10000\} ``` #### 1.4.2 命令行参数选项 @@ -83,7 +83,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * **addjar** * 描述:扩展jar路径,当前主要是UDF定义的jar; - * 格式:json \[\"udf.jar\"\] + * 格式:json * 必选:否 * 默认值:无 From 31ccedac1d54c8b229e2b0099edf3856fe7efe2b Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 25 Oct 2018 21:16:27 +0800 Subject: [PATCH 086/470] add kafka partition lag metric; --- .../flink/sql/metric/MetricConstant.java | 6 ++ .../kafka/CustomerJsonDeserialization.java | 80 +++++++++++++++-- .../source/kafka/CustomerKafka09Consumer.java | 20 ++++- .../metric/KafkaTopicPartitionLagMetric.java | 30 +++++++ .../kafka/CustomerJsonDeserialization.java | 75 +++++++++++++++- .../kafka/CustomerKafka010Consumer.java | 21 ++++- .../metric/KafkaTopicPartitionLagMetric.java | 29 +++++++ .../kafka/CustomerJsonDeserialization.java | 87 ++++++++++++++++++- .../kafka/CustomerKafka011Consumer.java | 21 ++++- .../metric/KafkaTopicPartitionLagMetric.java | 30 +++++++ 10 files changed, 382 insertions(+), 17 deletions(-) create mode 100644 kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/metric/KafkaTopicPartitionLagMetric.java create mode 100644 kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/metric/KafkaTopicPartitionLagMetric.java create mode 100644 kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/metric/KafkaTopicPartitionLagMetric.java diff --git a/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java b/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java index 68e4c0a89..89c411479 100644 --- a/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java +++ b/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java @@ -48,4 +48,10 @@ public class MetricConstant { public static final String DT_NUM_RECORDS_OUT_RATE = "dtNumRecordsOutRate"; public static final String DT_EVENT_DELAY_GAUGE = "dtEventDelay"; + + public static final String DT_TOPIC_PARTITION_LAG_GAUGE = "dtTopicPartitionLag"; + + public static final String DT_TOPIC_GROUP = "topic"; + + public static final String DT_PARTITION_GROUP = "partition"; } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index dd058a25b..9deb04b94 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -16,35 +16,46 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.source.kafka; import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.lang.reflect.Field; import java.util.Iterator; +import java.util.Set; + +import static com.dtstack.flink.sql.metric.MetricConstant.*; /** * json string parsing custom - * Date: 2017/5/28 + * Date: 2018/09/18 * Company: www.dtstack.com - * @author xuchao + * @author sishu.yss */ public class CustomerJsonDeserialization extends AbsDeserialization { - private static final Logger logger = LoggerFactory.getLogger(CustomerJsonDeserialization.class); + private static final Logger LOG = LoggerFactory.getLogger(CustomerJsonDeserialization.class); - private static final long serialVersionUID = -2706012724306826506L; + private static final long serialVersionUID = 2385115520960444192L; private final ObjectMapper objectMapper = new ObjectMapper(); @@ -60,6 +71,10 @@ public class CustomerJsonDeserialization extends AbsDeserialization { /** Flag indicating whether to fail on a missing field. */ private boolean failOnMissingField; + private AbstractFetcher fetcher; + + private boolean firstMsg = true; + public CustomerJsonDeserialization(TypeInformation typeInfo){ this.typeInfo = typeInfo; @@ -70,9 +85,21 @@ public CustomerJsonDeserialization(TypeInformation typeInfo){ @Override public Row deserialize(byte[] message) throws IOException { + + if(firstMsg){ + try { + registerPtMetric(fetcher); + } catch (Exception e) { + LOG.error("register topic partition metric error.", e); + } + + firstMsg = false; + } + try { numInRecord.inc(); numInBytes.inc(message.length); + JsonNode root = objectMapper.readTree(message); Row row = new Row(fieldNames.length); for (int i = 0; i < fieldNames.length; i++) { @@ -119,4 +146,47 @@ public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { } + public void setFetcher(AbstractFetcher fetcher) { + this.fetcher = fetcher; + } + + + protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + + Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + consumerThreadField.setAccessible(true); + KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); + + Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); + hasAssignedPartitionsField.setAccessible(true); + + //wait until assignedPartitions + + boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); + + if(!hasAssignedPartitions){ + throw new RuntimeException("wait 50 secs, but not assignedPartitions"); + } + + Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); + consumerField.setAccessible(true); + + KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); + Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); + subscriptionStateField.setAccessible(true); + + //topic partitions lag + SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); + Set assignedPartitions = subscriptionState.assignedPartitions(); + for(TopicPartition topicPartition : assignedPartitions){ + MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) + .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); + metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); + } + + } + + private static String partitionLagMetricName(TopicPartition tp) { + return tp + ".records-lag"; + } } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java index 870cabfea..052b4f048 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java @@ -19,9 +19,18 @@ package com.dtstack.flink.sql.source.kafka; import com.dtstack.flink.sql.source.AbsDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; +import java.util.Map; import java.util.Properties; /** @@ -35,11 +44,11 @@ public class CustomerKafka09Consumer extends FlinkKafkaConsumer09 { private static final long serialVersionUID = 4451177393982291909L; - private AbsDeserialization customerJsonDeserialization; + private CustomerJsonDeserialization customerJsonDeserialization; public CustomerKafka09Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { super(topic, valueDeserializer, props); - this.customerJsonDeserialization = valueDeserializer; + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } @Override @@ -48,4 +57,11 @@ public void run(SourceContext sourceContext) throws Exception { customerJsonDeserialization.initMetric(); super.run(sourceContext); } + + @Override + protected AbstractFetcher createFetcher(SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); + customerJsonDeserialization.setFetcher(fetcher); + return fetcher; + } } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/metric/KafkaTopicPartitionLagMetric.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/metric/KafkaTopicPartitionLagMetric.java new file mode 100644 index 000000000..2519b4ad6 --- /dev/null +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/metric/KafkaTopicPartitionLagMetric.java @@ -0,0 +1,30 @@ +package com.dtstack.flink.sql.source.kafka.metric; + +import org.apache.flink.metrics.Gauge; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; + +/** + * Reason: + * Date: 2018/10/24 + * Company: www.dtstack.com + * @author xuchao + */ + +public class KafkaTopicPartitionLagMetric implements Gauge { + + private SubscriptionState subscriptionState; + + private TopicPartition tp; + + public KafkaTopicPartitionLagMetric(SubscriptionState subscriptionState, TopicPartition tp){ + this.subscriptionState = subscriptionState; + this.tp = tp; + } + + @Override + public Long getValue() { + //return subscriptionState.partitionLag(tp); + return 0L; + } +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index e0ef3ff05..4abcaf8c8 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -22,17 +22,27 @@ import com.dtstack.flink.sql.source.AbsDeserialization; -import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; +import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.lang.reflect.Field; import java.util.Iterator; +import java.util.Set; + +import static com.dtstack.flink.sql.metric.MetricConstant.*; /** * json string parsing custom @@ -43,7 +53,9 @@ public class CustomerJsonDeserialization extends AbsDeserialization { - private static final Logger logger = LoggerFactory.getLogger(CustomerJsonDeserialization.class); + private static final Logger LOG = LoggerFactory.getLogger(CustomerJsonDeserialization.class); + + private static final long serialVersionUID = 2385115520960444192L; private final ObjectMapper objectMapper = new ObjectMapper(); @@ -59,6 +71,10 @@ public class CustomerJsonDeserialization extends AbsDeserialization { /** Flag indicating whether to fail on a missing field. */ private boolean failOnMissingField; + private AbstractFetcher fetcher; + + private boolean firstMsg = true; + public CustomerJsonDeserialization(TypeInformation typeInfo){ this.typeInfo = typeInfo; @@ -69,6 +85,17 @@ public CustomerJsonDeserialization(TypeInformation typeInfo){ @Override public Row deserialize(byte[] message) throws IOException { + + if(firstMsg){ + try { + registerPtMetric(fetcher); + } catch (Exception e) { + LOG.error("register topic partition metric error.", e); + } + + firstMsg = false; + } + try { numInRecord.inc(); numInBytes.inc(message.length); @@ -118,4 +145,48 @@ public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { return null; } + + public void setFetcher(AbstractFetcher fetcher) { + this.fetcher = fetcher; + } + + + protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + + Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + consumerThreadField.setAccessible(true); + KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); + + Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); + hasAssignedPartitionsField.setAccessible(true); + + //wait until assignedPartitions + + boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); + + if(!hasAssignedPartitions){ + throw new RuntimeException("wait 50 secs, but not assignedPartitions"); + } + + Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); + consumerField.setAccessible(true); + + KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); + Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); + subscriptionStateField.setAccessible(true); + + //topic partitions lag + SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); + Set assignedPartitions = subscriptionState.assignedPartitions(); + for(TopicPartition topicPartition : assignedPartitions){ + MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) + .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); + metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); + } + + } + + private static String partitionLagMetricName(TopicPartition tp) { + return tp + ".records-lag"; + } } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka010Consumer.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka010Consumer.java index 862e3530f..37c1d6ab6 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka010Consumer.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka010Consumer.java @@ -19,9 +19,18 @@ package com.dtstack.flink.sql.source.kafka; import com.dtstack.flink.sql.source.AbsDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; +import java.util.Map; import java.util.Properties; /** @@ -35,11 +44,11 @@ public class CustomerKafka010Consumer extends FlinkKafkaConsumer010 { private static final long serialVersionUID = 4873757508981691375L; - private AbsDeserialization customerJsonDeserialization; + private CustomerJsonDeserialization customerJsonDeserialization; public CustomerKafka010Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { super(topic, valueDeserializer, props); - this.customerJsonDeserialization = valueDeserializer; + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } @Override @@ -48,4 +57,12 @@ public void run(SourceContext sourceContext) throws Exception { customerJsonDeserialization.initMetric(); super.run(sourceContext); } + + @Override + protected AbstractFetcher createFetcher(SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); + customerJsonDeserialization.setFetcher(fetcher); + return fetcher; + } + } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/metric/KafkaTopicPartitionLagMetric.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/metric/KafkaTopicPartitionLagMetric.java new file mode 100644 index 000000000..433699f7a --- /dev/null +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/metric/KafkaTopicPartitionLagMetric.java @@ -0,0 +1,29 @@ +package com.dtstack.flink.sql.source.kafka.metric; + +import org.apache.flink.metrics.Gauge; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; + +/** + * Reason: + * Date: 2018/10/24 + * Company: www.dtstack.com + * @author xuchao + */ + +public class KafkaTopicPartitionLagMetric implements Gauge { + + private SubscriptionState subscriptionState; + + private TopicPartition tp; + + public KafkaTopicPartitionLagMetric(SubscriptionState subscriptionState, TopicPartition tp){ + this.subscriptionState = subscriptionState; + this.tp = tp; + } + + @Override + public Long getValue() { + return subscriptionState.partitionLag(tp); + } +} diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index aa0a5fe2c..09350bfc1 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -16,23 +16,35 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.source.kafka; import com.dtstack.flink.sql.source.AbsDeserialization; -import org.apache.flink.api.common.serialization.AbstractDeserializationSchema; +import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.metrics.MetricGroup; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.lang.reflect.Field; import java.util.Iterator; +import java.util.Set; + +import static com.dtstack.flink.sql.metric.MetricConstant.DT_PARTITION_GROUP; +import static com.dtstack.flink.sql.metric.MetricConstant.DT_TOPIC_GROUP; +import static com.dtstack.flink.sql.metric.MetricConstant.DT_TOPIC_PARTITION_LAG_GAUGE; /** * json string parsing custom @@ -43,7 +55,9 @@ public class CustomerJsonDeserialization extends AbsDeserialization { - private static final Logger logger = LoggerFactory.getLogger(CustomerJsonDeserialization.class); + private static final Logger LOG = LoggerFactory.getLogger(CustomerJsonDeserialization.class); + + private static final long serialVersionUID = 2385115520960444192L; private final ObjectMapper objectMapper = new ObjectMapper(); @@ -59,6 +73,10 @@ public class CustomerJsonDeserialization extends AbsDeserialization { /** Flag indicating whether to fail on a missing field. */ private boolean failOnMissingField; + private AbstractFetcher fetcher; + + private boolean firstMsg = true; + public CustomerJsonDeserialization(TypeInformation typeInfo){ this.typeInfo = typeInfo; @@ -69,7 +87,21 @@ public CustomerJsonDeserialization(TypeInformation typeInfo){ @Override public Row deserialize(byte[] message) throws IOException { + + if(firstMsg){ + try { + registerPtMetric(fetcher); + } catch (Exception e) { + LOG.error("register topic partition metric error.", e); + } + + firstMsg = false; + } + try { + numInRecord.inc(); + numInBytes.inc(message.length); + JsonNode root = objectMapper.readTree(message); Row row = new Row(fieldNames.length); for (int i = 0; i < fieldNames.length; i++) { @@ -89,9 +121,12 @@ public Row deserialize(byte[] message) throws IOException { } } + numInResolveRecord.inc(); return row; } catch (Throwable t) { - throw new IOException("Failed to deserialize JSON object.", t); + //add metric of dirty data + dirtyDataCounter.inc(); + return new Row(fieldNames.length); } } @@ -112,4 +147,48 @@ public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { return null; } + + public void setFetcher(AbstractFetcher fetcher) { + this.fetcher = fetcher; + } + + + protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + + Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + consumerThreadField.setAccessible(true); + KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); + + Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); + hasAssignedPartitionsField.setAccessible(true); + + //wait until assignedPartitions + + boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); + + if(!hasAssignedPartitions){ + throw new RuntimeException("wait 50 secs, but not assignedPartitions"); + } + + Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); + consumerField.setAccessible(true); + + KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); + Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); + subscriptionStateField.setAccessible(true); + + //topic partitions lag + SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); + Set assignedPartitions = subscriptionState.assignedPartitions(); + for(TopicPartition topicPartition : assignedPartitions){ + MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) + .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); + metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); + } + + } + + private static String partitionLagMetricName(TopicPartition tp) { + return tp + ".records-lag"; + } } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka011Consumer.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka011Consumer.java index 390028d20..743e7f6d4 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka011Consumer.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka011Consumer.java @@ -19,10 +19,20 @@ package com.dtstack.flink.sql.source.kafka; import com.dtstack.flink.sql.source.AbsDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; +import java.util.Map; import java.util.Properties; /** @@ -37,11 +47,11 @@ public class CustomerKafka011Consumer extends FlinkKafkaConsumer011 { private static final long serialVersionUID = -2265366268827807739L; - private AbsDeserialization customerJsonDeserialization; + private CustomerJsonDeserialization customerJsonDeserialization; public CustomerKafka011Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { super(topic, valueDeserializer, props); - this.customerJsonDeserialization = valueDeserializer; + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } @Override @@ -50,4 +60,11 @@ public void run(SourceContext sourceContext) throws Exception { customerJsonDeserialization.initMetric(); super.run(sourceContext); } + + @Override + protected AbstractFetcher createFetcher(SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); + customerJsonDeserialization.setFetcher(fetcher); + return fetcher; + } } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/metric/KafkaTopicPartitionLagMetric.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/metric/KafkaTopicPartitionLagMetric.java new file mode 100644 index 000000000..7059690f8 --- /dev/null +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/metric/KafkaTopicPartitionLagMetric.java @@ -0,0 +1,30 @@ +package com.dtstack.flink.sql.source.kafka.metric; + +import org.apache.flink.metrics.Gauge; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.requests.IsolationLevel; + +/** + * Reason: + * Date: 2018/10/24 + * Company: www.dtstack.com + * @author xuchao + */ + +public class KafkaTopicPartitionLagMetric implements Gauge { + + private SubscriptionState subscriptionState; + + private TopicPartition tp; + + public KafkaTopicPartitionLagMetric(SubscriptionState subscriptionState, TopicPartition tp){ + this.subscriptionState = subscriptionState; + this.tp = tp; + } + + @Override + public Long getValue() { + return subscriptionState.partitionLag(tp, IsolationLevel.READ_UNCOMMITTED); + } +} From 03c4f299f80133b30dcd4c15d7a7a6e5abd8fb1f Mon Sep 17 00:00:00 2001 From: chunguangli3 Date: Mon, 29 Oct 2018 19:57:43 +0800 Subject: [PATCH 087/470] flink1.6+perjob --- bin/create.txt | 35 ++ core/pom.xml | 12 + .../flink/sql/MyLocalStreamEnvironment.java | 2 +- .../com/dtstack/flink/yarn/JobParameter.java | 130 ++++++ .../flink/yarn/YarnClusterConfiguration.java | 84 ++++ .../flink/yarn/YarnClusterDescriptor.java | 393 ++++++++++++++++++ .../sql/launcher/ClusterClientFactory.java | 150 ++++--- .../flink/sql/launcher/LauncherMain.java | 8 + pom.xml | 2 +- 9 files changed, 754 insertions(+), 62 deletions(-) create mode 100644 bin/create.txt create mode 100644 core/src/main/java/com/dtstack/flink/yarn/JobParameter.java create mode 100644 core/src/main/java/com/dtstack/flink/yarn/YarnClusterConfiguration.java create mode 100644 core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java diff --git a/bin/create.txt b/bin/create.txt new file mode 100644 index 000000000..819de4c41 --- /dev/null +++ b/bin/create.txt @@ -0,0 +1,35 @@ +CREATE TABLE MyTable( + name string, + channel STRING, + pv INT, + xctime bigint, + CHARACTER_LENGTH(channel) AS timeLeng + )WITH( + type ='kafka10', + bootstrapServers ='10.148.9.92:9092', + zookeeperQuorum ='10.148.9.92:2181/kafka_test_1.0', + offsetReset ='latest', + topic ='flume_sink_test', + parallelism ='1' + ); + +CREATE TABLE MyResult( + channel VARCHAR, + name VARCHAR + )WITH( + type ='mysql', + url ='jdbc:mysql://10.9.14.120:3306/test?charset=utf8&useSSL=true', + userName ='test', + password ='test123', + tableName ='MyResult', + parallelism ='1' + ); + +insert +into + MyResult +select + channel, + name +from + MyTable; diff --git a/core/pom.xml b/core/pom.xml index 1040fcea6..851566fe0 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -56,6 +56,18 @@ flink-streaming-scala_2.11 ${flink.version} + + + org.apache.flink + flink-shaded-hadoop2 + ${flink.version} + + + + org.apache.flink + flink-yarn_2.11 + ${flink.version} + diff --git a/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java b/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java index 54ddaa647..f41ecf00b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java +++ b/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java @@ -100,7 +100,7 @@ public JobExecutionResult execute(String jobName) throws Exception { Configuration configuration = new Configuration(); configuration.addAll(jobGraph.getJobConfiguration()); - configuration.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, -1L); + configuration.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "-1L"); configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, jobGraph.getMaximumParallelism()); // add (and override) the settings with what the user defined diff --git a/core/src/main/java/com/dtstack/flink/yarn/JobParameter.java b/core/src/main/java/com/dtstack/flink/yarn/JobParameter.java new file mode 100644 index 000000000..1879c6ea1 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/yarn/JobParameter.java @@ -0,0 +1,130 @@ +/* + * Copyright (C) 2018 The Sylph Authors + * + * Licensed 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 com.dtstack.flink.yarn; + +import java.util.Objects; +import java.util.Properties; + +public class JobParameter +{ + private int parallelism = 1; + private String queue = "default"; + private int taskManagerMemoryMb = 1024; + private int taskManagerCount = 1; + private int taskManagerSlots = 1; + private int jobManagerMemoryMb = 1024; + + public JobParameter() {} + + public JobParameter(Properties confProperties) { + this.parallelism = confProperties.getProperty("parallelism")==null?parallelism:Integer.parseInt(confProperties.getProperty("parallelism")); + this.queue = confProperties.getProperty("queue")==null?queue:confProperties.getProperty("queue"); + this.taskManagerMemoryMb = confProperties.getProperty("taskManagerMemoryMb")==null?taskManagerMemoryMb:Integer.parseInt(confProperties.getProperty("taskManagerMemoryMb")); + this.taskManagerCount = confProperties.getProperty("taskManagerCount")==null?taskManagerCount:Integer.parseInt(confProperties.getProperty("taskManagerCount")); + this.taskManagerSlots = confProperties.getProperty("taskManagerSlots")==null?taskManagerSlots:Integer.parseInt(confProperties.getProperty("taskManagerSlots")); + this.jobManagerMemoryMb = confProperties.getProperty("jobManagerMemoryMb")==null?jobManagerMemoryMb:Integer.parseInt(confProperties.getProperty("jobManagerMemoryMb")); + } + + public JobParameter(int parallelism, String queue, int taskManagerMemoryMb, int taskManagerCount, int taskManagerSlots, int jobManagerMemoryMb) { + this.parallelism = parallelism; + this.queue = queue; + this.taskManagerMemoryMb = taskManagerMemoryMb; + this.taskManagerCount = taskManagerCount; + this.taskManagerSlots = taskManagerSlots; + this.jobManagerMemoryMb = jobManagerMemoryMb; + } + + public void setQueue(String queue) + { + this.queue = queue; + } + + public void setTaskManagerCount(int taskManagerCount) + { + this.taskManagerCount = taskManagerCount; + } + + public void setTaskManagerMemoryMb(int taskManagerMemoryMb) + { + this.taskManagerMemoryMb = taskManagerMemoryMb; + } + + public void setTaskManagerSlots(int taskManagerSlots) + { + this.taskManagerSlots = taskManagerSlots; + } + + public void setJobManagerMemoryMb(int jobManagerMemoryMb) + { + this.jobManagerMemoryMb = jobManagerMemoryMb; + } + + public void setParallelism(int parallelism) + { + this.parallelism = parallelism; + } + + public int getParallelism() + { + return parallelism; + } + + public String getQueue() + { + return queue; + } + + public int getJobManagerMemoryMb() + { + return jobManagerMemoryMb; + } + + public int getTaskManagerSlots() + { + return taskManagerSlots; + } + + public int getTaskManagerCount() + { + return taskManagerCount; + } + + public int getTaskManagerMemoryMb() + { + return taskManagerMemoryMb; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + JobParameter jobParameter = (JobParameter) o; + return Objects.equals(this.queue, jobParameter.queue) && + Objects.equals(this.taskManagerCount, jobParameter.taskManagerCount) && + Objects.equals(this.taskManagerMemoryMb, jobParameter.taskManagerMemoryMb); + } + + @Override + public int hashCode() + { + return Objects.hash(queue, taskManagerMemoryMb, taskManagerCount); + } +} diff --git a/core/src/main/java/com/dtstack/flink/yarn/YarnClusterConfiguration.java b/core/src/main/java/com/dtstack/flink/yarn/YarnClusterConfiguration.java new file mode 100644 index 000000000..f2ccea2c4 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/yarn/YarnClusterConfiguration.java @@ -0,0 +1,84 @@ +/* + * Copyright (C) 2018 The Sylph Authors + * + * Licensed 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 com.dtstack.flink.yarn; + +import org.apache.flink.configuration.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +import java.util.Set; + +public class YarnClusterConfiguration { + /** + * The configuration used by YARN (i.e.,
yarn-site.xml
). + */ + private final YarnConfiguration yarnConf; + + /** + * The home directory of all job where all the temporary files for each jobs are stored. + */ + private final String appRootDir; + + /** + * The location of the Flink jar. + */ + private final Path flinkJar; + + /** + * Additional resources to be localized for both JobManager and TaskManager. + * They will NOT be added into the classpaths. + */ + private final Set resourcesToLocalize; + + /** + * flink conf + */ + private final Configuration flinkConfiguration; + + public YarnClusterConfiguration( + Configuration flinkConf, + YarnConfiguration conf, + String appRootDir, + Path flinkJar, + Set resourcesToLocalize) { + this.flinkConfiguration = flinkConf; + this.yarnConf = conf; + this.appRootDir = appRootDir; + this.flinkJar = flinkJar; + this.resourcesToLocalize = resourcesToLocalize; + } + + YarnConfiguration yarnConf() { + return yarnConf; + } + + public String appRootDir() { + return appRootDir; + } + + public Configuration flinkConfiguration() { + return flinkConfiguration; + } + + public Path flinkJar() { + return flinkJar; + } + + public Set resourcesToLocalize() { + return resourcesToLocalize; + } + +} diff --git a/core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java b/core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java new file mode 100644 index 000000000..d442d16f2 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java @@ -0,0 +1,393 @@ +/* + * Copyright (C) 2018 The Sylph Authors + * + * Licensed 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 com.dtstack.flink.yarn; + +import org.apache.flink.client.deployment.ClusterDeploymentException; +import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.rest.RestClusterClient; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.yarn.*; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.yarn.api.records.*; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.util.Records; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.net.URISyntaxException; +import java.util.*; +import java.util.concurrent.*; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.yarn.api.records.YarnApplicationState.NEW; + +public class YarnClusterDescriptor + extends AbstractYarnClusterDescriptor +{ + private static final String APPLICATION_TYPE = "58_FLINK"; + private static final Logger LOG = LoggerFactory.getLogger(YarnClusterDescriptor.class); + private static final int MAX_ATTEMPT = 1; + private static final long DEPLOY_TIMEOUT_MS = 600 * 1000; + private static final long RETRY_DELAY_MS = 250; + private static final ScheduledExecutorService YARN_POLL_EXECUTOR = Executors.newSingleThreadScheduledExecutor(); + + private final YarnClusterConfiguration clusterConf; + private final YarnClient yarnClient; + private final JobParameter appConf; + private final Path homedir; + private final ApplicationId yarnAppId; + private final String jobName; + private final Iterable userProvidedJars; + private Path flinkJar; + + public YarnClusterDescriptor( + final YarnClusterConfiguration clusterConf, + final YarnClient yarnClient, + final JobParameter appConf, + ApplicationId yarnAppId, + String jobName, + Iterable userProvidedJars) + { + super(clusterConf.flinkConfiguration(), clusterConf.yarnConf(), clusterConf.appRootDir(), yarnClient, false); + this.jobName = jobName; + this.clusterConf = clusterConf; + this.yarnClient = yarnClient; + this.appConf = appConf; + this.yarnAppId = yarnAppId; + this.userProvidedJars = userProvidedJars; + this.homedir = new Path(clusterConf.appRootDir(), yarnAppId.toString()); + } + + @Override + protected String getYarnSessionClusterEntrypoint() + { + return YarnApplicationMasterRunner.class.getName(); + } + + /** + * 提交到yarn时 任务启动入口类 + */ + @Override + protected String getYarnJobClusterEntrypoint() + { + return YarnApplicationMasterRunner.class.getName(); + } + + @Override + protected ClusterClient createYarnClusterClient(AbstractYarnClusterDescriptor descriptor, int numberTaskManagers, int slotsPerTaskManager, ApplicationReport report, Configuration flinkConfiguration, boolean perJobCluster) + throws Exception + { + return new RestClusterClient<>( + flinkConfiguration, + report.getApplicationId()); + } + + @Override + public YarnClient getYarnClient() + { + return this.yarnClient; + } + + public YarnClusterClient deploy() + { + ApplicationSubmissionContext context = Records.newRecord(ApplicationSubmissionContext.class); + context.setApplicationId(yarnAppId); + try { + ApplicationReport report = startAppMaster(context); + + Configuration conf = getFlinkConfiguration(); + conf.setString(JobManagerOptions.ADDRESS.key(), report.getHost()); + conf.setInteger(JobManagerOptions.PORT.key(), report.getRpcPort()); + + return new YarnClusterClient(this, + appConf.getTaskManagerCount(), + appConf.getTaskManagerSlots(), + report, conf, false); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + private ApplicationReport startAppMaster(ApplicationSubmissionContext appContext) + throws Exception + { + ApplicationId appId = appContext.getApplicationId(); + appContext.setMaxAppAttempts(MAX_ATTEMPT); + + Map localResources = new HashMap<>(); + Set shippedPaths = new HashSet<>(); + collectLocalResources(localResources, shippedPaths); + + final ContainerLaunchContext amContainer = setupApplicationMasterContainer( + getYarnJobClusterEntrypoint(), + false, + true, + false, + appConf.getJobManagerMemoryMb() + ); + + amContainer.setLocalResources(localResources); + + final String classPath = String.join(File.pathSeparator, localResources.keySet()); + + final String shippedFiles = shippedPaths.stream() + .map(path -> path.getName() + "=" + path) + .collect(Collectors.joining(",")); + + // Setup CLASSPATH and environment variables for ApplicationMaster + final Map appMasterEnv = setUpAmEnvironment( + appId, + classPath,shippedFiles, + //"","", + getDynamicPropertiesEncoded() + ); + + amContainer.setEnvironment(appMasterEnv); + + // Set up resource type requirements for ApplicationMaster + Resource capability = Records.newRecord(Resource.class); + capability.setMemory(appConf.getJobManagerMemoryMb()); //设置jobManneger + capability.setVirtualCores(1); //默认是1 + + appContext.setApplicationName(jobName); + appContext.setApplicationType(APPLICATION_TYPE); + appContext.setAMContainerSpec(amContainer); + appContext.setResource(capability); + //appContext.setApplicationTags(appConf.getAppTags()); + if (appConf.getQueue() != null) { + appContext.setQueue(appConf.getQueue()); + } + + LOG.info("Submitting application master {}", appId); + yarnClient.submitApplication(appContext); + + PollDeploymentStatus poll = new PollDeploymentStatus(appId); + YARN_POLL_EXECUTOR.submit(poll); + try { + return poll.result.get(); + } + catch (ExecutionException e) { + LOG.warn("Failed to deploy {}, cause: {}", appId.toString(), e.getCause()); + yarnClient.killApplication(appId); + throw (Exception) e.getCause(); + } + } + + private void collectLocalResources( + Map resources, + Set shippedPaths + ) + throws IOException, URISyntaxException + { + if(clusterConf.flinkJar() != null) { + Path flinkJar = clusterConf.flinkJar(); + LocalResource flinkJarResource = setupLocalResource(flinkJar, homedir, ""); //放到 Appid/根目录下 + this.flinkJar = ConverterUtils.getPathFromYarnURL(flinkJarResource.getResource()); + resources.put("flink.jar", flinkJarResource); + } + if(clusterConf.resourcesToLocalize() != null) { + for (Path p : clusterConf.resourcesToLocalize()) { //主要是 flink.jar log4f.propors 和 flink.yaml 三个文件 + LocalResource resource = setupLocalResource(p, homedir, ""); //这些需要放到根目录下 + resources.put(p.getName(), resource); + if ("log4j.properties".equals(p.getName())) { + shippedPaths.add(ConverterUtils.getPathFromYarnURL(resource.getResource())); + } + } + } + if(userProvidedJars != null) { + for (Path p : userProvidedJars) { + String name = p.getName(); + if (resources.containsKey(name)) { //这里当jar 有重复的时候 会抛出异常 + LOG.warn("Duplicated name in the shipped files {}", p); + } else { + LocalResource resource = setupLocalResource(p, homedir, "jars"); //这些放到 jars目录下 + resources.put(name, resource); + shippedPaths.add(ConverterUtils.getPathFromYarnURL(resource.getResource())); + } + } + } + } + + private LocalResource registerLocalResource(FileSystem fs, Path remoteRsrcPath) + throws IOException + { + LocalResource localResource = Records.newRecord(LocalResource.class); + FileStatus jarStat = fs.getFileStatus(remoteRsrcPath); + localResource.setResource(ConverterUtils.getYarnUrlFromURI(remoteRsrcPath.toUri())); + localResource.setSize(jarStat.getLen()); + localResource.setTimestamp(jarStat.getModificationTime()); + localResource.setType(LocalResourceType.FILE); + localResource.setVisibility(LocalResourceVisibility.APPLICATION); + return localResource; + } + + private LocalResource setupLocalResource( + Path localSrcPath, + Path homedir, + String relativeTargetPath) + throws IOException + { + if (new File(localSrcPath.toUri().getPath()).isDirectory()) { + throw new IllegalArgumentException("File to copy must not be a directory: " + + localSrcPath); + } + + // copy resource to HDFS + String suffix = "." + (relativeTargetPath.isEmpty() ? "" : "/" + relativeTargetPath) + + "/" + localSrcPath.getName(); + + Path dst = new Path(homedir, suffix); + + LOG.info("Uploading {}", dst); + + FileSystem hdfs = FileSystem.get(clusterConf.yarnConf()); + hdfs.copyFromLocalFile(false, true, localSrcPath, dst); + + // now create the resource instance + LocalResource resource = registerLocalResource(hdfs, dst); + return resource; + } + + private Map setUpAmEnvironment( + ApplicationId appId, + String amClassPath, + String shipFiles, + String dynamicProperties) + throws IOException, URISyntaxException + { + final Map appMasterEnv = new HashMap<>(); + + // set Flink app class path + appMasterEnv.put(YarnConfigKeys.ENV_FLINK_CLASSPATH, amClassPath); + + // set Flink on YARN internal configuration values + appMasterEnv.put(YarnConfigKeys.ENV_TM_COUNT, String.valueOf(appConf.getTaskManagerCount())); + appMasterEnv.put(YarnConfigKeys.ENV_TM_MEMORY, String.valueOf(appConf.getTaskManagerMemoryMb())); + appMasterEnv.put(YarnConfigKeys.ENV_SLOTS, String.valueOf(appConf.getTaskManagerSlots())); + appMasterEnv.put(YarnConfigKeys.FLINK_JAR_PATH, flinkJar.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_APP_ID, appId.toString()); + appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_HOME_DIR, homedir.toString()); //$home/.flink/appid 这个目录里面存放临时数据 + appMasterEnv.put(YarnConfigKeys.ENV_CLIENT_SHIP_FILES, shipFiles); + + appMasterEnv.put(YarnConfigKeys.ENV_DETACHED, String.valueOf(true)); //是否分离 分离就cluser模式 否则是client模式 + + appMasterEnv.put(YarnConfigKeys.ENV_HADOOP_USER_NAME, + UserGroupInformation.getCurrentUser().getUserName()); + + if (dynamicProperties != null) { + appMasterEnv.put(YarnConfigKeys.ENV_DYNAMIC_PROPERTIES, dynamicProperties); + } + + // set classpath from YARN configuration + Utils.setupYarnClassPath(clusterConf.yarnConf(), appMasterEnv); + + return appMasterEnv; + } + + /** + * flink 1.5 add + */ + @Override + public ClusterClient deployJobCluster(ClusterSpecification clusterSpecification, JobGraph jobGraph, boolean detached) + throws ClusterDeploymentException + { + throw new UnsupportedOperationException("this method have't support!"); + } + + private final class PollDeploymentStatus + implements Runnable + { + private final CompletableFuture result = new CompletableFuture<>(); + private final ApplicationId appId; + private YarnApplicationState lastAppState = NEW; + private long startTime; + + private PollDeploymentStatus(ApplicationId appId) + { + this.appId = appId; + } + + @Override + public void run() + { + if (startTime == 0) { + startTime = System.currentTimeMillis(); + } + + try { + ApplicationReport report = poll(); + if (report == null) { + YARN_POLL_EXECUTOR.schedule(this, RETRY_DELAY_MS, TimeUnit.MILLISECONDS); + } + else { + result.complete(report); + } + } + catch (YarnException | IOException e) { + result.completeExceptionally(e); + } + } + + private ApplicationReport poll() + throws IOException, YarnException + { + ApplicationReport report; + report = yarnClient.getApplicationReport(appId); + YarnApplicationState appState = report.getYarnApplicationState(); + LOG.debug("Application State: {}", appState); + + switch (appState) { + case FAILED: + case FINISHED: + //TODO: the finished state may be valid in flip-6 + case KILLED: + throw new IOException("The YARN application unexpectedly switched to state " + + appState + " during deployment. \n" + + "Diagnostics from YARN: " + report.getDiagnostics() + "\n" + + "If log aggregation is enabled on your cluster, use this command to further investigate the issue:\n" + + "yarn logs -applicationId " + appId); + //break .. + case RUNNING: + LOG.info("YARN application has been deployed successfully."); + break; + default: + if (appState != lastAppState) { + LOG.info("Deploying cluster, current state " + appState); + } + lastAppState = appState; + if (System.currentTimeMillis() - startTime > DEPLOY_TIMEOUT_MS) { + throw new RuntimeException(String.format("Deployment took more than %d seconds. " + + "Please check if the requested resources are available in the YARN cluster", DEPLOY_TIMEOUT_MS)); + } + return null; + } + return report; + } + } +} diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 39bc01bbb..9bd160966 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -18,6 +18,10 @@ package com.dtstack.flink.sql.launcher; +import com.dtstack.flink.sql.util.PluginUtil; +import com.dtstack.flink.yarn.JobParameter; +import com.dtstack.flink.yarn.YarnClusterConfiguration; +import org.apache.commons.io.Charsets; import org.apache.commons.lang.StringUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.StandaloneClusterClient; @@ -30,6 +34,7 @@ import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.YarnApplicationState; @@ -39,15 +44,16 @@ import java.io.File; import java.io.FilenameFilter; import java.net.InetSocketAddress; -import java.util.EnumSet; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.net.URLDecoder; +import java.util.*; + import com.dtstack.flink.sql.ClusterMode; import org.apache.hadoop.yarn.exceptions.YarnException; import java.io.IOException; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Objects.requireNonNull; /** * The Factory of ClusterClient @@ -81,61 +87,112 @@ public static ClusterClient createStandaloneClient(LauncherOptions launcherOptio public static ClusterClient createYarnClient(LauncherOptions launcherOptions,String mode) { String flinkConfDir = launcherOptions.getFlinkconf(); - Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); + Configuration flinkConf = GlobalConfiguration.loadConfiguration(flinkConfDir); String yarnConfDir = launcherOptions.getYarnconf(); - YarnConfiguration yarnConf = new YarnConfiguration(); + YarnConfiguration yarnConf; if(StringUtils.isNotBlank(yarnConfDir)) { try { - - config.setString(ConfigConstants.PATH_HADOOP_CONFIG, yarnConfDir); - FileSystem.initialize(config); + flinkConf.setString(ConfigConstants.PATH_HADOOP_CONFIG, yarnConfDir); + FileSystem.initialize(flinkConf); File dir = new File(yarnConfDir); if(dir.exists() && dir.isDirectory()) { - File[] xmlFileList = new File(yarnConfDir).listFiles(new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - if(name.endsWith(".xml")){ - return true; - } - return false; - } - }); - - if(xmlFileList != null) { - for(File xmlFile : xmlFileList) { - yarnConf.addResource(xmlFile.toURI().toURL()); - } - } + yarnConf = loadYarnConfiguration(yarnConfDir); YarnClient yarnClient = YarnClient.createYarnClient(); haYarnConf(yarnConf); yarnClient.init(yarnConf); yarnClient.start(); + String confProp = launcherOptions.getConfProp(); + confProp = URLDecoder.decode(confProp, Charsets.UTF_8.toString()); + System.out.println("confProp="+confProp); + Properties confProperties = PluginUtil.jsonStrToObject(confProp, Properties.class); + ApplicationId applicationId = null; + ClusterClient clusterClient = null; if(mode.equals(ClusterMode.yarn.name())) {//on yarn cluster mode applicationId = getYarnClusterApplicationId(yarnClient); - } else {//on yarn job mode + System.out.println("applicationId="+applicationId.toString()); + + AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( + flinkConf, yarnConf, ".", yarnClient, false); + clusterClient = clusterDescriptor.retrieve(applicationId); + + System.out.println("applicationId="+applicationId.toString()+" has retrieve!"); + } else {//on yarn per-job mode applicationId = createApplication(yarnClient); - } - System.out.println("applicationId="+applicationId.toString()); + System.out.println("applicationId="+applicationId.toString()); - yarnClient.stop(); + YarnClusterConfiguration clusterConf = getYarnClusterConfiguration(flinkConf,yarnConf,flinkConfDir); + //jobmanager+taskmanager param + JobParameter appConf = new JobParameter(confProperties); - AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor(config, yarnConf, ".", yarnClient, false); - ClusterClient clusterClient = clusterDescriptor.retrieve(applicationId); + com.dtstack.flink.yarn.YarnClusterDescriptor clusterDescriptor = new com.dtstack.flink.yarn.YarnClusterDescriptor( + clusterConf, yarnClient, appConf,applicationId, launcherOptions.getName(),null ); + clusterClient = clusterDescriptor.deploy(); + + System.out.println("applicationId="+applicationId.toString()+" has deploy!"); + } clusterClient.setDetached(true); + yarnClient.stop(); return clusterClient; } } catch(Exception e) { throw new RuntimeException(e); } } + throw new UnsupportedOperationException("Haven't been developed yet!"); + } + + private static YarnConfiguration loadYarnConfiguration(String yarnConfDir) + { + org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); + hadoopConf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem"); + Stream.of("yarn-site.xml", "core-site.xml", "hdfs-site.xml").forEach(file -> { + File site = new File(requireNonNull(yarnConfDir, "ENV HADOOP_CONF_DIR is not setting"), file); + if (site.exists() && site.isFile()) { + hadoopConf.addResource(new org.apache.hadoop.fs.Path(site.toURI())); + } + else { + throw new RuntimeException(site + " not exists"); + } + }); + + YarnConfiguration yarnConf = new YarnConfiguration(hadoopConf); + // try (PrintWriter pw = new PrintWriter(new FileWriter(yarnSite))) { //write local file + // yarnConf.writeXml(pw); + // } + return yarnConf; + } + public static YarnClusterConfiguration getYarnClusterConfiguration(Configuration flinkConf,YarnConfiguration yarnConf,String flinkConfDir) + { + Path flinkJar = new Path(getFlinkJarFile(flinkConfDir).toURI()); + @SuppressWarnings("ConstantConditions") final Set resourcesToLocalize = Stream + .of("flink-conf.yaml", "log4j.properties") + .map(x -> new Path(new File(flinkConfDir, x).toURI())) + .collect(Collectors.toSet()); - throw new UnsupportedOperationException("Haven't been developed yet!"); + String home = "hdfs:///tmp/flink/apps"; + //String home = "hdfs://test-cluster/tmp/flink/apps"; + return new YarnClusterConfiguration( + flinkConf, + yarnConf, + home, + flinkJar, + resourcesToLocalize); + } + + public static final String FLINK_DIST = "flink-dist"; + private static File getFlinkJarFile(String flinkConfDir) + { + String errorMessage = "error not search " + FLINK_DIST + "*.jar"; + File[] files = requireNonNull(new File(flinkConfDir, "/../lib").listFiles(), errorMessage); + Optional file = Arrays.stream(files) + .filter(f -> f.getName().startsWith(FLINK_DIST)).findFirst(); + return file.orElseThrow(() -> new IllegalArgumentException(errorMessage)); } private static ApplicationId createApplication(YarnClient yarnClient)throws IOException, YarnException { @@ -198,31 +255,4 @@ private static org.apache.hadoop.conf.Configuration haYarnConf(org.apache.hadoop return yarnConf; } - private static org.apache.hadoop.conf.Configuration getYarnConf(String yarnConfDir) { - org.apache.hadoop.conf.Configuration yarnConf = new YarnConfiguration(); - try { - - File dir = new File(yarnConfDir); - if(dir.exists() && dir.isDirectory()) { - File[] xmlFileList = new File(yarnConfDir).listFiles(new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - if(name.endsWith(".xml")){ - return true; - } - return false; - } - }); - if(xmlFileList != null) { - for(File xmlFile : xmlFileList) { - yarnConf.addResource(xmlFile.toURI().toURL()); - } - } - } - } catch(Exception e) { - throw new RuntimeException(e); - } - return yarnConf; - } - } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 55f085c98..957d8cb18 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -27,6 +27,9 @@ import java.io.File; import java.util.List; import com.dtstack.flink.sql.ClusterMode; +import org.apache.flink.client.program.PackagedProgramUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.table.shaded.org.apache.commons.lang.StringUtils; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.table.shaded.org.apache.commons.lang.BooleanUtils; @@ -64,8 +67,13 @@ public static void main(String[] args) throws Exception { if(StringUtils.isNotBlank(launcherOptions.getSavePointPath())){ program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getSavePointPath(), BooleanUtils.toBoolean(launcherOptions.getAllowNonRestoredState()))); } + //final JobGraph jobGraph; + //jobGraph = PackagedProgramUtils.createJobGraph(program, new Configuration(), 1); + //clusterClient.runDetached(jobGraph,null); clusterClient.run(program, 1); clusterClient.shutdown(); + + System.exit(0); } } } diff --git a/pom.xml b/pom.xml index 0922f739b..9b8f3ea45 100644 --- a/pom.xml +++ b/pom.xml @@ -22,7 +22,7 @@ UTF-8 - 1.5.3 + 1.6.0 From b72adbd571bea820e4d099592bfc39ff06f033a5 Mon Sep 17 00:00:00 2001 From: chunguangli3 Date: Mon, 29 Oct 2018 20:14:42 +0800 Subject: [PATCH 088/470] flink1.6+perjob --- bin/create.txt | 35 ------------------- .../sql/launcher/ClusterClientFactory.java | 1 - 2 files changed, 36 deletions(-) delete mode 100644 bin/create.txt diff --git a/bin/create.txt b/bin/create.txt deleted file mode 100644 index 819de4c41..000000000 --- a/bin/create.txt +++ /dev/null @@ -1,35 +0,0 @@ -CREATE TABLE MyTable( - name string, - channel STRING, - pv INT, - xctime bigint, - CHARACTER_LENGTH(channel) AS timeLeng - )WITH( - type ='kafka10', - bootstrapServers ='10.148.9.92:9092', - zookeeperQuorum ='10.148.9.92:2181/kafka_test_1.0', - offsetReset ='latest', - topic ='flume_sink_test', - parallelism ='1' - ); - -CREATE TABLE MyResult( - channel VARCHAR, - name VARCHAR - )WITH( - type ='mysql', - url ='jdbc:mysql://10.9.14.120:3306/test?charset=utf8&useSSL=true', - userName ='test', - password ='test123', - tableName ='MyResult', - parallelism ='1' - ); - -insert -into - MyResult -select - channel, - name -from - MyTable; diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 9bd160966..36ab25d2a 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -176,7 +176,6 @@ public static YarnClusterConfiguration getYarnClusterConfiguration(Configuration .collect(Collectors.toSet()); String home = "hdfs:///tmp/flink/apps"; - //String home = "hdfs://test-cluster/tmp/flink/apps"; return new YarnClusterConfiguration( flinkConf, yarnConf, From 39fb8f6110fe8c4a15a7c2db6ded739b65015bc3 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 1 Nov 2018 14:09:41 +0800 Subject: [PATCH 089/470] Update README.md --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index c34f8a6fb..b8d612809 100644 --- a/README.md +++ b/README.md @@ -18,6 +18,7 @@ * 增加oracle维表,结果表功能 * 增加SQlServer维表,结果表功能 * 增加kafka结果表功能 + * 增加SQL支持CEP ## 1 快速起步 ### 1.1 运行模式 From d8e3f4f803a1bb5105395314728970c8c832e9b5 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 2 Nov 2018 16:02:43 +0800 Subject: [PATCH 090/470] modify dbsink bug (not support TIMESTAMP) --- .../dtstack/flink/sql/side/operator/SideAsyncOperator.java | 5 ++++- .../main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java | 3 +++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java index 5e9da7605..df9a1c175 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java @@ -44,6 +44,9 @@ public class SideAsyncOperator { private static final String PATH_FORMAT = "%sasyncside"; + //TODO need to set by create table task + private static int asyncCapacity = 100; + private static AsyncReqRow loadAsyncReq(String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); @@ -60,7 +63,7 @@ public static DataStream getSideJoinDataStream(DataStream inputStream, String si List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { AsyncReqRow asyncDbReq = loadAsyncReq(sideType, sqlRootDir, rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); //TODO How much should be set for the degree of parallelism? Timeout? capacity settings? - return AsyncDataStream.orderedWait(inputStream, asyncDbReq, 10000, TimeUnit.MILLISECONDS, 10) + return AsyncDataStream.orderedWait(inputStream, asyncDbReq, 10000, TimeUnit.MILLISECONDS, asyncCapacity) .setParallelism(sideTableInfo.getParallelism()); } } diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java index e6d6ca9d2..83e76d77d 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java @@ -32,6 +32,7 @@ import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; +import java.sql.Timestamp; import java.sql.Types; import java.util.List; @@ -113,6 +114,8 @@ protected void buildSqlTypes(List fieldTypeArray){ tmpFieldsType[i] = Types.BINARY; }else if(fieldType.equals(Float.class.getName()) || fieldType.equals(Double.class.getName())){ tmpFieldsType[i] = Types.DOUBLE; + }else if (fieldType.equals(Timestamp.class.getName())){ + tmpFieldsType[i] = Types.TIMESTAMP; }else{ throw new RuntimeException("no support field type for sql. the input type:" + fieldType); } From f8850614488c465f1802b9c8df02faa0414742d5 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 2 Nov 2018 16:07:46 +0800 Subject: [PATCH 091/470] modify dbsink bug (not support TIMESTAMP) --- .../src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java index 26ab2da73..3c18bf0e6 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java @@ -32,6 +32,7 @@ import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; +import java.sql.Timestamp; import java.sql.Types; import java.util.List; @@ -113,6 +114,8 @@ protected void buildSqlTypes(List fieldTypeArray){ tmpFieldsType[i] = Types.BINARY; }else if(fieldType.equals(Float.class.getName()) || fieldType.equals(Double.class.getName())){ tmpFieldsType[i] = Types.DOUBLE; + }else if (fieldType.equals(Timestamp.class.getName())){ + tmpFieldsType[i] = Types.TIMESTAMP; }else{ throw new RuntimeException("no support field type for sql. the input type:" + fieldType); } From 08e66db51fe1f4580e768d7d690cf5584a736fed Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 2 Nov 2018 16:12:04 +0800 Subject: [PATCH 092/470] modify dbsink bug (not support TIMESTAMP) --- .../src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java index 26ab2da73..3c18bf0e6 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java @@ -32,6 +32,7 @@ import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; +import java.sql.Timestamp; import java.sql.Types; import java.util.List; @@ -113,6 +114,8 @@ protected void buildSqlTypes(List fieldTypeArray){ tmpFieldsType[i] = Types.BINARY; }else if(fieldType.equals(Float.class.getName()) || fieldType.equals(Double.class.getName())){ tmpFieldsType[i] = Types.DOUBLE; + }else if (fieldType.equals(Timestamp.class.getName())){ + tmpFieldsType[i] = Types.TIMESTAMP; }else{ throw new RuntimeException("no support field type for sql. the input type:" + fieldType); } From 595330075c29796885f632a7337f0f284b13610a Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 2 Nov 2018 16:13:04 +0800 Subject: [PATCH 093/470] Update README.md --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index b8d612809..74d798449 100644 --- a/README.md +++ b/README.md @@ -150,7 +150,7 @@ CREATE TABLE MyTable( channel varchar, pv int, xctime bigint, - CHARACTER_LENGTH(channel) AS timeLeng + CHARACTER_LENGTH(channel) AS timeLeng //自定义的函数 )WITH( type ='kafka09', bootstrapServers ='172.16.8.198:9092', @@ -188,7 +188,7 @@ CREATE TABLE sideTable( cf:name varchar as name, cf:info varchar as info, PRIMARY KEY(name), - PERIOD FOR SYSTEM_TIME + PERIOD FOR SYSTEM_TIME //维表标识 )WITH( type ='hbase', zookeeperQuorum ='rdos1:2181', From 1f4c1b962bb1df000afbad73612f360f2ca12480 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 3 Nov 2018 11:22:31 +0800 Subject: [PATCH 094/470] add license --- .../flink/sql/side/mysql/MysqlAllReqRow.java | 18 ++++++++++++++++++ .../flink/sql/side/mysql/MysqlAllSideInfo.java | 18 ++++++++++++++++++ 2 files changed, 36 insertions(+) diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java index ed7f611d5..a61d40b8a 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java @@ -1,3 +1,21 @@ +/* + * 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 com.dtstack.flink.sql.side.mysql; import com.dtstack.flink.sql.side.AllReqRow; diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java index 298311a05..2364aaac8 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java @@ -1,3 +1,21 @@ +/* + * 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 com.dtstack.flink.sql.side.mysql; import com.dtstack.flink.sql.side.FieldInfo; From a2f81a52d4182959103c91321dd46ebafb1ed584 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 3 Nov 2018 11:31:23 +0800 Subject: [PATCH 095/470] add license --- .../flink/sql/side/mysql/MysqlAllReqRow.java | 18 ++++++++++++++++++ .../flink/sql/side/mysql/MysqlAllSideInfo.java | 18 ++++++++++++++++++ 2 files changed, 36 insertions(+) diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java index 078ee589c..a8839e951 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java @@ -1,3 +1,21 @@ +/* + * 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 com.dtstack.flink.sql.side.mysql; import com.dtstack.flink.sql.side.AllReqRow; diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java index 298311a05..2364aaac8 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java @@ -1,3 +1,21 @@ +/* + * 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 com.dtstack.flink.sql.side.mysql; import com.dtstack.flink.sql.side.FieldInfo; From a250c0207201c834ee7e9e62311f6b19f8e033a4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Mon, 5 Nov 2018 22:37:51 +0800 Subject: [PATCH 096/470] add temporary table function --- .../main/java/com/dtstack/flink/sql/Main.java | 45 ++-- .../sql/parser/CreateTmpTableParser.java | 187 +++++++++++++ .../dtstack/flink/sql/parser/SqlParser.java | 50 +++- .../com/dtstack/flink/sql/parser/SqlTree.java | 20 ++ .../dtstack/flink/sql/side/SideSqlExec.java | 249 ++++++++++++------ hbase/hbase-side/hbase-all-side/pom.xml | 15 +- .../flink/sql/side/hbase/HbaseAllReqRow.java | 189 +++++++++++++ .../sql/side/hbase/HbaseAllSideInfo.java | 75 ++++++ .../sql/side/hbase/table/HbaseSideParser.java | 3 + 9 files changed, 726 insertions(+), 107 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java create mode 100644 hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java create mode 100644 hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 319e88a97..ba8218bea 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -22,20 +22,20 @@ import com.dtstack.flink.sql.classloader.DtClassLoader; import com.dtstack.flink.sql.enums.ECacheType; -import com.dtstack.flink.sql.parser.CreateFuncParser; -import com.dtstack.flink.sql.parser.InsertSqlParser; +import com.dtstack.flink.sql.parser.*; import com.dtstack.flink.sql.side.SideSqlExec; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; -import com.dtstack.flink.sql.parser.SqlParser; -import com.dtstack.flink.sql.parser.SqlTree; import com.dtstack.flink.sql.table.TableInfo; import com.dtstack.flink.sql.table.TargetTableInfo; import com.dtstack.flink.sql.sink.StreamSinkFactory; import com.dtstack.flink.sql.source.StreamSourceFactory; +import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.watermarker.WaterMarkerAssigner; import com.dtstack.flink.sql.util.FlinkUtil; import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.calcite.sql.SqlInsert; +import org.apache.calcite.sql.SqlNode; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; import org.apache.commons.cli.DefaultParser; @@ -171,6 +171,10 @@ public static void main(String[] args) throws Exception { SideSqlExec sideSqlExec = new SideSqlExec(); sideSqlExec.setLocalSqlPluginPath(localSqlPluginPath); + for (CreateTmpTableParser.SqlParserResult result : sqlTree.getTmpSqlList()) { + sideSqlExec.registerTmpTable(result, sideTableMap, tableEnv, registerTableCache); + } + for (InsertSqlParser.SqlParseResult result : sqlTree.getExecSqlList()) { if(LOG.isInfoEnabled()){ LOG.info("exe-sql:\n" + result.getExecSql()); @@ -178,19 +182,30 @@ public static void main(String[] args) throws Exception { boolean isSide = false; - for(String tableName : result.getSourceTableList()){ - if(sideTableMap.containsKey(tableName)){ - isSide = true; - break; + for (String tableName : result.getTargetTableList()) { + if (sqlTree.getTmpTableMap().containsKey(tableName)) { + CreateTmpTableParser.SqlParserResult tmp = sqlTree.getTmpTableMap().get(tableName); + String realSql = DtStringUtil.replaceIgnoreQuota(result.getExecSql(), "`", ""); + SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql).parseStmt(); + String tmpSql = ((SqlInsert) sqlNode).getSource().toString(); + tmp.setExecSql(tmpSql); + sideSqlExec.registerTmpTable(tmp, sideTableMap, tableEnv, registerTableCache); + } else { + for(String sourceTable : result.getSourceTableList()){ + if(sideTableMap.containsKey(sourceTable)){ + isSide = true; + break; + } + } + + if(isSide){ + //sql-dimensional table contains the dimension table of execution + sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache); + }else{ + tableEnv.sqlUpdate(result.getExecSql()); + } } } - - if(isSide){ - //sql-dimensional table contains the dimension table of execution - sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache); - }else{ - tableEnv.sqlUpdate(result.getExecSql()); - } } if(env instanceof MyLocalStreamEnvironment) { diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java new file mode 100644 index 000000000..89554013f --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -0,0 +1,187 @@ +/* + * 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 com.dtstack.flink.sql.parser; + +import org.apache.calcite.sql.*; +import org.apache.calcite.sql.parser.SqlParseException; +import org.apache.calcite.sql.parser.SqlParser; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static org.apache.calcite.sql.SqlKind.IDENTIFIER; + +public class CreateTmpTableParser implements IParser { + + //select table tableName as select + private static final String PATTERN_STR = "(?i)create\\s+view\\s+([^\\s]+)\\s+as\\s+select\\s+(.*)"; + + private static final String EMPTY_STR = "(?i)^\\screate\\s+view\\s+(\\S+)\\s*\\((.+)\\)$"; + + private static final Pattern NONEMPTYVIEW = Pattern.compile(PATTERN_STR); + + private static final Pattern EMPTYVIEW = Pattern.compile(EMPTY_STR); + + public static CreateTmpTableParser newInstance(){ + return new CreateTmpTableParser(); + } + + @Override + public boolean verify(String sql) { + if (Pattern.compile(EMPTY_STR).matcher(sql).find()){ + return true; + } + return NONEMPTYVIEW.matcher(sql).find(); + } + + @Override + public void parseSql(String sql, SqlTree sqlTree) { + if (NONEMPTYVIEW.matcher(sql).find()){ + Matcher matcher = NONEMPTYVIEW.matcher(sql); + String tableName = null; + String selectSql = null; + if(matcher.find()) { + tableName = matcher.group(1).toUpperCase(); + selectSql = "select " + matcher.group(2); + } + + SqlParser sqlParser = SqlParser.create(selectSql); + SqlNode sqlNode = null; + try { + sqlNode = sqlParser.parseStmt(); + } catch (SqlParseException e) { + throw new RuntimeException("", e); + } + + CreateTmpTableParser.SqlParserResult sqlParseResult = new CreateTmpTableParser.SqlParserResult(); + parseNode(sqlNode, sqlParseResult); + + sqlParseResult.setTableName(tableName); + sqlParseResult.setExecSql(selectSql.toUpperCase()); + sqlTree.addTmpSql(sqlParseResult); + sqlTree.addTmplTableInfo(tableName, sqlParseResult); + } else { + if (EMPTYVIEW.matcher(sql).find()) + { + Matcher matcher = EMPTYVIEW.matcher(sql); + String tableName = null; + String fieldsInfoStr = null; + if (matcher.find()){ + tableName = matcher.group(1).toUpperCase(); + fieldsInfoStr = matcher.group(2); + } + CreateTmpTableParser.SqlParserResult sqlParseResult = new CreateTmpTableParser.SqlParserResult(); + sqlParseResult.setFieldsInfoStr(fieldsInfoStr); + sqlParseResult.setTableName(tableName); + sqlTree.addTmplTableInfo(tableName, sqlParseResult); + } + + } + + } + + private static void parseNode(SqlNode sqlNode, CreateTmpTableParser.SqlParserResult sqlParseResult){ + SqlKind sqlKind = sqlNode.getKind(); + switch (sqlKind){ + case SELECT: + SqlNode sqlFrom = ((SqlSelect)sqlNode).getFrom(); + if(sqlFrom.getKind() == IDENTIFIER){ + sqlParseResult.addSourceTable(sqlFrom.toString()); + }else{ + parseNode(sqlFrom, sqlParseResult); + } + break; + case JOIN: + SqlNode leftNode = ((SqlJoin)sqlNode).getLeft(); + SqlNode rightNode = ((SqlJoin)sqlNode).getRight(); + + if(leftNode.getKind() == IDENTIFIER){ + sqlParseResult.addSourceTable(leftNode.toString()); + }else{ + parseNode(leftNode, sqlParseResult); + } + + if(rightNode.getKind() == IDENTIFIER){ + sqlParseResult.addSourceTable(rightNode.toString()); + }else{ + parseNode(rightNode, sqlParseResult); + } + break; + case AS: + //不解析column,所以 as 相关的都是表 + SqlNode identifierNode = ((SqlBasicCall)sqlNode).getOperands()[0]; + if(identifierNode.getKind() != IDENTIFIER){ + parseNode(identifierNode, sqlParseResult); + }else { + sqlParseResult.addSourceTable(identifierNode.toString()); + } + break; + default: + //do nothing + break; + } + } + + public static class SqlParserResult { + private String tableName; + + private String fieldsInfoStr; + + private String execSql; + + private List sourceTableList = Lists.newArrayList(); + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String getExecSql() { + return execSql; + } + + public void setExecSql(String execSql) { + this.execSql = execSql; + } + + public String getFieldsInfoStr() { + return fieldsInfoStr; + } + + public void setFieldsInfoStr(String fieldsInfoStr) { + this.fieldsInfoStr = fieldsInfoStr; + } + + public void addSourceTable(String sourceTable){ + sourceTableList.add(sourceTable); + } + + public List getSourceTableList() { + return sourceTableList; + } + + } +} \ No newline at end of file diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java index b5a1fbc74..4bd17c65e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java @@ -29,6 +29,7 @@ import org.apache.flink.shaded.guava18.com.google.common.base.Strings; import java.util.List; +import java.util.Set; /** * Reason: @@ -44,7 +45,7 @@ public class SqlParser { private static String LOCAL_SQL_PLUGIN_ROOT; private static List sqlParserList = Lists.newArrayList(CreateFuncParser.newInstance(), - CreateTableParser.newInstance(), InsertSqlParser.newInstance()); + CreateTableParser.newInstance(), InsertSqlParser.newInstance(), CreateTmpTableParser.newInstance()); public static void setLocalSqlPluginRoot(String localSqlPluginRoot){ LOCAL_SQL_PLUGIN_ROOT = localSqlPluginRoot; @@ -102,27 +103,48 @@ public static SqlTree parseSql(String sql) throws Exception { for(InsertSqlParser.SqlParseResult result : sqlTree.getExecSqlList()){ List sourceTableList = result.getSourceTableList(); List targetTableList = result.getTargetTableList(); + Set tmpTableList = sqlTree.getTmpTableMap().keySet(); for(String tableName : sourceTableList){ - CreateTableParser.SqlParserResult createTableResult = sqlTree.getPreDealTableMap().get(tableName); - if(createTableResult == null){ - throw new RuntimeException("can't find table " + tableName); + if (!tmpTableList.contains(tableName)){ + CreateTableParser.SqlParserResult createTableResult = sqlTree.getPreDealTableMap().get(tableName); + if(createTableResult == null){ + throw new RuntimeException("can't find table " + tableName); + } + + TableInfo tableInfo = TableInfoParserFactory.parseWithTableType(ETableType.SOURCE.getType(), + createTableResult, LOCAL_SQL_PLUGIN_ROOT); + sqlTree.addTableInfo(tableName, tableInfo); } - - TableInfo tableInfo = TableInfoParserFactory.parseWithTableType(ETableType.SOURCE.getType(), - createTableResult, LOCAL_SQL_PLUGIN_ROOT); - sqlTree.addTableInfo(tableName, tableInfo); } for(String tableName : targetTableList){ - CreateTableParser.SqlParserResult createTableResult = sqlTree.getPreDealTableMap().get(tableName); - if(createTableResult == null){ - throw new RuntimeException("can't find table " + tableName); + if (!tmpTableList.contains(tableName)){ + CreateTableParser.SqlParserResult createTableResult = sqlTree.getPreDealTableMap().get(tableName); + if(createTableResult == null){ + throw new RuntimeException("can't find table " + tableName); + } + + TableInfo tableInfo = TableInfoParserFactory.parseWithTableType(ETableType.SINK.getType(), + createTableResult, LOCAL_SQL_PLUGIN_ROOT); + sqlTree.addTableInfo(tableName, tableInfo); } + } + } - TableInfo tableInfo = TableInfoParserFactory.parseWithTableType(ETableType.SINK.getType(), - createTableResult, LOCAL_SQL_PLUGIN_ROOT); - sqlTree.addTableInfo(tableName, tableInfo); + for (CreateTmpTableParser.SqlParserResult result : sqlTree.getTmpSqlList()){ + List sourceTableList = result.getSourceTableList(); + for(String tableName : sourceTableList){ + if (!sqlTree.getTableInfoMap().keySet().contains(tableName)){ + CreateTableParser.SqlParserResult createTableResult = sqlTree.getPreDealTableMap().get(tableName); + if(createTableResult == null){ + throw new RuntimeException("can't find table " + tableName); + } + + TableInfo tableInfo = TableInfoParserFactory.parseWithTableType(ETableType.SOURCE.getType(), + createTableResult, LOCAL_SQL_PLUGIN_ROOT); + sqlTree.addTableInfo(tableName, tableInfo); + } } } diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java b/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java index 6ba524818..3ed37c51e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java @@ -45,6 +45,10 @@ public class SqlTree { private List execSqlList = Lists.newArrayList(); + private List tmpSqlList = Lists.newArrayList(); + + private Map tmpTableMap = Maps.newHashMap(); + public List getFunctionList() { return functionList; } @@ -53,6 +57,10 @@ public Map getPreDealTableMap() { return preDealTableMap; } + public Map getTmpTableMap() { + return tmpTableMap; + } + public List getExecSqlList() { return execSqlList; } @@ -65,10 +73,22 @@ public void addPreDealTableInfo(String tableName, CreateTableParser.SqlParserRes preDealTableMap.put(tableName, table); } + public void addTmplTableInfo(String tableName, CreateTmpTableParser.SqlParserResult table){ + tmpTableMap.put(tableName, table); + } + public void addExecSql(InsertSqlParser.SqlParseResult execSql){ execSqlList.add(execSql); } + public void addTmpSql(CreateTmpTableParser.SqlParserResult tmpSql){ + tmpSqlList.add(tmpSql); + } + + public List getTmpSqlList(){ + return tmpSqlList; + } + public Map getTableInfoMap() { return tableInfoMap; } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 3aed54554..7032bce05 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -21,8 +21,10 @@ package com.dtstack.flink.sql.side; import com.dtstack.flink.sql.enums.ECacheType; +import com.dtstack.flink.sql.parser.CreateTmpTableParser; import com.dtstack.flink.sql.side.operator.SideAsyncOperator; import com.dtstack.flink.sql.side.operator.SideWithAllCacheOperator; +import com.dtstack.flink.sql.util.ClassUtil; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlDataTypeSpec; import org.apache.calcite.sql.SqlIdentifier; @@ -46,10 +48,7 @@ import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.Queue; +import java.util.*; import static org.apache.calcite.sql.SqlKind.*; @@ -64,6 +63,8 @@ public class SideSqlExec { private String localSqlPluginPath = null; + private String tmpFields = null; + private SideSQLParser sideSQLParser = new SideSQLParser(); public void exec(String sql, Map sideTableMap, StreamTableEnvironment tableEnv, @@ -105,79 +106,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl }else if (pollObj instanceof JoinInfo){ preIsSideJoin = true; - JoinInfo joinInfo = (JoinInfo) pollObj; - - JoinScope joinScope = new JoinScope(); - JoinScope.ScopeChild leftScopeChild = new JoinScope.ScopeChild(); - leftScopeChild.setAlias(joinInfo.getLeftTableAlias()); - leftScopeChild.setTableName(joinInfo.getLeftTableName()); - - Table leftTable = getTableFromCache(localTableCache, joinInfo.getLeftTableAlias(), joinInfo.getLeftTableName()); - RowTypeInfo leftTypeInfo = new RowTypeInfo(leftTable.getSchema().getTypes(), leftTable.getSchema().getColumnNames()); - leftScopeChild.setRowTypeInfo(leftTypeInfo); - - JoinScope.ScopeChild rightScopeChild = new JoinScope.ScopeChild(); - rightScopeChild.setAlias(joinInfo.getRightTableAlias()); - rightScopeChild.setTableName(joinInfo.getRightTableName()); - SideTableInfo sideTableInfo = sideTableMap.get(joinInfo.getRightTableName()); - if(sideTableInfo == null){ - sideTableInfo = sideTableMap.get(joinInfo.getRightTableName()); - } - - if(sideTableInfo == null){ - throw new RuntimeException("can't not find side table:" + joinInfo.getRightTableName()); - } - - if(!checkJoinCondition(joinInfo.getCondition(), joinInfo.getRightTableAlias(), sideTableInfo.getPrimaryKeys())){ - throw new RuntimeException("ON condition must contain all equal fields!!!"); - } - - rightScopeChild.setRowTypeInfo(sideTableInfo.getRowTypeInfo()); - - joinScope.addScope(leftScopeChild); - joinScope.addScope(rightScopeChild); - - //获取两个表的所有字段 - List sideJoinFieldInfo = ParserJoinField.getRowTypeInfo(joinInfo.getSelectNode(), joinScope, true); - - String leftTableAlias = joinInfo.getLeftTableAlias(); - Table targetTable = localTableCache.get(leftTableAlias); - if(targetTable == null){ - targetTable = localTableCache.get(joinInfo.getLeftTableName()); - } - - RowTypeInfo typeInfo = new RowTypeInfo(targetTable.getSchema().getTypes(), targetTable.getSchema().getColumnNames()); - DataStream adaptStream = tableEnv.toAppendStream(targetTable, org.apache.flink.types.Row.class); - - //join side table before keyby ===> Reducing the size of each dimension table cache of async - if(sideTableInfo.isPartitionedJoin()){ - List leftJoinColList = getConditionFields(joinInfo.getCondition(), joinInfo.getLeftTableAlias()); - String[] leftJoinColArr = new String[leftJoinColList.size()]; - leftJoinColArr = leftJoinColList.toArray(leftJoinColArr); - adaptStream = adaptStream.keyBy(leftJoinColArr); - } - - DataStream dsOut = null; - if(ECacheType.ALL.name().equalsIgnoreCase(sideTableInfo.getCacheType())){ - dsOut = SideWithAllCacheOperator.getSideJoinDataStream(adaptStream, sideTableInfo.getType(), localSqlPluginPath, typeInfo, joinInfo, sideJoinFieldInfo, sideTableInfo); - }else{ - dsOut = SideAsyncOperator.getSideJoinDataStream(adaptStream, sideTableInfo.getType(), localSqlPluginPath, typeInfo, joinInfo, sideJoinFieldInfo, sideTableInfo); - } - - HashBasedTable mappingTable = HashBasedTable.create(); - RowTypeInfo sideOutTypeInfo = buildOutRowTypeInfo(sideJoinFieldInfo, mappingTable); - dsOut.getTransformation().setOutputType(sideOutTypeInfo); - String targetTableName = joinInfo.getNewTableName(); - String targetTableAlias = joinInfo.getNewTableAlias(); - - FieldReplaceInfo replaceInfo = new FieldReplaceInfo(); - replaceInfo.setMappingTable(mappingTable); - replaceInfo.setTargetTableName(targetTableName); - replaceInfo.setTargetTableAlias(targetTableAlias); - - replaceInfoList.add(replaceInfo); - - tableEnv.registerDataStream(joinInfo.getNewTableName(), dsOut, String.join(",", sideOutTypeInfo.getFieldNames())); + jionFun(pollObj, localTableCache, sideTableMap, tableEnv, replaceInfoList); } } @@ -563,4 +492,170 @@ public List getConditionFields(SqlNode conditionNode, String specifyTabl return conditionFields; } + + public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, + Map sideTableMap, StreamTableEnvironment tableEnv, + Map tableCache) + throws Exception { + + if(localSqlPluginPath == null){ + throw new RuntimeException("need to set localSqlPluginPath"); + } + + Map localTableCache = Maps.newHashMap(tableCache); + Queue exeQueue = sideSQLParser.getExeQueue(result.getExecSql(), sideTableMap.keySet()); + Object pollObj = null; + + //need clean + boolean preIsSideJoin = false; + List replaceInfoList = Lists.newArrayList(); + + while((pollObj = exeQueue.poll()) != null){ + + if(pollObj instanceof SqlNode){ + SqlNode pollSqlNode = (SqlNode) pollObj; + + if(preIsSideJoin){ + preIsSideJoin = false; + for(FieldReplaceInfo replaceInfo : replaceInfoList){ + replaceFieldName(pollSqlNode, replaceInfo.getMappingTable(), replaceInfo.getTargetTableName(), replaceInfo.getTargetTableAlias()); + } + } + + if(pollSqlNode.getKind() == INSERT){ + tableEnv.sqlUpdate(pollSqlNode.toString()); + }else if(pollSqlNode.getKind() == AS){ + AliasInfo aliasInfo = parseASNode(pollSqlNode); + Table table = tableEnv.sql(aliasInfo.getName()); + tableEnv.registerTable(aliasInfo.getAlias(), table); + localTableCache.put(aliasInfo.getAlias(), table); + } else if (pollSqlNode.getKind() == SELECT){ + Table table = tableEnv.sqlQuery(pollObj.toString()); + if (result.getFieldsInfoStr() == null){ + tableEnv.registerTable(result.getTableName(), table); + } else { + if (checkFieldsInfo(result, table)){ + table = table.as(tmpFields); + tableEnv.registerTable(result.getTableName(), table); + } else { + throw new RuntimeException("Fields mismatch"); + } + } + + } + + }else if (pollObj instanceof JoinInfo){ + preIsSideJoin = true; + jionFun(pollObj, localTableCache, sideTableMap, tableEnv, replaceInfoList); + } + } + } + private void jionFun(Object pollObj, Map localTableCache, + Map sideTableMap, StreamTableEnvironment tableEnv, + List replaceInfoList) throws Exception{ + JoinInfo joinInfo = (JoinInfo) pollObj; + + JoinScope joinScope = new JoinScope(); + JoinScope.ScopeChild leftScopeChild = new JoinScope.ScopeChild(); + leftScopeChild.setAlias(joinInfo.getLeftTableAlias()); + leftScopeChild.setTableName(joinInfo.getLeftTableName()); + + Table leftTable = getTableFromCache(localTableCache, joinInfo.getLeftTableAlias(), joinInfo.getLeftTableName()); + RowTypeInfo leftTypeInfo = new RowTypeInfo(leftTable.getSchema().getTypes(), leftTable.getSchema().getColumnNames()); + leftScopeChild.setRowTypeInfo(leftTypeInfo); + + JoinScope.ScopeChild rightScopeChild = new JoinScope.ScopeChild(); + rightScopeChild.setAlias(joinInfo.getRightTableAlias()); + rightScopeChild.setTableName(joinInfo.getRightTableName()); + SideTableInfo sideTableInfo = sideTableMap.get(joinInfo.getRightTableName()); + if(sideTableInfo == null){ + sideTableInfo = sideTableMap.get(joinInfo.getRightTableName()); + } + + if(sideTableInfo == null){ + throw new RuntimeException("can't not find side table:" + joinInfo.getRightTableName()); + } + + if(!checkJoinCondition(joinInfo.getCondition(), joinInfo.getRightTableAlias(), sideTableInfo.getPrimaryKeys())){ + throw new RuntimeException("ON condition must contain all equal fields!!!"); + } + + rightScopeChild.setRowTypeInfo(sideTableInfo.getRowTypeInfo()); + + joinScope.addScope(leftScopeChild); + joinScope.addScope(rightScopeChild); + + //获取两个表的所有字段 + List sideJoinFieldInfo = ParserJoinField.getRowTypeInfo(joinInfo.getSelectNode(), joinScope, true); + + String leftTableAlias = joinInfo.getLeftTableAlias(); + Table targetTable = localTableCache.get(leftTableAlias); + if(targetTable == null){ + targetTable = localTableCache.get(joinInfo.getLeftTableName()); + } + + RowTypeInfo typeInfo = new RowTypeInfo(targetTable.getSchema().getTypes(), targetTable.getSchema().getColumnNames()); + DataStream adaptStream = tableEnv.toAppendStream(targetTable, org.apache.flink.types.Row.class); + + //join side table before keyby ===> Reducing the size of each dimension table cache of async + if(sideTableInfo.isPartitionedJoin()){ + List leftJoinColList = getConditionFields(joinInfo.getCondition(), joinInfo.getLeftTableAlias()); + String[] leftJoinColArr = new String[leftJoinColList.size()]; + leftJoinColArr = leftJoinColList.toArray(leftJoinColArr); + adaptStream = adaptStream.keyBy(leftJoinColArr); + } + + DataStream dsOut = null; + if(ECacheType.ALL.name().equalsIgnoreCase(sideTableInfo.getCacheType())){ + dsOut = SideWithAllCacheOperator.getSideJoinDataStream(adaptStream, sideTableInfo.getType(), localSqlPluginPath, typeInfo, joinInfo, sideJoinFieldInfo, sideTableInfo); + }else{ + dsOut = SideAsyncOperator.getSideJoinDataStream(adaptStream, sideTableInfo.getType(), localSqlPluginPath, typeInfo, joinInfo, sideJoinFieldInfo, sideTableInfo); + } + + HashBasedTable mappingTable = HashBasedTable.create(); + RowTypeInfo sideOutTypeInfo = buildOutRowTypeInfo(sideJoinFieldInfo, mappingTable); + dsOut.getTransformation().setOutputType(sideOutTypeInfo); + String targetTableName = joinInfo.getNewTableName(); + String targetTableAlias = joinInfo.getNewTableAlias(); + + FieldReplaceInfo replaceInfo = new FieldReplaceInfo(); + replaceInfo.setMappingTable(mappingTable); + replaceInfo.setTargetTableName(targetTableName); + replaceInfo.setTargetTableAlias(targetTableAlias); + + replaceInfoList.add(replaceInfo); + + if (!tableEnv.isRegistered(joinInfo.getNewTableName())){ + tableEnv.registerDataStream(joinInfo.getNewTableName(), dsOut, String.join(",", sideOutTypeInfo.getFieldNames())); + } + } + + private boolean checkFieldsInfo(CreateTmpTableParser.SqlParserResult result, Table table){ + List fieldNames = new LinkedList<>(); + String fieldsInfo = result.getFieldsInfoStr(); + String[] fields = fieldsInfo.split(","); + for (int i=0; i < fields.length; i++) + { + String[] filed = fields[i].split("\\s"); + if (filed.length < 2 || fields.length != table.getSchema().getColumnNames().length){ + return false; + } else { + String[] filedNameArr = new String[filed.length - 1]; + System.arraycopy(filed, 0, filedNameArr, 0, filed.length - 1); + String fieldName = String.join(" ", filedNameArr); + fieldNames.add(fieldName.toUpperCase()); + String fieldType = filed[filed.length - 1 ].trim(); + Class fieldClass = ClassUtil.stringConvertClass(fieldType); + Class tableField = table.getSchema().getType(i).get().getTypeClass(); + if (fieldClass == tableField){ + continue; + } else { + return false; + } + } + } + tmpFields = String.join(",", fieldNames); + return true; + } + } diff --git a/hbase/hbase-side/hbase-all-side/pom.xml b/hbase/hbase-side/hbase-all-side/pom.xml index f70bdacdc..9b461ce94 100644 --- a/hbase/hbase-side/hbase-all-side/pom.xml +++ b/hbase/hbase-side/hbase-all-side/pom.xml @@ -13,6 +13,20 @@ sql.side.all.hbase hbase-all-side + + + com.dtstack.flink + sql.side.hbase.core + 1.0-SNAPSHOT + + + + org.apache.hbase + hbase-client + 1.3.1 + + + @@ -28,7 +42,6 @@ - org.slf4j:slf4j-log4j12 org.apache.hadoop:hadoop-common org.apache.hadoop:hadoop-auth org.apache.hadoop:hadoop-mapreduce-client-core diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java new file mode 100644 index 000000000..29fca4bee --- /dev/null +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java @@ -0,0 +1,189 @@ +/* + * 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 com.dtstack.flink.sql.side.hbase; + +import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo; +import org.apache.commons.collections.map.HashedMap; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.*; +import org.apache.hadoop.hbase.util.Bytes; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.sql.SQLException; +import java.sql.Timestamp; +import java.util.*; +import java.util.concurrent.atomic.AtomicReference; + +public class HbaseAllReqRow extends AllReqRow { + + private static final Logger LOG = LoggerFactory.getLogger(HbaseAllReqRow.class); + + private String tableName; + + private Map aliasNameInversion; + + private AtomicReference>> cacheRef = new AtomicReference<>(); + + public HbaseAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new HbaseAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + tableName = ((HbaseSideTableInfo)sideTableInfo).getTableName(); + + HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo; + Map aliasNameRef = hbaseSideTableInfo.getAliasNameRef(); + aliasNameInversion = new HashMap<>(); + for(Map.Entry entry : aliasNameRef.entrySet()){ + aliasNameInversion.put(entry.getValue(), entry.getKey()); + } + } + + @Override + protected Row fillData(Row input, Object sideInput) { + Map sideInputList = (Map) sideInput; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. + if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ + obj = ((Timestamp)obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ + if(sideInputList == null){ + row.setField(entry.getKey(), null); + }else{ + String key = sideInfo.getSideFieldNameIndex().get(entry.getKey()); + row.setField(entry.getKey(), sideInputList.get(key)); + } + } + + return row; + } + + @Override + protected void initCache() throws SQLException { + Map> newCache = Maps.newConcurrentMap(); + cacheRef.set(newCache); + loadData(newCache); + } + + @Override + protected void reloadCache() { + Map> newCache = Maps.newConcurrentMap(); + try { + loadData(newCache); + } catch (SQLException e) { + LOG.error("", e); + } + + cacheRef.set(newCache); + LOG.info("----- HBase all cacheRef reload end:{}", Calendar.getInstance()); + } + + @Override + public void flatMap(Row value, Collector out) throws Exception { + Map refData = Maps.newHashMap(); + for (int i = 0; i < sideInfo.getEqualValIndex().size(); i++) { + Integer conValIndex = sideInfo.getEqualValIndex().get(i); + Object equalObj = value.getField(conValIndex); + if(equalObj == null){ + out.collect(null); + } + refData.put(sideInfo.getEqualFieldList().get(i), equalObj); + } + + String rowKeyStr = ((HbaseAllSideInfo)sideInfo).getRowKeyBuilder().getRowKey(refData); + + Map cacheList = null; + + SideTableInfo sideTableInfo = sideInfo.getSideTableInfo(); + HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo; + if (hbaseSideTableInfo.isPreRowKey()) + { + for (Map.Entry> entry : cacheRef.get().entrySet()){ + if (entry.getKey().startsWith(rowKeyStr)) + { + cacheList = cacheRef.get().get(entry.getKey()); + Row row = fillData(value, cacheList); + out.collect(row); + } + } + } else { + cacheList = cacheRef.get().get(rowKeyStr); + Row row = fillData(value, cacheList); + out.collect(row); + } + + } + + private void loadData(Map> tmpCache) throws SQLException { + SideTableInfo sideTableInfo = sideInfo.getSideTableInfo(); + HbaseSideTableInfo hbaseSideTableInfo = (HbaseSideTableInfo) sideTableInfo; + Configuration conf = new Configuration(); + conf.set("hbase.zookeeper.quorum", hbaseSideTableInfo.getHost()); + Connection conn = null; + Table table = null; + ResultScanner resultScanner = null; + try { + conn = ConnectionFactory.createConnection(conf); + table = conn.getTable(TableName.valueOf(tableName)); + resultScanner = table.getScanner(new Scan()); + for (Result r : resultScanner) { + Map kv = new HashedMap(); + for (Cell cell : r.listCells()) + { + String family = Bytes.toString(CellUtil.cloneFamily(cell)); + String qualifier = Bytes.toString(CellUtil.cloneQualifier(cell)); + String value = Bytes.toString(CellUtil.cloneValue(cell)); + StringBuilder key = new StringBuilder(); + key.append(family).append(":").append(qualifier); + + kv.put(aliasNameInversion.get(key.toString().toUpperCase()), value); + } + tmpCache.put(new String(r.getRow()), kv); + } + } catch (IOException e) { + e.printStackTrace(); + } finally { + try { + conn.close(); + table.close(); + resultScanner.close(); + } catch (IOException e) { + e.printStackTrace(); + } + } + } +} \ No newline at end of file diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java new file mode 100644 index 000000000..7c02dba85 --- /dev/null +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java @@ -0,0 +1,75 @@ +/* + * 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 com.dtstack.flink.sql.side.hbase; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + +import java.util.List; + +public class HbaseAllSideInfo extends SideInfo { + + private RowKeyBuilder rowKeyBuilder; + + public HbaseAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + rowKeyBuilder = new RowKeyBuilder(); + if(sideTableInfo.getPrimaryKeys().size() < 1){ + throw new RuntimeException("Primary key dimension table must be filled"); + } + + rowKeyBuilder.init(sideTableInfo.getPrimaryKeys().get(0)); + + String sideTableName = joinInfo.getSideTableName(); + SqlNode conditionNode = joinInfo.getCondition(); + + List sqlNodeList = Lists.newArrayList(); + if(conditionNode.getKind() == SqlKind.AND){ + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); + }else{ + sqlNodeList.add(conditionNode); + } + + for(SqlNode sqlNode : sqlNodeList){ + dealOneEqualCon(sqlNode, sideTableName); + } + } + + public RowKeyBuilder getRowKeyBuilder() { + return rowKeyBuilder; + } + + public void setRowKeyBuilder(RowKeyBuilder rowKeyBuilder) { + this.rowKeyBuilder = rowKeyBuilder; + } + +} \ No newline at end of file diff --git a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java index 48bb9d66a..299db961d 100644 --- a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java +++ b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java @@ -56,6 +56,8 @@ public class HbaseSideParser extends AbsSideTableParser { public static final String PRE_ROW_KEY = "preRowKey"; + public static final String CACHE = "cache"; + static { keyPatternMap.put(SIDE_SIGN_KEY, SIDE_TABLE_SIGN); @@ -76,6 +78,7 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Tue, 6 Nov 2018 11:59:51 +0800 Subject: [PATCH 097/470] support union all --- .../flink/sql/parser/CreateTmpTableParser.java | 14 ++++++++++++++ .../dtstack/flink/sql/parser/InsertSqlParser.java | 14 ++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java index 89554013f..9bd1374a0 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -136,6 +136,20 @@ private static void parseNode(SqlNode sqlNode, CreateTmpTableParser.SqlParserRes sqlParseResult.addSourceTable(identifierNode.toString()); } break; + case UNION: + SqlNode unionLeft = ((SqlBasicCall)sqlNode).getOperands()[0]; + SqlNode unionRight = ((SqlBasicCall)sqlNode).getOperands()[1]; + if(unionLeft.getKind() == IDENTIFIER){ + sqlParseResult.addSourceTable(unionLeft.toString()); + }else{ + parseNode(unionLeft, sqlParseResult); + } + if(unionRight.getKind() == IDENTIFIER){ + sqlParseResult.addSourceTable(unionRight.toString()); + }else{ + parseNode(unionRight, sqlParseResult); + } + break; default: //do nothing break; diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index e3d65f420..e08540c92 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -113,6 +113,20 @@ private static void parseNode(SqlNode sqlNode, SqlParseResult sqlParseResult){ sqlParseResult.addSourceTable(identifierNode.toString()); } break; + case UNION: + SqlNode unionLeft = ((SqlBasicCall)sqlNode).getOperands()[0]; + SqlNode unionRight = ((SqlBasicCall)sqlNode).getOperands()[1]; + if(unionLeft.getKind() == IDENTIFIER){ + sqlParseResult.addSourceTable(unionLeft.toString()); + }else{ + parseNode(unionLeft, sqlParseResult); + } + if(unionRight.getKind() == IDENTIFIER){ + sqlParseResult.addSourceTable(unionRight.toString()); + }else{ + parseNode(unionRight, sqlParseResult); + } + break; default: //do nothing break; From a6e58cb1a1dd19248fea1ae5b8f54f6398ddda52 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Tue, 6 Nov 2018 20:23:40 +0800 Subject: [PATCH 098/470] bigfix --- core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 3aed54554..24332883c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -121,7 +121,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl rightScopeChild.setTableName(joinInfo.getRightTableName()); SideTableInfo sideTableInfo = sideTableMap.get(joinInfo.getRightTableName()); if(sideTableInfo == null){ - sideTableInfo = sideTableMap.get(joinInfo.getRightTableName()); + sideTableInfo = sideTableMap.get(joinInfo.getRightTableAlias()); } if(sideTableInfo == null){ From c7c9f7af209896a65d6908fe684b6d78607a39a5 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Tue, 6 Nov 2018 20:26:04 +0800 Subject: [PATCH 099/470] bugfix --- core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 7032bce05..ae3824f8b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -569,7 +569,7 @@ private void jionFun(Object pollObj, Map localTableCache, rightScopeChild.setTableName(joinInfo.getRightTableName()); SideTableInfo sideTableInfo = sideTableMap.get(joinInfo.getRightTableName()); if(sideTableInfo == null){ - sideTableInfo = sideTableMap.get(joinInfo.getRightTableName()); + sideTableInfo = sideTableMap.get(joinInfo.getRightTableAlias()); } if(sideTableInfo == null){ From dec4b16b8b70b798fb4abc1612c664857ea19b18 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Tue, 6 Nov 2018 21:29:47 +0800 Subject: [PATCH 100/470] check table exist --- .../com/dtstack/flink/sql/sink/mysql/DBSink.java | 1 + .../sql/sink/mysql/RetractJDBCOutputFormat.java | 13 ++++++++++++- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java index 3c18bf0e6..2f9d9c20b 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java @@ -84,6 +84,7 @@ public RichSinkFunction createJdbcSinkFunc(){ jdbcFormatBuild.setInsertQuery(sql); jdbcFormatBuild.setBatchInterval(batchInterval); jdbcFormatBuild.setSqlTypes(sqlTypes); + jdbcFormatBuild.setTableName(tableName); RetractJDBCOutputFormat outputFormat = jdbcFormatBuild.finish(); OutputFormatSinkFunction outputFormatSinkFunc = new OutputFormatSinkFunction(outputFormat); diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java index 3a6d76364..d4e129b0a 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java @@ -69,6 +69,7 @@ public class RetractJDBCOutputFormat extends RichOutputFormat { private String drivername; private String dbURL; private String insertQuery; + private String tableName; private int batchInterval = 5000; private Connection dbConn; @@ -96,7 +97,12 @@ public void configure(Configuration parameters) { public void open(int taskNumber, int numTasks) throws IOException { try { establishConnection(); - upload = dbConn.prepareStatement(insertQuery); + if (dbConn.getMetaData().getTables(null, null, tableName, null).next()){ + upload = dbConn.prepareStatement(insertQuery); + } else { + throw new SQLException("Table " + tableName +" doesn't exist"); + } + } catch (SQLException sqe) { throw new IllegalArgumentException("open() failed.", sqe); } catch (ClassNotFoundException cnfe) { @@ -325,6 +331,11 @@ public JDBCOutputFormatBuilder setSqlTypes(int[] typesArray) { format.typesArray = typesArray; return this; } + + public JDBCOutputFormatBuilder setTableName(String tableName) { + format.tableName = tableName; + return this; + } /** * Finalizes the configuration and checks validity. From 628331b0e6774d014f8c9add9ff22ca4c3689392 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 7 Nov 2018 13:58:45 +0800 Subject: [PATCH 101/470] =?UTF-8?q?1=EF=BC=9Achange=20flink=20version=20to?= =?UTF-8?q?=201.5.4=202=EF=BC=9Afix=20standalone=20upload=20client?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/launcher/ClusterClientFactory.java | 13 ++++++++++++- pom.xml | 2 +- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index aa8b5db5f..108ec0989 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -20,7 +20,9 @@ import org.apache.commons.lang.StringUtils; import org.apache.flink.client.deployment.ClusterRetrieveException; +import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.deployment.StandaloneClusterDescriptor; +import org.apache.flink.client.deployment.StandaloneClusterId; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.StandaloneClusterClient; import org.apache.flink.client.program.rest.RestClusterClient; @@ -30,6 +32,9 @@ import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; +import org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperHaServices; import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; @@ -47,6 +52,9 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; + import com.dtstack.flink.sql.ClusterMode; /** @@ -70,7 +78,10 @@ public static ClusterClient createClusterClient(LauncherOptions launcherOptions) public static ClusterClient createStandaloneClient(LauncherOptions launcherOptions) throws Exception { String flinkConfDir = launcherOptions.getFlinkconf(); Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); - StandaloneClusterClient clusterClient = new StandaloneClusterClient(config); + + StandaloneClusterDescriptor standaloneClusterDescriptor = new StandaloneClusterDescriptor(config); + RestClusterClient clusterClient = standaloneClusterDescriptor.retrieve(StandaloneClusterId.getInstance()); + LeaderConnectionInfo connectionInfo = clusterClient.getClusterConnectionInfo(); InetSocketAddress address = AkkaUtils.getInetSocketAddressFromAkkaURL(connectionInfo.getAddress()); config.setString(JobManagerOptions.ADDRESS, address.getAddress().getHostName()); diff --git a/pom.xml b/pom.xml index 0922f739b..b4e022c77 100644 --- a/pom.xml +++ b/pom.xml @@ -22,7 +22,7 @@ UTF-8 - 1.5.3 + 1.5.4 From 9bdc487a96e2b3ff6b89368346b0f51b6764a015 Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Wed, 7 Nov 2018 17:02:04 +0800 Subject: [PATCH 102/470] add mongo sink --- docs/mongoSink.md | 56 +++++ mongo/mongo-side/mongo-all-side/pom.xml | 88 +++++++ .../flink/sql/side/mongo/MongoAllReqRow.java | 238 ++++++++++++++++++ .../sql/side/mongo/MongoAllSideInfo.java | 113 +++++++++ mongo/mongo-side/mongo-async-side/pom.xml | 103 ++++++++ .../sql/side/mongo/MongoAsyncReqRow.java | 227 +++++++++++++++++ .../sql/side/mongo/MongoAsyncSideInfo.java | 137 ++++++++++ mongo/mongo-side/mongo-side-core/pom.xml | 24 ++ .../sql/side/mongo/table/MongoSideParser.java | 69 +++++ .../side/mongo/table/MongoSideTableInfo.java | 100 ++++++++ mongo/mongo-side/pom.xml | 23 ++ mongo/mongo-sink/pom.xml | 86 +++++++ .../sql/sink/mongo/MongoOutputFormat.java | 234 +++++++++++++++++ .../flink/sql/sink/mongo/MongoSink.java | 109 ++++++++ .../sql/sink/mongo/table/MongoSinkParser.java | 54 ++++ .../sql/sink/mongo/table/MongoTableInfo.java | 114 +++++++++ .../test/java/com/dtstack/flinkx/AppTest.java | 58 +++++ mongo/pom.xml | 40 +++ pom.xml | 1 + 19 files changed, 1874 insertions(+) create mode 100644 docs/mongoSink.md create mode 100644 mongo/mongo-side/mongo-all-side/pom.xml create mode 100644 mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java create mode 100644 mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java create mode 100644 mongo/mongo-side/mongo-async-side/pom.xml create mode 100644 mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java create mode 100644 mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java create mode 100644 mongo/mongo-side/mongo-side-core/pom.xml create mode 100644 mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideParser.java create mode 100644 mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideTableInfo.java create mode 100644 mongo/mongo-side/pom.xml create mode 100644 mongo/mongo-sink/pom.xml create mode 100644 mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java create mode 100644 mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoSink.java create mode 100644 mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java create mode 100644 mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoTableInfo.java create mode 100644 mongo/mongo-sink/src/test/java/com/dtstack/flinkx/AppTest.java create mode 100644 mongo/pom.xml diff --git a/docs/mongoSink.md b/docs/mongoSink.md new file mode 100644 index 000000000..b0f916aab --- /dev/null +++ b/docs/mongoSink.md @@ -0,0 +1,56 @@ +## 1.格式: +``` +CREATE TABLE tableName( + colName colType, + ... + colNameX colType + )WITH( + type ='mongo', + address ='ip:port[,ip:port]', + userName ='userName', + password ='pwd', + database ='databaseName', + tableName ='tableName', + parallelism ='parllNum' + ); + +``` + +## 2.支持版本 + mongo-3.8.2 + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName| 在 sql 中使用的名称;即注册到flink-table-env上的名称| +| colName | 列名称| +| colType | 列类型 [colType支持的类型](colType.md)| + +## 4.参数: + +|参数名称|含义|是否必填|默认值| +|----|----|----|----| +|type |表明 输出表类型 mongo|是|| +|address | 连接mongo数据库 jdbcUrl |是|| +|userName | mongo连接用户名|否|| +|password | mongo连接密码|否|| +|tableName | mongo表名称|是|| +|database | mongo表名称|是|| +|parallelism | 并行度设置|否|1| + +## 5.样例: +``` +CREATE TABLE MyResult( + channel VARCHAR, + pv VARCHAR + )WITH( + type ='mongo', + address ='172.21.32.1:27017,172.21.32.1:27017', + userName ='dtstack', + password ='abc123', + database ='test', + tableName ='pv', + parallelism ='1' + ) + ``` \ No newline at end of file diff --git a/mongo/mongo-side/mongo-all-side/pom.xml b/mongo/mongo-side/mongo-all-side/pom.xml new file mode 100644 index 000000000..28b15bd7c --- /dev/null +++ b/mongo/mongo-side/mongo-all-side/pom.xml @@ -0,0 +1,88 @@ + + + + sql.side.mongo + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.all.mongo + mongo-all-side + + jar + + + + com.dtstack.flink + sql.side.mongo.core + 1.0-SNAPSHOT + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java new file mode 100644 index 000000000..395aba0a3 --- /dev/null +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java @@ -0,0 +1,238 @@ +/* + * 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 com.dtstack.flink.sql.side.mongo; + +import com.dtstack.flink.sql.side.AllReqRow; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.mongo.MongoAllSideInfo; +import com.dtstack.flink.sql.side.mongo.table.MongoSideTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.calcite.sql.JoinType; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Timestamp; +import java.util.Calendar; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; + +/** + * side operator with cache for all(period reload) + * Date: 2018/9/19 + * Company: www.dtstack.com + * @author xuchao + */ + +public class MongoAllReqRow extends AllReqRow{ + + private static final long serialVersionUID = 2098635140857937717L; + + private static final Logger LOG = LoggerFactory.getLogger(MongoAllReqRow.class); + + private static final String Mongo_DRIVER = "com.mongo.jdbc.Driver"; + + private static final int CONN_RETRY_NUM = 3; + + private static final int FETCH_SIZE = 1000; + + private AtomicReference>>> cacheRef = new AtomicReference<>(); + + public MongoAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo){ + super(new MongoAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + + @Override + protected Row fillData(Row input, Object sideInput) { + Map cacheInfo = (Map) sideInput; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. + if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ + obj = ((Timestamp)obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for(Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()){ + if(cacheInfo == null){ + row.setField(entry.getKey(), null); + }else{ + row.setField(entry.getKey(), cacheInfo.get(entry.getValue())); + } + } + + return row; + } + + @Override + protected void initCache() throws SQLException { + Map>> newCache = Maps.newConcurrentMap(); + cacheRef.set(newCache); + loadData(newCache); + } + + @Override + protected void reloadCache() { + //reload cacheRef and replace to old cacheRef + Map>> newCache = Maps.newConcurrentMap(); + try { + loadData(newCache); + } catch (SQLException e) { + LOG.error("", e); + } + + cacheRef.set(newCache); + LOG.info("----- Mongo all cacheRef reload end:{}", Calendar.getInstance()); + } + + + @Override + public void flatMap(Row value, Collector out) throws Exception { + List inputParams = Lists.newArrayList(); + for(Integer conValIndex : sideInfo.getEqualValIndex()){ + Object equalObj = value.getField(conValIndex); + if(equalObj == null){ + out.collect(null); + } + + inputParams.add(equalObj); + } + + String key = buildKey(inputParams); + List> cacheList = cacheRef.get().get(key); + if(CollectionUtils.isEmpty(cacheList)){ + if(sideInfo.getJoinType() == JoinType.LEFT){ + Row row = fillData(value, null); + out.collect(row); + }else{ + return; + } + + return; + } + + for(Map one : cacheList){ + out.collect(fillData(value, one)); + } + + } + + private String buildKey(List equalValList){ + StringBuilder sb = new StringBuilder(""); + for(Object equalVal : equalValList){ + sb.append(equalVal).append("_"); + } + + return sb.toString(); + } + + private String buildKey(Map val, List equalFieldList){ + StringBuilder sb = new StringBuilder(""); + for(String equalField : equalFieldList){ + sb.append(val.get(equalField)).append("_"); + } + + return sb.toString(); + } + + private Connection getConn(String dbURL, String userName, String password){ + try{ + Class.forName(Mongo_DRIVER); + //add param useCursorFetch=true + Map addParams = Maps.newHashMap(); + addParams.put("useCursorFetch", "true"); + String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams,true); + return DriverManager.getConnection(targetDbUrl, userName, password); + }catch (Exception e){ + LOG.error("", e); + throw new RuntimeException("", e); + } + } + + + private void loadData(Map>> tmpCache) throws SQLException { + MongoSideTableInfo tableInfo = (MongoSideTableInfo) sideInfo.getSideTableInfo(); + Connection connection = null; + + try{ + for(int i=0; i oneRow = Maps.newHashMap(); + for(String fieldName : sideFieldNames){ + oneRow.put(fieldName.trim(), resultSet.getObject(fieldName.trim())); + } + + String cacheKey = buildKey(oneRow, sideInfo.getEqualFieldList()); + List> list = tmpCache.computeIfAbsent(cacheKey, key -> Lists.newArrayList()); + list.add(oneRow); + } + }catch (Exception e){ + LOG.error("", e); + }finally { + if(connection != null){ + connection.close(); + } + } + } +} diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java new file mode 100644 index 000000000..7087da815 --- /dev/null +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java @@ -0,0 +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 com.dtstack.flink.sql.side.mongo; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.mongo.table.MongoSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.List; + +/** + * Reason: + * Date: 2018/9/19 + * Company: www.dtstack.com + * @author xuchao + */ + +public class MongoAllSideInfo extends SideInfo{ + + private static final long serialVersionUID = -5858335638589472158L; + + public MongoAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + MongoSideTableInfo MongoSideTableInfo = (MongoSideTableInfo) sideTableInfo; + + sqlCondition = "select ${selectField} from ${tableName} "; + sqlCondition = sqlCondition.replace("${tableName}", MongoSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); + System.out.println("---------side_exe_sql-----\n" + sqlCondition); + } + + @Override + public void parseSelectFields(JoinInfo joinInfo){ + String sideTableName = joinInfo.getSideTableName(); + String nonSideTableName = joinInfo.getNonSideTable(); + List fields = Lists.newArrayList(); + + int sideIndex = 0; + for( int i=0; i sqlNodeList = Lists.newArrayList(); + if(conditionNode.getKind() == SqlKind.AND){ + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); + }else{ + sqlNodeList.add(conditionNode); + } + + for(SqlNode sqlNode : sqlNodeList){ + dealOneEqualCon(sqlNode, sideTableName); + } + + if(CollectionUtils.isEmpty(equalFieldList)){ + throw new RuntimeException("no join condition found after table " + joinInfo.getLeftTableName()); + } + + for(String equalField : equalFieldList){ + if(fields.contains(equalField)){ + continue; + } + + fields.add(equalField); + } + + sideSelectFields = String.join(",", fields); + } +} diff --git a/mongo/mongo-side/mongo-async-side/pom.xml b/mongo/mongo-side/mongo-async-side/pom.xml new file mode 100644 index 000000000..68df02fd0 --- /dev/null +++ b/mongo/mongo-side/mongo-async-side/pom.xml @@ -0,0 +1,103 @@ + + + + sql.side.mongo + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.async.mongo + + mongo-async-side + + jar + + + + + + io.vertx + vertx-jdbc-client + 3.5.2 + + + + io.vertx + vertx-core + 3.5.2 + + + + com.dtstack.flink + sql.side.mongo.core + 1.0-SNAPSHOT + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java new file mode 100644 index 000000000..d658e6be2 --- /dev/null +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.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 com.dtstack.flink.sql.side.mongo; + +import com.dtstack.flink.sql.enums.ECacheContentType; +import com.dtstack.flink.sql.side.AsyncReqRow; +import com.dtstack.flink.sql.side.CacheMissVal; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.cache.CacheObj; +import com.dtstack.flink.sql.side.mongo.MongoAsyncSideInfo; +import com.dtstack.flink.sql.side.mongo.table.MongoSideTableInfo; +import io.vertx.core.Vertx; +import io.vertx.core.VertxOptions; +import io.vertx.core.json.JsonArray; +import io.vertx.core.json.JsonObject; +import io.vertx.ext.jdbc.JDBCClient; +import io.vertx.ext.sql.SQLClient; +import io.vertx.ext.sql.SQLConnection; +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Timestamp; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Mongo dim table + * Date: 2018/7/27 + * Company: www.dtstack.com + * @author xuchao + */ + +public class MongoAsyncReqRow extends AsyncReqRow { + + private static final long serialVersionUID = 2098635244857937717L; + + private static final Logger LOG = LoggerFactory.getLogger(MongoAsyncReqRow.class); + + private transient SQLClient MongoClient; + + private final static String Mongo_DRIVER = "com.Mongo.jdbc.Driver"; + + private final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 10; + + private final static int DEFAULT_VERTX_WORKER_POOL_SIZE = 20; + + private final static int DEFAULT_MAX_DB_CONN_POOL_SIZE = 20; + + + public MongoAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new MongoAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + JsonObject MongoClientConfig = new JsonObject(); + MongoSideTableInfo MongoSideTableInfo = (MongoSideTableInfo) sideInfo.getSideTableInfo(); + MongoClientConfig.put("url", MongoSideTableInfo.getUrl()) + .put("driver_class", Mongo_DRIVER) + .put("max_pool_size", DEFAULT_MAX_DB_CONN_POOL_SIZE) + .put("user", MongoSideTableInfo.getUserName()) + .put("password", MongoSideTableInfo.getPassword()); + + VertxOptions vo = new VertxOptions(); + vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); + vo.setWorkerPoolSize(DEFAULT_VERTX_WORKER_POOL_SIZE); + Vertx vertx = Vertx.vertx(vo); + MongoClient = JDBCClient.createNonShared(vertx, MongoClientConfig); + } + + @Override + public void asyncInvoke(Row input, ResultFuture resultFuture) throws Exception { + + JsonArray inputParams = new JsonArray(); + for(Integer conValIndex : sideInfo.getEqualValIndex()){ + Object equalObj = input.getField(conValIndex); + if(equalObj == null){ + resultFuture.complete(null); + } + + inputParams.add(equalObj); + } + + String key = buildCacheKey(inputParams); + if(openCache()){ + CacheObj val = getFromCache(key); + if(val != null){ + + if(ECacheContentType.MissVal == val.getType()){ + dealMissKey(input, resultFuture); + return; + }else if(ECacheContentType.MultiLine == val.getType()){ + + for(Object jsonArray : (List)val.getContent()){ + Row row = fillData(input, jsonArray); + resultFuture.complete(Collections.singleton(row)); + } + + }else{ + throw new RuntimeException("not support cache obj type " + val.getType()); + } + return; + } + } + + MongoClient.getConnection(conn -> { + if (conn.failed()) { + //Treatment failures + resultFuture.completeExceptionally(conn.cause()); + return; + } + + final SQLConnection connection = conn.result(); + String sqlCondition = sideInfo.getSqlCondition(); + connection.queryWithParams(sqlCondition, inputParams, rs -> { + if (rs.failed()) { + LOG.error("Cannot retrieve the data from the database"); + LOG.error("", rs.cause()); + resultFuture.complete(null); + return; + } + + List cacheContent = Lists.newArrayList(); + + int resultSize = rs.result().getResults().size(); + if(resultSize > 0){ + for (JsonArray line : rs.result().getResults()) { + Row row = fillData(input, line); + if(openCache()){ + cacheContent.add(line); + } + resultFuture.complete(Collections.singleton(row)); + } + + if(openCache()){ + putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); + } + }else{ + dealMissKey(input, resultFuture); + if(openCache()){ + putCache(key, CacheMissVal.getMissKeyObj()); + } + } + + // and close the connection + connection.close(done -> { + if (done.failed()) { + throw new RuntimeException(done.cause()); + } + }); + }); + }); + } + + @Override + public Row fillData(Row input, Object line){ + JsonArray jsonArray = (JsonArray) line; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ + obj = ((Timestamp)obj).getTime(); + } + + row.setField(entry.getKey(), obj); + } + + for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ + if(jsonArray == null){ + row.setField(entry.getKey(), null); + }else{ + row.setField(entry.getKey(), jsonArray.getValue(entry.getValue())); + } + } + + return row; + } + + @Override + public void close() throws Exception { + super.close(); + MongoClient.close(); + } + + public String buildCacheKey(JsonArray jsonArray){ + StringBuilder sb = new StringBuilder(); + for(Object ele : jsonArray.getList()){ + sb.append(ele.toString()) + .append("_"); + } + + return sb.toString(); + } + +} diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java new file mode 100644 index 000000000..7ef5b2d5e --- /dev/null +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java @@ -0,0 +1,137 @@ +/* + * 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 com.dtstack.flink.sql.side.mongo; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.mongo.table.MongoSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.List; + +/** + * Reason: + * Date: 2018/9/18 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class MongoAsyncSideInfo extends SideInfo { + + private static final long serialVersionUID = -5931494270201575201L; + + public MongoAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + MongoSideTableInfo MongoSideTableInfo = (MongoSideTableInfo) sideTableInfo; + + String sideTableName = joinInfo.getSideTableName(); + + SqlNode conditionNode = joinInfo.getCondition(); + + List sqlNodeList = Lists.newArrayList(); + if(conditionNode.getKind() == SqlKind.AND){ + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); + }else{ + sqlNodeList.add(conditionNode); + } + + for(SqlNode sqlNode : sqlNodeList){ + dealOneEqualCon(sqlNode, sideTableName); + } + + sqlCondition = "select ${selectField} from ${tableName} where "; + for(int i=0; i + + + sql.side.mongo + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.mongo.core + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + jar + + \ No newline at end of file diff --git a/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideParser.java b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideParser.java new file mode 100644 index 000000000..3e0b4fe6a --- /dev/null +++ b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideParser.java @@ -0,0 +1,69 @@ +/* + * 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 com.dtstack.flink.sql.side.mongo.table; + +import com.dtstack.flink.sql.side.mongo.table.MongoSideTableInfo; +import com.dtstack.flink.sql.table.AbsSideTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/11/6 + * + * @author xuqianjin + */ + + +public class MongoSideParser extends AbsSideTableParser { + + private final static String SIDE_SIGN_KEY = "sideSignKey"; + + private final static Pattern SIDE_TABLE_SIGN = Pattern.compile("(?i)^PERIOD\\s+FOR\\s+SYSTEM_TIME$"); + + static { + keyPatternMap.put(SIDE_SIGN_KEY, SIDE_TABLE_SIGN); + keyHandlerMap.put(SIDE_SIGN_KEY, MongoSideParser::dealSideSign); + } + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + MongoSideTableInfo MongoTableInfo = new MongoSideTableInfo(); + MongoTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, MongoTableInfo); + + parseCacheProp(MongoTableInfo, props); + MongoTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(MongoSideTableInfo.PARALLELISM_KEY.toLowerCase()))); + MongoTableInfo.setUrl(MathUtil.getString(props.get(MongoSideTableInfo.URL_KEY.toLowerCase()))); + MongoTableInfo.setTableName(MathUtil.getString(props.get(MongoSideTableInfo.TABLE_NAME_KEY.toLowerCase()))); + MongoTableInfo.setUserName(MathUtil.getString(props.get(MongoSideTableInfo.USER_NAME_KEY.toLowerCase()))); + MongoTableInfo.setPassword(MathUtil.getString(props.get(MongoSideTableInfo.PASSWORD_KEY.toLowerCase()))); + + return MongoTableInfo; + } + + private static void dealSideSign(Matcher matcher, TableInfo tableInfo){ + } +} diff --git a/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideTableInfo.java b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideTableInfo.java new file mode 100644 index 000000000..488c92861 --- /dev/null +++ b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideTableInfo.java @@ -0,0 +1,100 @@ +/* + * 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 com.dtstack.flink.sql.side.mongo.table; + +import com.dtstack.flink.sql.side.SideTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * Reason: + * Date: 2018/11/6 + * + * @author xuqianjin + */ + + +public class MongoSideTableInfo extends SideTableInfo { + + private static final long serialVersionUID = -1L; + + private static final String CURR_TYPE = "Mongo"; + + public static final String URL_KEY = "url"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String USER_NAME_KEY = "userName"; + + public static final String PASSWORD_KEY = "password"; + + public MongoSideTableInfo(){ + setType(CURR_TYPE); + } + + @Override + public boolean check() { + Preconditions.checkNotNull(url, "Mongo of URL is required"); + Preconditions.checkNotNull(tableName, "Mongo of tableName is required"); + Preconditions.checkNotNull(userName, "Mongo of userName is required"); + Preconditions.checkNotNull(password, "Mongo of password is required"); + return true; + } + + private String url; + + private String tableName; + + private String userName; + + private String password; + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } +} diff --git a/mongo/mongo-side/pom.xml b/mongo/mongo-side/pom.xml new file mode 100644 index 000000000..c2c6f2f61 --- /dev/null +++ b/mongo/mongo-side/pom.xml @@ -0,0 +1,23 @@ + + + + sql.mongo + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.mongo + mongo-side + + mongo-side-core + mongo-async-side + mongo-all-side + + + pom + + \ No newline at end of file diff --git a/mongo/mongo-sink/pom.xml b/mongo/mongo-sink/pom.xml new file mode 100644 index 000000000..b148104a0 --- /dev/null +++ b/mongo/mongo-sink/pom.xml @@ -0,0 +1,86 @@ + + + sql.mongo + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.mongo + jar + + mongo-sink + http://maven.apache.org + + + + org.apache.flink + flink-jdbc + ${flink.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java new file mode 100644 index 000000000..c1ba890e0 --- /dev/null +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java @@ -0,0 +1,234 @@ +/* + * 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 com.dtstack.flink.sql.sink.mongo; + +import com.mongodb.BasicDBObject; +import com.mongodb.InsertOptions; +import com.mongodb.MongoClient; +import com.mongodb.MongoClientOptions; +import com.mongodb.MongoCredential; +import com.mongodb.ServerAddress; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; +import com.mongodb.client.model.InsertOneOptions; +import com.mongodb.client.model.UpdateOptions; +import com.mongodb.client.result.UpdateResult; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.scala.row; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; +import org.bson.BSON; +import org.bson.BsonDocument; +import org.bson.Document; +import org.bson.conversions.Bson; +import org.bson.types.ObjectId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.List; + +/** + * Reason: + * Date: 2018/11/6 + * + * @author xuqianjin + */ +public class MongoOutputFormat extends RichOutputFormat { + private static final Logger LOG = LoggerFactory.getLogger(MongoOutputFormat.class); + + private String address; + private String tableName; + private String userName; + private String password; + private String database; + protected String[] fieldNames; + TypeInformation[] fieldTypes; + + private MongoClient mongoClient; + private MongoDatabase db; + + private static String PK = "_ID"; + + public final SimpleDateFormat ROWKEY_DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss"); + + @Override + public void configure(Configuration parameters) { + } + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + establishConnection(); + } + + @Override + public void writeRecord(Tuple2 tuple2) throws IOException { + + Tuple2 tupleTrans = tuple2; + Boolean retract = tupleTrans.getField(0); + if (!retract) { + //FIXME 暂时不处理Mongo删除操作--->Mongo要求有key,所有认为都是可以执行update查找 + return; + } + + Row record = tupleTrans.getField(1); + if (record.getArity() != fieldNames.length) { + return; + } + + Document doc = new Document(); + MongoCollection dbCollection = db.getCollection(tableName, Document.class); + for (int i = 0; i < fieldNames.length; i++) { + doc.append(fieldNames[i], record.getField(i)); + } + if (doc.containsKey(PK)) { + Document updateValue = new Document(); + Document filter = new Document(PK.toLowerCase(), new ObjectId(doc.getString(PK))); + doc.remove(PK); + updateValue.append("$set", doc); + UpdateResult updateResult = dbCollection.updateOne(filter, updateValue); + if (updateResult.getMatchedCount() <= 0) { + dbCollection.insertOne(doc); + } + } else { + dbCollection.insertOne(doc); + } + } + + @Override + public void close() { + try { + if (mongoClient != null) { + mongoClient.close(); + } + } catch (Exception e) { + throw new IllegalArgumentException("[closeMongoDB]:" + e.getMessage()); + } + } + + private void establishConnection() { + try { + MongoCredential credential; + String[] servers = address.split(","); + String host; + Integer port; + String[] hostAndPort; + List lists = new ArrayList<>(); + for (String server : servers) { + hostAndPort = server.split(":"); + host = hostAndPort[0]; + port = Integer.parseInt(hostAndPort[1]); + lists.add(new ServerAddress(host, port)); + } + if (!StringUtils.isEmpty(userName) || !StringUtils.isEmpty(password)) { + credential = MongoCredential.createCredential(userName, database, password.toCharArray()); + // To connect to mongodb server + mongoClient = new MongoClient(lists, credential, new MongoClientOptions.Builder().build()); + } else { + mongoClient = new MongoClient(lists); + } + db = mongoClient.getDatabase(database); + } catch (Exception e) { + throw new IllegalArgumentException("[connMongoDB]:" + e.getMessage()); + } + } + + private MongoOutputFormat() { + } + + public static MongoOutputFormatBuilder buildMongoOutputFormat() { + return new MongoOutputFormatBuilder(); + } + + public static class MongoOutputFormatBuilder { + private final MongoOutputFormat format; + + protected MongoOutputFormatBuilder() { + this.format = new MongoOutputFormat(); + } + + public MongoOutputFormatBuilder setUsername(String username) { + format.userName = username; + return this; + } + + public MongoOutputFormatBuilder setPassword(String password) { + format.password = password; + return this; + } + + public MongoOutputFormatBuilder setAddress(String address) { + format.address = address; + return this; + } + + public MongoOutputFormatBuilder setTableName(String tableName) { + format.tableName = tableName; + return this; + } + + public MongoOutputFormatBuilder setDatabase(String database) { + format.database = database; + return this; + } + + public MongoOutputFormatBuilder setFieldNames(String[] fieldNames) { + format.fieldNames = fieldNames; + return this; + } + + public MongoOutputFormatBuilder setFieldTypes(TypeInformation[] fieldTypes) { + format.fieldTypes = fieldTypes; + return this; + } + + /** + * Finalizes the configuration and checks validity. + * + * @return Configured RetractJDBCOutputFormat + */ + public MongoOutputFormat finish() { + if (format.userName == null) { + LOG.info("Username was not supplied separately."); + } + if (format.password == null) { + LOG.info("Password was not supplied separately."); + } + if (format.address == null) { + throw new IllegalArgumentException("No address URL supplied."); + } + if (format.database == null) { + throw new IllegalArgumentException("No dababase suplied"); + } + if (format.tableName == null) { + throw new IllegalArgumentException("No tableName supplied"); + } + return format; + } + } + + +} diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoSink.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoSink.java new file mode 100644 index 000000000..4e28d8fd2 --- /dev/null +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoSink.java @@ -0,0 +1,109 @@ +/* + * 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 com.dtstack.flink.sql.sink.mongo; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.mongo.table.MongoTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.table.sinks.RetractStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +/** + * Reason: + * Date: 2018/11/6 + * + * @author xuqianjin + */ +public class MongoSink implements RetractStreamTableSink, IStreamSinkGener { + + protected String[] fieldNames; + TypeInformation[] fieldTypes; + protected String address; + protected String tableName; + protected String userName; + protected String password; + protected String database; + + public MongoSink() { + // TO DO NOTHING + } + + @Override + public MongoSink genStreamSink(TargetTableInfo targetTableInfo) { + MongoTableInfo mongoTableInfo = (MongoTableInfo) targetTableInfo; + this.address = mongoTableInfo.getAddress(); + this.tableName = mongoTableInfo.getTableName(); + this.userName = mongoTableInfo.getUserName(); + this.password = mongoTableInfo.getPassword(); + this.database = mongoTableInfo.getDatabase(); + return this; + } + + @Override + public void emitDataStream(DataStream> dataStream) { + MongoOutputFormat.MongoOutputFormatBuilder builder = MongoOutputFormat.buildMongoOutputFormat(); + builder.setAddress(this.address) + .setDatabase(this.database) + .setTableName(this.tableName) + .setPassword(this.password) + .setUsername(this.userName) + .setFieldNames(this.fieldNames) + .setFieldTypes(this.fieldTypes); + + MongoOutputFormat outputFormat = builder.finish(); + RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(outputFormat); + dataStream.addSink(richSinkFunction); + } + + @Override + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + + @Override + public TupleTypeInfo> getOutputType() { + return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), getRecordType()); + } + + @Override + public TypeInformation getRecordType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } +} diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java new file mode 100644 index 000000000..e720f8f2a --- /dev/null +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java @@ -0,0 +1,54 @@ +/* + * 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 com.dtstack.flink.sql.sink.mongo.table; + +import com.dtstack.flink.sql.sink.mongo.table.MongoTableInfo; +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/6 + * + * @author xuqianjin + */ + +public class MongoSinkParser extends AbsTableParser { + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + MongoTableInfo MongoTableInfo = new MongoTableInfo(); + MongoTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, MongoTableInfo); + + MongoTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(MongoTableInfo.PARALLELISM_KEY.toLowerCase()))); + MongoTableInfo.setAddress(MathUtil.getString(props.get(MongoTableInfo.ADDRESS_KEY.toLowerCase()))); + MongoTableInfo.setTableName(MathUtil.getString(props.get(MongoTableInfo.TABLE_NAME_KEY.toLowerCase()))); + MongoTableInfo.setDatabase(MathUtil.getString(props.get(MongoTableInfo.DATABASE_KEY.toLowerCase()))); + MongoTableInfo.setUserName(MathUtil.getString(props.get(MongoTableInfo.USER_NAME_KEY.toLowerCase()))); + MongoTableInfo.setPassword(MathUtil.getString(props.get(MongoTableInfo.PASSWORD_KEY.toLowerCase()))); + + return MongoTableInfo; + } +} diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoTableInfo.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoTableInfo.java new file mode 100644 index 000000000..ba50bbf08 --- /dev/null +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoTableInfo.java @@ -0,0 +1,114 @@ +/* + * 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 com.dtstack.flink.sql.sink.mongo.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * Reason: + * Date: 2018/11/6 + * + * @author xuqianjin + */ + + +public class MongoTableInfo extends TargetTableInfo { + + private static final String CURR_TYPE = "mongo"; + + public static final String ADDRESS_KEY = "address"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String USER_NAME_KEY = "userName"; + + public static final String PASSWORD_KEY = "password"; + + public static final String DATABASE_KEY = "database"; + + public MongoTableInfo() { + setType(CURR_TYPE); + } + + private String address; + + private String tableName; + + private String userName; + + private String password; + + private String database; + + public String getAddress() { + return address; + } + + public void setAddress(String address) { + this.address = address; + } + + public String getDatabase() { + return database; + } + + public void setDatabase(String database) { + this.database = database; + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } + + @Override + public boolean check() { + Preconditions.checkNotNull(address, "Mongo field of ADDRESS is required"); + Preconditions.checkNotNull(database, "Mongo field of database is required"); + Preconditions.checkNotNull(tableName, "Mongo field of tableName is required"); + return true; + } + + @Override + public String getType() { + // return super.getType().toLowerCase() + TARGET_SUFFIX; + return super.getType().toLowerCase(); + } +} diff --git a/mongo/mongo-sink/src/test/java/com/dtstack/flinkx/AppTest.java b/mongo/mongo-sink/src/test/java/com/dtstack/flinkx/AppTest.java new file mode 100644 index 000000000..33a0233ac --- /dev/null +++ b/mongo/mongo-sink/src/test/java/com/dtstack/flinkx/AppTest.java @@ -0,0 +1,58 @@ +/* + * 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 com.dtstack.flinkx; + +import junit.framework.Test; +import junit.framework.TestCase; +import junit.framework.TestSuite; + +/** + * Unit test for simple App. + */ +public class AppTest + extends TestCase +{ + /** + * Create the test case + * + * @param testName name of the test case + */ + public AppTest( String testName ) + { + super( testName ); + } + + /** + * @return the suite of tests being tested + */ + public static Test suite() + { + return new TestSuite( AppTest.class ); + } + + /** + * Rigourous Test :-) + */ + public void testApp() + { + assertTrue( true ); + } +} diff --git a/mongo/pom.xml b/mongo/pom.xml new file mode 100644 index 000000000..b9f789f17 --- /dev/null +++ b/mongo/pom.xml @@ -0,0 +1,40 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + sql.mongo + pom + + + mongo-sink + mongo-side + + + + + junit + junit + 3.8.1 + test + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + org.mongodb + mongo-java-driver + 3.8.2 + + + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index 0922f739b..953105f9d 100644 --- a/pom.xml +++ b/pom.xml @@ -13,6 +13,7 @@ mysql hbase elasticsearch5 + mongo launcher pom From cbdb01c3c9f2a78aedff1c90d8d6785cbe2ff463 Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Wed, 7 Nov 2018 17:58:01 +0800 Subject: [PATCH 103/470] modify mongo sink --- .../sql/sink/mongo/table/MongoSinkParser.java | 26 ++++++++++++++----- .../sql/sink/mongo/table/MongoTableInfo.java | 10 ------- 2 files changed, 20 insertions(+), 16 deletions(-) diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java index e720f8f2a..c8f1bdcd8 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java @@ -27,6 +27,8 @@ import java.util.Map; +import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; + /** * Reason: * Date: 2018/11/6 @@ -36,18 +38,30 @@ public class MongoSinkParser extends AbsTableParser { + private static final String CURR_TYPE = "mongo"; + + public static final String ADDRESS_KEY = "address"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String USER_NAME_KEY = "userName"; + + public static final String PASSWORD_KEY = "password"; + + public static final String DATABASE_KEY = "database"; + @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { MongoTableInfo MongoTableInfo = new MongoTableInfo(); MongoTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, MongoTableInfo); - MongoTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(MongoTableInfo.PARALLELISM_KEY.toLowerCase()))); - MongoTableInfo.setAddress(MathUtil.getString(props.get(MongoTableInfo.ADDRESS_KEY.toLowerCase()))); - MongoTableInfo.setTableName(MathUtil.getString(props.get(MongoTableInfo.TABLE_NAME_KEY.toLowerCase()))); - MongoTableInfo.setDatabase(MathUtil.getString(props.get(MongoTableInfo.DATABASE_KEY.toLowerCase()))); - MongoTableInfo.setUserName(MathUtil.getString(props.get(MongoTableInfo.USER_NAME_KEY.toLowerCase()))); - MongoTableInfo.setPassword(MathUtil.getString(props.get(MongoTableInfo.PASSWORD_KEY.toLowerCase()))); + MongoTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase()))); + MongoTableInfo.setAddress(MathUtil.getString(props.get(ADDRESS_KEY.toLowerCase()))); + MongoTableInfo.setTableName(MathUtil.getString(props.get(TABLE_NAME_KEY.toLowerCase()))); + MongoTableInfo.setDatabase(MathUtil.getString(props.get(DATABASE_KEY.toLowerCase()))); + MongoTableInfo.setUserName(MathUtil.getString(props.get(USER_NAME_KEY.toLowerCase()))); + MongoTableInfo.setPassword(MathUtil.getString(props.get(PASSWORD_KEY.toLowerCase()))); return MongoTableInfo; } diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoTableInfo.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoTableInfo.java index ba50bbf08..91de9eba6 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoTableInfo.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoTableInfo.java @@ -34,16 +34,6 @@ public class MongoTableInfo extends TargetTableInfo { private static final String CURR_TYPE = "mongo"; - public static final String ADDRESS_KEY = "address"; - - public static final String TABLE_NAME_KEY = "tableName"; - - public static final String USER_NAME_KEY = "userName"; - - public static final String PASSWORD_KEY = "password"; - - public static final String DATABASE_KEY = "database"; - public MongoTableInfo() { setType(CURR_TYPE); } From 9177de93338e575727618886d6c42df285a52039 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 7 Nov 2018 21:33:39 +0800 Subject: [PATCH 104/470] rename field --- .../sql/side/mongo/MongoAsyncReqRow.java | 22 ++++++++--------- .../sql/sink/mongo/table/MongoSinkParser.java | 24 +++++++++---------- 2 files changed, 23 insertions(+), 23 deletions(-) diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index d658e6be2..1c200a2b1 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -64,9 +64,9 @@ public class MongoAsyncReqRow extends AsyncReqRow { private static final Logger LOG = LoggerFactory.getLogger(MongoAsyncReqRow.class); - private transient SQLClient MongoClient; + private transient SQLClient mongoClient; - private final static String Mongo_DRIVER = "com.Mongo.jdbc.Driver"; + private final static String mongo_driver = "com.Mongo.jdbc.Driver"; private final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 10; @@ -83,19 +83,19 @@ public MongoAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List resultFuture) throws Except } } - MongoClient.getConnection(conn -> { + mongoClient.getConnection(conn -> { if (conn.failed()) { //Treatment failures resultFuture.completeExceptionally(conn.cause()); @@ -211,7 +211,7 @@ public Row fillData(Row input, Object line){ @Override public void close() throws Exception { super.close(); - MongoClient.close(); + mongoClient.close(); } public String buildCacheKey(JsonArray jsonArray){ diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java index c8f1bdcd8..d6bc01e64 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java @@ -52,17 +52,17 @@ public class MongoSinkParser extends AbsTableParser { @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - MongoTableInfo MongoTableInfo = new MongoTableInfo(); - MongoTableInfo.setName(tableName); - parseFieldsInfo(fieldsInfo, MongoTableInfo); - - MongoTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase()))); - MongoTableInfo.setAddress(MathUtil.getString(props.get(ADDRESS_KEY.toLowerCase()))); - MongoTableInfo.setTableName(MathUtil.getString(props.get(TABLE_NAME_KEY.toLowerCase()))); - MongoTableInfo.setDatabase(MathUtil.getString(props.get(DATABASE_KEY.toLowerCase()))); - MongoTableInfo.setUserName(MathUtil.getString(props.get(USER_NAME_KEY.toLowerCase()))); - MongoTableInfo.setPassword(MathUtil.getString(props.get(PASSWORD_KEY.toLowerCase()))); - - return MongoTableInfo; + MongoTableInfo mongoTableInfo = new MongoTableInfo(); + mongoTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, mongoTableInfo); + + mongoTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase()))); + mongoTableInfo.setAddress(MathUtil.getString(props.get(ADDRESS_KEY.toLowerCase()))); + mongoTableInfo.setTableName(MathUtil.getString(props.get(TABLE_NAME_KEY.toLowerCase()))); + mongoTableInfo.setDatabase(MathUtil.getString(props.get(DATABASE_KEY.toLowerCase()))); + mongoTableInfo.setUserName(MathUtil.getString(props.get(USER_NAME_KEY.toLowerCase()))); + mongoTableInfo.setPassword(MathUtil.getString(props.get(PASSWORD_KEY.toLowerCase()))); + + return mongoTableInfo; } } From faf253ff6da3c8902fdbd122d17dc916c7729491 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 7 Nov 2018 21:53:01 +0800 Subject: [PATCH 105/470] remove flink jdbc pom --- mongo/mongo-sink/pom.xml | 9 --------- 1 file changed, 9 deletions(-) diff --git a/mongo/mongo-sink/pom.xml b/mongo/mongo-sink/pom.xml index b148104a0..79a770684 100644 --- a/mongo/mongo-sink/pom.xml +++ b/mongo/mongo-sink/pom.xml @@ -13,15 +13,6 @@ mongo-sink http://maven.apache.org - - - - org.apache.flink - flink-jdbc - ${flink.version} - - - From 1ed590022b742c056c734376f0f04ace3773f714 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 8 Nov 2018 10:05:32 +0800 Subject: [PATCH 106/470] remove flink-jdbc dependency --- mysql/mysql-sink/pom.xml | 6 +----- .../flink/sql/sink/mysql/RetractJDBCOutputFormat.java | 2 +- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/mysql/mysql-sink/pom.xml b/mysql/mysql-sink/pom.xml index 9fd82deb3..1aeeccba6 100644 --- a/mysql/mysql-sink/pom.xml +++ b/mysql/mysql-sink/pom.xml @@ -15,11 +15,7 @@ http://maven.apache.org - - org.apache.flink - flink-jdbc - ${flink.version} - + diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java index d4e129b0a..a788dcdad 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java @@ -62,7 +62,7 @@ public class RetractJDBCOutputFormat extends RichOutputFormat { private static final long serialVersionUID = 1L; - private static final Logger LOG = LoggerFactory.getLogger(org.apache.flink.api.java.io.jdbc.JDBCOutputFormat.class); + private static final Logger LOG = LoggerFactory.getLogger(RetractJDBCOutputFormat.class); private String username; private String password; From c87c31c0898c28d2a608c70f469d832f5b3c7354 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 8 Nov 2018 16:56:45 +0800 Subject: [PATCH 107/470] fix md demo --- docs/hbaseSide.md | 2 +- docs/hbaseSink.md | 4 ++-- docs/kafkaSource.md | 4 ++-- docs/mysqlSide.md | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/hbaseSide.md b/docs/hbaseSide.md index 60f0ef768..07b03026a 100644 --- a/docs/hbaseSide.md +++ b/docs/hbaseSide.md @@ -52,7 +52,7 @@ ## 4.样例 ``` CREATE TABLE sideTable( - cf:name String as name, + cf:name varchar as name, cf:info int as info, PRIMARY KEY(md5(name) + 'test'), PERIOD FOR SYSTEM_TIME diff --git a/docs/hbaseSink.md b/docs/hbaseSink.md index ff2f83091..080985a8e 100644 --- a/docs/hbaseSink.md +++ b/docs/hbaseSink.md @@ -41,8 +41,8 @@ hbase2.0 ## 5.样例: ``` CREATE TABLE MyResult( - cf:channel STRING, - cf:pv BIGINT + cf:channel varchar, + cf:pv varchar )WITH( type ='hbase', zookeeperQuorum ='rdos1:2181', diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index 54bab7516..2968dbe22 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -44,8 +44,8 @@ CREATE TABLE tableName( ## 5.样例: ``` CREATE TABLE MyTable( - name string, - channel STRING, + name varchar, + channel varchar, pv INT, xctime bigint, CHARACTER_LENGTH(channel) AS timeLeng diff --git a/docs/mysqlSide.md b/docs/mysqlSide.md index f374a51c6..d0fec5832 100644 --- a/docs/mysqlSide.md +++ b/docs/mysqlSide.md @@ -57,7 +57,7 @@ ## 5.样例 ``` create table sideTable( - channel String, + channel varchar, xccount int, PRIMARY KEY(channel), PERIOD FOR SYSTEM_TIME From 49544e30dc7f769481c377cdb932a901a5d2043a Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 8 Nov 2018 16:58:16 +0800 Subject: [PATCH 108/470] fix md demo --- docs/hbaseSide.md | 2 +- docs/hbaseSink.md | 2 +- docs/kafkaSource.md | 4 ++-- docs/mysqlSide.md | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/hbaseSide.md b/docs/hbaseSide.md index 60f0ef768..07b03026a 100644 --- a/docs/hbaseSide.md +++ b/docs/hbaseSide.md @@ -52,7 +52,7 @@ ## 4.样例 ``` CREATE TABLE sideTable( - cf:name String as name, + cf:name varchar as name, cf:info int as info, PRIMARY KEY(md5(name) + 'test'), PERIOD FOR SYSTEM_TIME diff --git a/docs/hbaseSink.md b/docs/hbaseSink.md index ff2f83091..b41abb281 100644 --- a/docs/hbaseSink.md +++ b/docs/hbaseSink.md @@ -41,7 +41,7 @@ hbase2.0 ## 5.样例: ``` CREATE TABLE MyResult( - cf:channel STRING, + cf:channel varchar, cf:pv BIGINT )WITH( type ='hbase', diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index 54bab7516..2968dbe22 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -44,8 +44,8 @@ CREATE TABLE tableName( ## 5.样例: ``` CREATE TABLE MyTable( - name string, - channel STRING, + name varchar, + channel varchar, pv INT, xctime bigint, CHARACTER_LENGTH(channel) AS timeLeng diff --git a/docs/mysqlSide.md b/docs/mysqlSide.md index f374a51c6..d0fec5832 100644 --- a/docs/mysqlSide.md +++ b/docs/mysqlSide.md @@ -57,7 +57,7 @@ ## 5.样例 ``` create table sideTable( - channel String, + channel varchar, xccount int, PRIMARY KEY(channel), PERIOD FOR SYSTEM_TIME From f2e013094c8a27b3f0a01e9c315bc9ea844a2cdc Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Thu, 8 Nov 2018 23:10:14 +0800 Subject: [PATCH 109/470] add mongo-all-side --- .../flink/sql/side/SideSqlExecTest.java | 295 ++++++++++++++++-- .../flink/sql/side/mongo/MongoAllReqRow.java | 163 ++++++---- .../sql/side/mongo/MongoAllSideInfo.java | 9 +- .../sql/side/mongo/table/MongoSideParser.java | 44 ++- .../side/mongo/table/MongoSideTableInfo.java | 42 ++- .../sql/sink/mongo/MongoOutputFormat.java | 9 - .../sql/sink/mongo/table/MongoSinkParser.java | 2 - mongo/pom.xml | 5 + 8 files changed, 412 insertions(+), 157 deletions(-) diff --git a/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java b/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java index e29277fee..8eb09ee18 100644 --- a/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java +++ b/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java @@ -16,11 +16,12 @@ * limitations under the License. */ - package com.dtstack.flink.sql.side; import com.dtstack.flink.sql.Main; +import com.dtstack.flink.sql.parser.SqlParser; +import com.dtstack.flink.sql.parser.SqlTree; import org.apache.flink.calcite.shaded.com.google.common.base.Charsets; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.junit.Test; @@ -34,6 +35,7 @@ * Reason: * Date: 2018/7/24 * Company: www.dtstack.com + * * @author xuchao */ @@ -133,36 +135,36 @@ public void testRunSideSql() throws Exception { @Test public void testRunHbaseSideTable() throws Exception { - String sql = "CREATE TABLE MyTable ( name string, channel STRING, pv INT, xctime bigint, CHARACTER_LENGTH(channel) AS timeLeng) " + - "WITH (type = 'kafka09',bootstrapServers = '172.16.8.198:9092',zookeeperQuorum = '172.16.8.198:2181/kafka', " + - "offsetReset = 'latest',topic = 'nbTest1',parallelism = '1');" + - "CREATE TABLE MyResult ( channel VARCHAR, pv VARCHAR) WITH (type = 'mysql'," + - "url = 'jdbc:mysql://172.16.8.104:3306/test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 'pv2'," + - "parallelism = '1');CREATE TABLE workerinfo ( cast(logtime as TIMESTAMP) AS rtime, cast(logtime) AS rtime) " + - "WITH (type = 'hbase',zookeeperQuorum = 'rdos1:2181',tableName = 'workerinfo',rowKey = 'ce,de'," + - "parallelism = '1',zookeeperParent = '/hbase');CREATE TABLE sideTable " + - "( cf:name String as name, cf:info String as info,PRIMARY KEY (name) , PERIOD FOR SYSTEM_TIME ) WITH " + - "(type = 'hbase',zookeeperQuorum = 'rdos1:2181',zookeeperParent = '/hbase',tableName = 'workerinfo',cache = 'LRU'," + - "cacheSize = '10000',cacheTTLMs = '60000',parallelism = '1');" + - "insert \n" + - "into\n" + - " MyResult\n" + - " select d.channel,d.info\n" + - " from\n" + - " (\n" + - " select\n" + - " *\n" + - " from\n" + - " MyTable a \n" + - " join\n" + - " sideTable b \n" + - " on a.channel=b.name \n" + - " where\n" + - " a.channel = 'xc2' \n" + - " and a.pv=10\n" + - " ) as d\n"; - - test(sql); + String sql = "CREATE TABLE MyTable ( name string, channel STRING, pv INT, xctime bigint, CHARACTER_LENGTH(channel) AS timeLeng) " + + "WITH (type = 'kafka09',bootstrapServers = '172.16.8.198:9092',zookeeperQuorum = '172.16.8.198:2181/kafka', " + + "offsetReset = 'latest',topic = 'nbTest1',parallelism = '1');" + + "CREATE TABLE MyResult ( channel VARCHAR, pv VARCHAR) WITH (type = 'mysql'," + + "url = 'jdbc:mysql://172.16.8.104:3306/test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 'pv2'," + + "parallelism = '1');CREATE TABLE workerinfo ( cast(logtime as TIMESTAMP) AS rtime, cast(logtime) AS rtime) " + + "WITH (type = 'hbase',zookeeperQuorum = 'rdos1:2181',tableName = 'workerinfo',rowKey = 'ce,de'," + + "parallelism = '1',zookeeperParent = '/hbase');CREATE TABLE sideTable " + + "( cf:name String as name, cf:info String as info,PRIMARY KEY (name) , PERIOD FOR SYSTEM_TIME ) WITH " + + "(type = 'hbase',zookeeperQuorum = 'rdos1:2181',zookeeperParent = '/hbase',tableName = 'workerinfo',cache = 'LRU'," + + "cacheSize = '10000',cacheTTLMs = '60000',parallelism = '1');" + + "insert \n" + + "into\n" + + " MyResult\n" + + " select d.channel,d.info\n" + + " from\n" + + " (\n" + + " select\n" + + " *\n" + + " from\n" + + " MyTable a \n" + + " join\n" + + " sideTable b \n" + + " on a.channel=b.name \n" + + " where\n" + + " a.channel = 'xc2' \n" + + " and a.pv=10\n" + + " ) as d\n"; + + test(sql); } @Test @@ -228,11 +230,11 @@ public void test(String sql) throws Exception { paramList.add("-name"); paramList.add("xc"); paramList.add("-localSqlPluginPath"); - paramList.add("D:\\gitspace\\flinkStreamSQL\\plugins"); + paramList.add("D:\\soucecode\\flinkStreamSQL\\plugins"); paramList.add("-mode"); paramList.add("local"); paramList.add("-addjar"); - paramList.add(URLEncoder.encode("[\"D:\\\\gitspace\\\\rdos-execution-engine\\\\..\\\\tmp140\\\\flink14Test-1.0-SNAPSHOT.jar\"]", Charsets.UTF_8.name())); + paramList.add(URLEncoder.encode("[\"D:\\\\soucecode\\\\rdos-execution-engine\\\\..\\\\tmp140\\\\flink14Test-1.0-SNAPSHOT.jar\"]", Charsets.UTF_8.name())); paramList.add("-remoteSqlPluginPath"); paramList.add("/opt/dtstack/flinkplugin"); paramList.add("-confProp"); @@ -244,4 +246,231 @@ public void test(String sql) throws Exception { paramList.toArray(params); Main.main(params); } + + @Test + public void testParseSql() throws Exception { + String sql = "CREATE TABLE bal_cur_batch_s30_pb_tranflow_input (trf_flowno varchar, trf_batno varchar, trf_cstno varchar, trf_bsncode varchar, trf_transtype varchar, trf_payacc varchar, trf_paysubacc varchar, trf_payname varchar, trf_rcvacc varchar, trf_rcvsubacc varchar, trf_rcvname varchar, trf_rcvbank varchar, trf_comitrno varchar, trf_crytype varchar, trf_subtime varchar, trf_tranamet varchar, trf_fee1 varchar, trf_fee2 varchar, trf_fee3 varchar, trf_userrem varchar, trf_hostflwno varchar, trf_hostsendtime varchar, trf_hosterror varchar, trf_lastupdtime varchar, trf_stt varchar, trf_schl_flowno varchar, trf_logontype varchar, trf_reccity varchar, trf_recprovice varchar, trf_channel varchar, trf_hosterrormsg varchar, trf_ext1 varchar, trf_ext2 varchar, trf_security varchar, cast(current_timestamp as varchar) as currTime) WITH (type = 'kafka09',bootstrapServers = 'kudu1:9092',zookeeperQuorum = '172.16.8.107:2181/kafka', offsetReset = 'latest',topic = 'tranflow_input',parallelism = '1');CREATE TABLE resultbank ( run_date VARCHAR, run_time VARCHAR, trf_flowno VARCHAR, trf_payname VARCHAR, trf_payacc VARCHAR, trf_tranamet VARCHAR, trf_subtime VARCHAR, trf_rcvbank VARCHAR,PRIMARY KEY (run_date,run_time,trf_flowno) ) WITH (type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/bank_test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 'resultbank',parallelism = '1');CREATE TABLE bal_cur_batch_rds_report_lrlct_trans_cur_input ( run_date VARCHAR, run_time VARCHAR, trf_flowno VARCHAR, trf_payname VARCHAR, trf_payacc VARCHAR, trf_subtime VARCHAR, trf_rcvbank VARCHAR,PRIMARY KEY (run_date,trf_flowno) , PERIOD FOR SYSTEM_TIME ) WITH (type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/bank_test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 'rds_report_lrlc_trans_cur',cache = 'LRU',cacheSize = '10000',cacheTTLMs = '60000',parallelism = '1');CREATE TABLE bal_cur_batch_t03_deposit_acct_base_info_cur_input ( data_dt VARCHAR, card_id VARCHAR, sub_acct_id VARCHAR, acct_org varchar, acct_stat_cd VARCHAR, core_cust_id VARCHAR, cust_rm VARCHAR, cust_scale_cd VARCHAR, item_id VARCHAR,PRIMARY KEY (card_id) , PERIOD FOR SYSTEM_TIME ) WITH (type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/bank_test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 't03_deposit_acct_base_info',cache = 'LRU',cacheSize = '10000',cacheTTLMs = '60000',parallelism = '1');CREATE TABLE bal_cur_batch_t04_org_cur_cur_input ( org_id VARCHAR, org_nm VARCHAR, org_short_nm VARCHAR, up_lvl_org_id VARCHAR, org_lvl VARCHAR, org_sort VARCHAR, org_cur VARCHAR,PRIMARY KEY (org_id) , PERIOD FOR SYSTEM_TIME ) WITH (type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/bank_test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 't04_org_cur',cache = 'LRU',cacheSize = '10000',cacheTTLMs = '60000',parallelism = '1');CREATE TABLE max_deposit_acct_base ( max_data_dt varchar, PRIMARY KEY (max_data_dt) , PERIOD FOR SYSTEM_TIME ) WITH (type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/bank_test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 'max_deposit_acct_base',cache = 'LRU',cacheSize = '10000',cacheTTLMs = '60000',parallelism = '1');\n" + + "INSERT \n" + + "INTO resultbank select d.run_date,\n" + + " d.run_time,\n" + + " d.trf_flowno,\n" + + " d.trf_payname,\n" + + " d.trf_payacc,\n" + + " d.trf_tranamet,\n" + + " d.trf_rcvbank,\n" + + " d.org_nm \n" + + " from\n" + + " (\n" + + " select\n" + + " cast(current_timestamp as varchar) as run_date,\n" + + " SUBSTRING(cast(current_timestamp as varchar) from 1 for 16) as run_time,\n" + + " b.trf_flowno,\n" + + " b.trf_payname,\n" + + " b.trf_payacc,\n" + + " b.trf_tranamet,\n" + + " b.trf_rcvbank,\n" + + " b.data_dt,\n" + + " t3.org_nm \n" + + " from\n" + + " ( select\n" + + " a.trf_flowno,\n" + + " a.currTime,\n" + + " a.trf_payname,\n" + + " a.trf_tranamet,\n" + + " a.trf_rcvbank,\n" + + " a.trf_payacc,\n" + + " a.trf_subtime,\n" + + " a.trf_bsncode,\n" + + " t2.acct_org,\n" + + " t2.data_dt,\n" + + " current_timestamp as nowtime\n" + + " from\n" + + " (select\n" + + " t1.trf_flowno,\n" + + " t1.currTime,\n" + + " t1.trf_payname,\n" + + " t1.trf_tranamet,\n" + + " t1.trf_rcvbank,\n" + + " t1.trf_subtime,\n" + + " t1.trf_payacc,\n" + + " t1.trf_bsncode\n" + + " from\n" + + " bal_cur_batch_s30_pb_tranflow_input t1 \n" + + " join\n" + + " bal_cur_batch_rds_report_lrlct_trans_cur_input x \n" + + " on t1.trf_flowno = x.trf_flowno \n" + + " and x.run_date = t1.currTime ) as a \n" + + " join\n" + + " bal_cur_batch_t03_deposit_acct_base_info_cur_input t2 \n" + + " on a.trf_payacc = t2.card_id \n" + + " ) as b \n" + + " join\n" + + " bal_cur_batch_t04_org_cur_cur_input t3 \n" + + " on b.acct_org = t3.org_id\n" + + " where \n" + + " b.trf_bsncode in('002002', '002003', '002011')\n" + + " and b.trf_flowno is null \n" + + " and substring(b.trf_subtime from 1 for 8) = DATE_FORMAT(b.nowtime, '%Y%m%d')\n" + + " and cast(b.trf_tranamet as decimal) >= 100000000\n" + + " ) as d\n" + + " join max_deposit_acct_base maxdep\n" + + " on d.data_dt = maxdep.max_data_dt\n"; +// sql = URLDecoder.decode(sql, org.apache.commons.io.Charsets.UTF_8.name()); + String localSqlPluginPath = "D:\\soucecode\\flinkStreamSQL\\plugins"; + SqlParser.setLocalSqlPluginRoot(localSqlPluginPath); + SqlTree sqlTree = SqlParser.parseSql(sql); + System.out.println("1111"); + } + + @Test + public void testParseSql2() throws Exception { + String sql = "CREATE TABLE MyTable(\n" + + " name varchar,\n" + + " channel varchar\n" + + " )WITH(\n" + + " type ='kafka10',\n" + + " bootstrapServers ='172.21.32.1:9092',\n" + + " zookeeperQuorum ='172.21.32.1:2181/kafka',\n" + + " offsetReset ='earliest',\n" + + " topic ='test1',\n" + + " parallelism ='3'\n" + + " );\n" + + " \n" + + " CREATE TABLE MyResult(\n" + + " name varchar,\n" + + " channel varchar\n" + + " )WITH(\n" + + " type ='mysql',\n" + + " url ='jdbc:mysql://127.0.0.1:3306/test?charset=utf8&useSSL=false',\n" + + " userName ='root',\n" + + " password ='123456',\n" + + " tableName ='pv',\n" + + " parallelism ='3'\n" + + " );\n" + + " \n" + + "insert into MyResult\n" + + "select a.name,a.channel from MyTable a"; + test2(sql); + } + + @Test + public void testParseMongo() throws Exception { + String sql = "CREATE TABLE MyTable(\n" + + " name varchar,\n" + + " channel varchar\n" + + " )WITH(\n" + + " type ='kafka10',\n" + + " bootstrapServers ='172.21.32.1:9092',\n" + + " zookeeperQuorum ='172.21.32.1:2181/kafka',\n" + + " offsetReset ='earliest',\n" + + " topic ='test',\n" + + " parallelism ='3'\n" + + " );\n" + + " \n" + + " CREATE TABLE MyResult(\n" + + " name varchar,\n" + + " channel varchar,\n" + + "\txccount int\n" + + " )WITH(\n" + + " type ='mongo',\n" + + " address ='172.21.32.1:27017,172.21.32.1:27017',\n" + + " database ='test',\n" + + " tableName ='pv',\n" + + " parallelism ='3'\n" + + " );\n" + + " \n" + + "create table sideTable(\n" + + " channel varchar,\n" + + " xccount int,\n" + + " PRIMARY KEY(channel),\n" + + " PERIOD FOR SYSTEM_TIME\n" + + " )WITH(\n" + + " type='mysql',\n" + + " url='jdbc:mysql://127.0.0.1:3306/test?charset=utf8&useSSL=true',\n" + + " userName='root',\n" + + " password='123456',\n" + + " tableName='sidetest',\n" + + " cache ='NONE',\n" + + " cacheSize ='10000',\n" + + " cacheTTLMs ='60000',\n" + + " parallelism ='1',\n" + + " partitionedJoin='false'\n" + + " );\n" + + " \n" + + "\n" + + "insert into MyResult\n" + + "select a.name,a.channel,b.xccount\n" + + "from MyTable a join sideTable b\n" + + "on a.channel=b.channel;\n"; + test2(sql); + } + + @Test + public void testParseMongo2() throws Exception { + String sql = "CREATE TABLE MyTable(\n" + + " name varchar,\n" + + " channel varchar\n" + + " )WITH(\n" + + " type ='kafka10',\n" + + " bootstrapServers ='172.21.32.1:9092',\n" + + " zookeeperQuorum ='172.21.32.1:2181/kafka',\n" + + " offsetReset ='earliest',\n" + + " topic ='test1',\n" + + " parallelism ='3'\n" + + " );\n" + + " \n" + + " CREATE TABLE MyResult(\n" + + " name varchar,\n" + + " channel varchar,\n" + + "\txccount int\n" + + " )WITH(\n" + + " type ='mongo',\n" + + " address ='172.21.32.1:27017,172.21.32.1:27017',\n" + + " database ='test',\n" + + " tableName ='pv',\n" + + " parallelism ='3'\n" + + " );\n" + + " \n" + + "create table sideTable(\n" + + " CHANNEL varchar,\n" + + " XCCOUNT int,\n" + + " PRIMARY KEY(channel),\n" + + " PERIOD FOR SYSTEM_TIME\n" + + " )WITH(\n" + + " type ='mongo',\n" + + " address ='172.21.32.1:27017,172.21.32.1:27017',\n" + + " database ='test',\n" + + " tableName ='sidetest',\n" + + " cache ='ALL',\n" + + " parallelism ='1',\n" + + " partitionedJoin='false'\n" + + " );\n" + + " \n" + + "insert into MyResult\n" + + "select a.name,a.channel,b.xccount\n" + + "from MyTable a join sideTable b\n" + + "on a.channel=b.channel;\n"; + test2(sql); + } + + public void test2(String sql) throws Exception { + List paramList = Lists.newArrayList(); + paramList.add("-sql"); + String exeSql = URLEncoder.encode(sql, Charsets.UTF_8.name()); + paramList.add(exeSql); + paramList.add("-name"); + paramList.add("xc"); + paramList.add("-localSqlPluginPath"); + paramList.add("D:\\soucecode\\flinkStreamSQL-my-src\\plugins"); + paramList.add("-mode"); + paramList.add("local"); + paramList.add("-confProp"); + String conf = "{\"time.characteristic\":\"ProcessingTime\",\"sql.checkpoint.interval\":10000}"; + String confEncode = URLEncoder.encode(conf, Charsets.UTF_8.name()); + paramList.add(confEncode); + + String[] params = new String[paramList.size()]; + paramList.toArray(params); + Main.main(params); + } } diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java index 395aba0a3..e1a398d4c 100644 --- a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java @@ -22,76 +22,82 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.mongo.MongoAllSideInfo; import com.dtstack.flink.sql.side.mongo.table.MongoSideTableInfo; -import com.dtstack.flink.sql.util.DtStringUtil; +import com.mongodb.BasicDBObject; +import com.mongodb.MongoClient; +import com.mongodb.MongoClientOptions; +import com.mongodb.MongoCredential; +import com.mongodb.ServerAddress; +import com.mongodb.client.FindIterable; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoCursor; +import com.mongodb.client.MongoDatabase; import org.apache.calcite.sql.JoinType; import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.apache.flink.util.Collector; +import org.bson.Document; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.ResultSet; import java.sql.SQLException; -import java.sql.Statement; import java.sql.Timestamp; +import java.util.ArrayList; import java.util.Calendar; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicReference; /** - * side operator with cache for all(period reload) - * Date: 2018/9/19 - * Company: www.dtstack.com - * @author xuchao + * Reason: + * Date: 2018/11/6 + * + * @author xuqianjin */ +public class MongoAllReqRow extends AllReqRow { -public class MongoAllReqRow extends AllReqRow{ - - private static final long serialVersionUID = 2098635140857937717L; + private static final long serialVersionUID = -675332795591842778L; private static final Logger LOG = LoggerFactory.getLogger(MongoAllReqRow.class); - private static final String Mongo_DRIVER = "com.mongo.jdbc.Driver"; - private static final int CONN_RETRY_NUM = 3; private static final int FETCH_SIZE = 1000; + private MongoClient mongoClient; + + private MongoDatabase db; + private AtomicReference>>> cacheRef = new AtomicReference<>(); - public MongoAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo){ + public MongoAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(new MongoAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } - @Override protected Row fillData(Row input, Object sideInput) { Map cacheInfo = (Map) sideInput; Row row = new Row(sideInfo.getOutFieldInfoList().size()); - for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ + for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { Object obj = input.getField(entry.getValue()); boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. - if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ - obj = ((Timestamp)obj).getTime(); + if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { + obj = ((Timestamp) obj).getTime(); } row.setField(entry.getKey(), obj); } - for(Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()){ - if(cacheInfo == null){ + for (Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()) { + if (cacheInfo == null) { row.setField(entry.getKey(), null); - }else{ + } else { row.setField(entry.getKey(), cacheInfo.get(entry.getValue())); } } @@ -120,13 +126,12 @@ protected void reloadCache() { LOG.info("----- Mongo all cacheRef reload end:{}", Calendar.getInstance()); } - @Override public void flatMap(Row value, Collector out) throws Exception { List inputParams = Lists.newArrayList(); - for(Integer conValIndex : sideInfo.getEqualValIndex()){ + for (Integer conValIndex : sideInfo.getEqualValIndex()) { Object equalObj = value.getField(conValIndex); - if(equalObj == null){ + if (equalObj == null) { out.collect(null); } @@ -135,103 +140,123 @@ public void flatMap(Row value, Collector out) throws Exception { String key = buildKey(inputParams); List> cacheList = cacheRef.get().get(key); - if(CollectionUtils.isEmpty(cacheList)){ - if(sideInfo.getJoinType() == JoinType.LEFT){ + if (CollectionUtils.isEmpty(cacheList)) { + if (sideInfo.getJoinType() == JoinType.LEFT) { Row row = fillData(value, null); out.collect(row); - }else{ + } else { return; } return; } - for(Map one : cacheList){ + for (Map one : cacheList) { + System.out.println(fillData(value, one)); out.collect(fillData(value, one)); } - } - private String buildKey(List equalValList){ + private String buildKey(List equalValList) { StringBuilder sb = new StringBuilder(""); - for(Object equalVal : equalValList){ + for (Object equalVal : equalValList) { sb.append(equalVal).append("_"); } return sb.toString(); } - private String buildKey(Map val, List equalFieldList){ + private String buildKey(Map val, List equalFieldList) { StringBuilder sb = new StringBuilder(""); - for(String equalField : equalFieldList){ + for (String equalField : equalFieldList) { sb.append(val.get(equalField)).append("_"); } return sb.toString(); } - private Connection getConn(String dbURL, String userName, String password){ - try{ - Class.forName(Mongo_DRIVER); - //add param useCursorFetch=true - Map addParams = Maps.newHashMap(); - addParams.put("useCursorFetch", "true"); - String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams,true); - return DriverManager.getConnection(targetDbUrl, userName, password); - }catch (Exception e){ - LOG.error("", e); - throw new RuntimeException("", e); + private MongoCollection getConn(String address, String userName, String password, String database, String tableName) { + MongoCollection dbCollection; + try { + MongoCredential credential; + String[] servers = address.split(","); + String host; + Integer port; + String[] hostAndPort; + List lists = new ArrayList<>(); + for (String server : servers) { + hostAndPort = server.split(":"); + host = hostAndPort[0]; + port = Integer.parseInt(hostAndPort[1]); + lists.add(new ServerAddress(host, port)); + } + if (!StringUtils.isEmpty(userName) || !StringUtils.isEmpty(password)) { + credential = MongoCredential.createCredential(userName, database, password.toCharArray()); + // To connect to mongodb server + mongoClient = new MongoClient(lists, credential, new MongoClientOptions.Builder().build()); + } else { + mongoClient = new MongoClient(lists); + } + db = mongoClient.getDatabase(database); + dbCollection = db.getCollection(tableName, Document.class); + return dbCollection; + } catch (Exception e) { + throw new RuntimeException("[connMongoDB]:" + e.getMessage()); } } - private void loadData(Map>> tmpCache) throws SQLException { MongoSideTableInfo tableInfo = (MongoSideTableInfo) sideInfo.getSideTableInfo(); - Connection connection = null; + MongoCollection dbCollection = null; - try{ - for(int i=0; i findIterable = dbCollection.find().projection(basicDBObject).limit(FETCH_SIZE); + MongoCursor mongoCursor = findIterable.iterator(); + while (mongoCursor.hasNext()) { + Document doc = mongoCursor.next(); Map oneRow = Maps.newHashMap(); - for(String fieldName : sideFieldNames){ - oneRow.put(fieldName.trim(), resultSet.getObject(fieldName.trim())); + for (String fieldName : sideFieldNames) { + oneRow.put(fieldName.trim(), doc.get(fieldName.trim())); } - String cacheKey = buildKey(oneRow, sideInfo.getEqualFieldList()); List> list = tmpCache.computeIfAbsent(cacheKey, key -> Lists.newArrayList()); list.add(oneRow); } - }catch (Exception e){ + } catch (Exception e) { LOG.error("", e); - }finally { - if(connection != null){ - connection.close(); + } finally { + try { + if (mongoClient != null) { + mongoClient.close(); + } + } catch (Exception e) { + throw new RuntimeException("[closeMongoDB]:" + e.getMessage()); } } } diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java index 7087da815..5d94531d2 100644 --- a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java @@ -34,14 +34,13 @@ /** * Reason: - * Date: 2018/9/19 - * Company: www.dtstack.com - * @author xuchao + * Date: 2018/11/6 + * + * @author xuqianjin */ - public class MongoAllSideInfo extends SideInfo{ - private static final long serialVersionUID = -5858335638589472158L; + private static final long serialVersionUID = -1696860430075523841L; public MongoAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); diff --git a/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideParser.java b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideParser.java index 3e0b4fe6a..a926053dc 100644 --- a/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideParser.java +++ b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideParser.java @@ -16,11 +16,9 @@ * limitations under the License. */ - package com.dtstack.flink.sql.side.mongo.table; -import com.dtstack.flink.sql.side.mongo.table.MongoSideTableInfo; import com.dtstack.flink.sql.table.AbsSideTableParser; import com.dtstack.flink.sql.table.TableInfo; import com.dtstack.flink.sql.util.MathUtil; @@ -29,18 +27,28 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; +import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; + /** * Reason: * Date: 2018/11/6 * * @author xuqianjin */ - - public class MongoSideParser extends AbsSideTableParser { private final static String SIDE_SIGN_KEY = "sideSignKey"; + public static final String ADDRESS_KEY = "address"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String USER_NAME_KEY = "userName"; + + public static final String PASSWORD_KEY = "password"; + + public static final String DATABASE_KEY = "database"; + private final static Pattern SIDE_TABLE_SIGN = Pattern.compile("(?i)^PERIOD\\s+FOR\\s+SYSTEM_TIME$"); static { @@ -50,20 +58,22 @@ public class MongoSideParser extends AbsSideTableParser { @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - MongoSideTableInfo MongoTableInfo = new MongoSideTableInfo(); - MongoTableInfo.setName(tableName); - parseFieldsInfo(fieldsInfo, MongoTableInfo); - - parseCacheProp(MongoTableInfo, props); - MongoTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(MongoSideTableInfo.PARALLELISM_KEY.toLowerCase()))); - MongoTableInfo.setUrl(MathUtil.getString(props.get(MongoSideTableInfo.URL_KEY.toLowerCase()))); - MongoTableInfo.setTableName(MathUtil.getString(props.get(MongoSideTableInfo.TABLE_NAME_KEY.toLowerCase()))); - MongoTableInfo.setUserName(MathUtil.getString(props.get(MongoSideTableInfo.USER_NAME_KEY.toLowerCase()))); - MongoTableInfo.setPassword(MathUtil.getString(props.get(MongoSideTableInfo.PASSWORD_KEY.toLowerCase()))); - - return MongoTableInfo; + MongoSideTableInfo mongoSideTableInfo = new MongoSideTableInfo(); + mongoSideTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, mongoSideTableInfo); + + parseCacheProp(mongoSideTableInfo, props); + + mongoSideTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase()))); + mongoSideTableInfo.setAddress(MathUtil.getString(props.get(ADDRESS_KEY.toLowerCase()))); + mongoSideTableInfo.setTableName(MathUtil.getString(props.get(TABLE_NAME_KEY.toLowerCase()))); + mongoSideTableInfo.setDatabase(MathUtil.getString(props.get(DATABASE_KEY.toLowerCase()))); + mongoSideTableInfo.setUserName(MathUtil.getString(props.get(USER_NAME_KEY.toLowerCase()))); + mongoSideTableInfo.setPassword(MathUtil.getString(props.get(PASSWORD_KEY.toLowerCase()))); + + return mongoSideTableInfo; } - private static void dealSideSign(Matcher matcher, TableInfo tableInfo){ + private static void dealSideSign(Matcher matcher, TableInfo tableInfo) { } } diff --git a/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideTableInfo.java b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideTableInfo.java index 488c92861..721960003 100644 --- a/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideTableInfo.java +++ b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideTableInfo.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package com.dtstack.flink.sql.side.mongo.table; @@ -29,36 +28,25 @@ * * @author xuqianjin */ - - public class MongoSideTableInfo extends SideTableInfo { private static final long serialVersionUID = -1L; - private static final String CURR_TYPE = "Mongo"; - - public static final String URL_KEY = "url"; - - public static final String TABLE_NAME_KEY = "tableName"; - - public static final String USER_NAME_KEY = "userName"; - - public static final String PASSWORD_KEY = "password"; + private static final String CURR_TYPE = "mongo"; - public MongoSideTableInfo(){ + public MongoSideTableInfo() { setType(CURR_TYPE); } @Override public boolean check() { - Preconditions.checkNotNull(url, "Mongo of URL is required"); - Preconditions.checkNotNull(tableName, "Mongo of tableName is required"); - Preconditions.checkNotNull(userName, "Mongo of userName is required"); - Preconditions.checkNotNull(password, "Mongo of password is required"); + Preconditions.checkNotNull(address, "Mongo field of ADDRESS is required"); + Preconditions.checkNotNull(database, "Mongo field of database is required"); + Preconditions.checkNotNull(tableName, "Mongo field of tableName is required"); return true; } - private String url; + private String address; private String tableName; @@ -66,12 +54,14 @@ public boolean check() { private String password; - public String getUrl() { - return url; + private String database; + + public String getAddress() { + return address; } - public void setUrl(String url) { - this.url = url; + public void setAddress(String address) { + this.address = address; } public String getTableName() { @@ -97,4 +87,12 @@ public String getPassword() { public void setPassword(String password) { this.password = password; } + + public String getDatabase() { + return database; + } + + public void setDatabase(String database) { + this.database = database; + } } diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java index c1ba890e0..f86be6212 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java @@ -19,29 +19,20 @@ package com.dtstack.flink.sql.sink.mongo; -import com.mongodb.BasicDBObject; -import com.mongodb.InsertOptions; import com.mongodb.MongoClient; import com.mongodb.MongoClientOptions; import com.mongodb.MongoCredential; import com.mongodb.ServerAddress; import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoDatabase; -import com.mongodb.client.model.InsertOneOptions; -import com.mongodb.client.model.UpdateOptions; import com.mongodb.client.result.UpdateResult; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.scala.row; import org.apache.flink.types.Row; -import org.apache.flink.util.Preconditions; -import org.bson.BSON; -import org.bson.BsonDocument; import org.bson.Document; -import org.bson.conversions.Bson; import org.bson.types.ObjectId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java index c8f1bdcd8..f812ff0f2 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoSinkParser.java @@ -19,10 +19,8 @@ package com.dtstack.flink.sql.sink.mongo.table; -import com.dtstack.flink.sql.sink.mongo.table.MongoTableInfo; import com.dtstack.flink.sql.table.AbsTableParser; import com.dtstack.flink.sql.table.TableInfo; -import com.dtstack.flink.sql.table.TableInfo; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; diff --git a/mongo/pom.xml b/mongo/pom.xml index b9f789f17..efe8b4afb 100644 --- a/mongo/pom.xml +++ b/mongo/pom.xml @@ -34,6 +34,11 @@ mongo-java-driver 3.8.2 + + org.mongodb + mongodb-driver-async + 3.8.2 + From 34a28f8dc6caec5f2a7b311ee6e9f6c1efbb0719 Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Fri, 9 Nov 2018 11:08:32 +0800 Subject: [PATCH 110/470] add mongo-async-side unfinish --- .../sql/side/mongo/MongoAsyncReqRow.java | 201 ++++++++++-------- .../sql/side/mongo/MongoAsyncSideInfo.java | 8 +- 2 files changed, 114 insertions(+), 95 deletions(-) diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index d658e6be2..d854bb788 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package com.dtstack.flink.sql.side.mongo; @@ -27,16 +26,18 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.cache.CacheObj; -import com.dtstack.flink.sql.side.mongo.MongoAsyncSideInfo; import com.dtstack.flink.sql.side.mongo.table.MongoSideTableInfo; -import io.vertx.core.Vertx; -import io.vertx.core.VertxOptions; +import com.mongodb.MongoCredential; +import com.mongodb.ServerAddress; +import com.mongodb.async.client.MongoClientSettings; +import com.mongodb.async.client.MongoClients; +import com.mongodb.async.client.MongoDatabase; +import com.mongodb.connection.ClusterSettings; import io.vertx.core.json.JsonArray; -import io.vertx.core.json.JsonObject; -import io.vertx.ext.jdbc.JDBCClient; import io.vertx.ext.sql.SQLClient; import io.vertx.ext.sql.SQLConnection; -import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; +import io.vertx.core.json.JsonArray; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.configuration.Configuration; @@ -47,26 +48,25 @@ import org.slf4j.LoggerFactory; import java.sql.Timestamp; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; /** - * Mongo dim table - * Date: 2018/7/27 - * Company: www.dtstack.com - * @author xuchao + * Reason: + * Date: 2018/11/6 + * + * @author xuqianjin */ - public class MongoAsyncReqRow extends AsyncReqRow { - - private static final long serialVersionUID = 2098635244857937717L; + private static final long serialVersionUID = -1183158242862673706L; private static final Logger LOG = LoggerFactory.getLogger(MongoAsyncReqRow.class); private transient SQLClient MongoClient; - private final static String Mongo_DRIVER = "com.Mongo.jdbc.Driver"; + private final static String Mongo_DRIVER = "com.mongo.jdbc.Driver"; private final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 10; @@ -74,6 +74,9 @@ public class MongoAsyncReqRow extends AsyncReqRow { private final static int DEFAULT_MAX_DB_CONN_POOL_SIZE = 20; + private com.mongodb.async.client.MongoClient mongoClient; + + private MongoDatabase db; public MongoAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(new MongoAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); @@ -83,28 +86,43 @@ public MongoAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List lists = new ArrayList<>(); + for (String server : servers) { + hostAndPort = server.split(":"); + host = hostAndPort[0]; + port = Integer.parseInt(hostAndPort[1]); + lists.add(new ServerAddress(host, port)); + } + ClusterSettings clusterSettings = ClusterSettings.builder().hosts(lists).build(); + if (!StringUtils.isEmpty(MongoSideTableInfo.getUserName()) || !StringUtils.isEmpty(MongoSideTableInfo.getPassword())) { + mongoCredential = MongoCredential.createCredential(MongoSideTableInfo.getUserName(), MongoSideTableInfo.getDatabase(), + MongoSideTableInfo.getPassword().toCharArray()); + MongoClientSettings settings = MongoClientSettings.builder().credential(mongoCredential).clusterSettings(clusterSettings).build(); + mongoClient = MongoClients.create(settings); + } else { + MongoClientSettings settings = MongoClientSettings.builder().clusterSettings(clusterSettings).build(); + mongoClient = MongoClients.create(settings); + } + db = mongoClient.getDatabase(MongoSideTableInfo.getDatabase()); } @Override public void asyncInvoke(Row input, ResultFuture resultFuture) throws Exception { JsonArray inputParams = new JsonArray(); - for(Integer conValIndex : sideInfo.getEqualValIndex()){ + for (Integer conValIndex : sideInfo.getEqualValIndex()) { Object equalObj = input.getField(conValIndex); - if(equalObj == null){ + if (equalObj == null) { resultFuture.complete(null); } @@ -112,95 +130,98 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except } String key = buildCacheKey(inputParams); - if(openCache()){ + System.out.println("inputParams:"+inputParams); + System.out.println("key:"+key); + if (openCache()) { CacheObj val = getFromCache(key); - if(val != null){ + if (val != null) { - if(ECacheContentType.MissVal == val.getType()){ + if (ECacheContentType.MissVal == val.getType()) { dealMissKey(input, resultFuture); return; - }else if(ECacheContentType.MultiLine == val.getType()){ + } else if (ECacheContentType.MultiLine == val.getType()) { - for(Object jsonArray : (List)val.getContent()){ + for (Object jsonArray : (List) val.getContent()) { Row row = fillData(input, jsonArray); resultFuture.complete(Collections.singleton(row)); } - }else{ + } else { throw new RuntimeException("not support cache obj type " + val.getType()); } return; } } - MongoClient.getConnection(conn -> { - if (conn.failed()) { - //Treatment failures - resultFuture.completeExceptionally(conn.cause()); - return; - } - - final SQLConnection connection = conn.result(); - String sqlCondition = sideInfo.getSqlCondition(); - connection.queryWithParams(sqlCondition, inputParams, rs -> { - if (rs.failed()) { - LOG.error("Cannot retrieve the data from the database"); - LOG.error("", rs.cause()); - resultFuture.complete(null); - return; - } - - List cacheContent = Lists.newArrayList(); - - int resultSize = rs.result().getResults().size(); - if(resultSize > 0){ - for (JsonArray line : rs.result().getResults()) { - Row row = fillData(input, line); - if(openCache()){ - cacheContent.add(line); - } - resultFuture.complete(Collections.singleton(row)); - } - if(openCache()){ - putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); - } - }else{ - dealMissKey(input, resultFuture); - if(openCache()){ - putCache(key, CacheMissVal.getMissKeyObj()); - } - } - - // and close the connection - connection.close(done -> { - if (done.failed()) { - throw new RuntimeException(done.cause()); - } - }); - }); - }); +// MongoClient.getConnection(conn -> { +// if (conn.failed()) { +// //Treatment failures +// resultFuture.completeExceptionally(conn.cause()); +// return; +// } +// +// final SQLConnection connection = conn.result(); +// String sqlCondition = sideInfo.getSqlCondition(); +// connection.queryWithParams(sqlCondition, inputParams, rs -> { +// if (rs.failed()) { +// LOG.error("Cannot retrieve the data from the database"); +// LOG.error("", rs.cause()); +// resultFuture.complete(null); +// return; +// } +// +// List cacheContent = Lists.newArrayList(); +// +// int resultSize = rs.result().getResults().size(); +// if (resultSize > 0) { +// for (JsonArray line : rs.result().getResults()) { +// Row row = fillData(input, line); +// if (openCache()) { +// cacheContent.add(line); +// } +// resultFuture.complete(Collections.singleton(row)); +// } +// +// if (openCache()) { +// putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); +// } +// } else { +// dealMissKey(input, resultFuture); +// if (openCache()) { +// putCache(key, CacheMissVal.getMissKeyObj()); +// } +// } +// +// // and close the connection +// connection.close(done -> { +// if (done.failed()) { +// throw new RuntimeException(done.cause()); +// } +// }); +// }); +// }); } @Override - public Row fillData(Row input, Object line){ + public Row fillData(Row input, Object line) { JsonArray jsonArray = (JsonArray) line; Row row = new Row(sideInfo.getOutFieldInfoList().size()); - for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ + for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { Object obj = input.getField(entry.getValue()); boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); - if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ - obj = ((Timestamp)obj).getTime(); + if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { + obj = ((Timestamp) obj).getTime(); } row.setField(entry.getKey(), obj); } - for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ - if(jsonArray == null){ + for (Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()) { + if (jsonArray == null) { row.setField(entry.getKey(), null); - }else{ + } else { row.setField(entry.getKey(), jsonArray.getValue(entry.getValue())); } } @@ -214,9 +235,9 @@ public void close() throws Exception { MongoClient.close(); } - public String buildCacheKey(JsonArray jsonArray){ + public String buildCacheKey(JsonArray jsonArray) { StringBuilder sb = new StringBuilder(); - for(Object ele : jsonArray.getList()){ + for (Object ele : jsonArray.getList()) { sb.append(ele.toString()) .append("_"); } diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java index 7ef5b2d5e..b09a12474 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java @@ -34,15 +34,13 @@ /** * Reason: - * Date: 2018/9/18 - * Company: www.dtstack.com + * Date: 2018/11/6 * - * @author xuchao + * @author xuqianjin */ - public class MongoAsyncSideInfo extends SideInfo { - private static final long serialVersionUID = -5931494270201575201L; + private static final long serialVersionUID = -3694857194254465989L; public MongoAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); From 2b29fc9a29fd109df79540f90917e7481171f234 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 9 Nov 2018 11:26:19 +0800 Subject: [PATCH 111/470] bugfix --- .../sql/side/mongo/MongoAsyncReqRow.java | 90 ++++++++++++------- 1 file changed, 58 insertions(+), 32 deletions(-) diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index 21baa5a56..0e0c352bb 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -64,9 +64,9 @@ public class MongoAsyncReqRow extends AsyncReqRow { private static final Logger LOG = LoggerFactory.getLogger(MongoAsyncReqRow.class); - private transient SQLClient mongoClient; + private transient SQLClient MongoClient; - private final static String mongo_driver = "com.Mongo.jdbc.Driver"; + private final static String Mongo_DRIVER = "com.mongo.jdbc.Driver"; private final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 10; @@ -130,8 +130,8 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except } String key = buildCacheKey(inputParams); - System.out.println("inputParams:"+inputParams); - System.out.println("key:"+key); + System.out.println("inputParams:" + inputParams); + System.out.println("key:" + key); if (openCache()) { CacheObj val = getFromCache(key); if (val != null) { @@ -152,34 +152,54 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except return; } } - mongoClient.getConnection(conn -> { - if (conn.failed()) { - //Treatment failures - resultFuture.completeExceptionally(conn.cause()); - return; - } - - final SQLConnection connection = conn.result(); - String sqlCondition = sideInfo.getSqlCondition(); - connection.queryWithParams(sqlCondition, inputParams, rs -> { - if (rs.failed()) { - LOG.error("Cannot retrieve the data from the database"); - LOG.error("", rs.cause()); - resultFuture.complete(null); - return; - } - List cacheContent = Lists.newArrayList(); - - int resultSize = rs.result().getResults().size(); - if(resultSize > 0){ - for (JsonArray line : rs.result().getResults()) { - Row row = fillData(input, line); - if(openCache()){ - cacheContent.add(line); - } - resultFuture.complete(Collections.singleton(row)); - } +// MongoClient.getConnection(conn -> { +// if (conn.failed()) { +// //Treatment failures +// resultFuture.completeExceptionally(conn.cause()); +// return; +// } +// +// final SQLConnection connection = conn.result(); +// String sqlCondition = sideInfo.getSqlCondition(); +// connection.queryWithParams(sqlCondition, inputParams, rs -> { +// if (rs.failed()) { +// LOG.error("Cannot retrieve the data from the database"); +// LOG.error("", rs.cause()); +// resultFuture.complete(null); +// return; +// } +// +// List cacheContent = Lists.newArrayList(); +// +// int resultSize = rs.result().getResults().size(); +// if (resultSize > 0) { +// for (JsonArray line : rs.result().getResults()) { +// Row row = fillData(input, line); +// if (openCache()) { +// cacheContent.add(line); +// } +// resultFuture.complete(Collections.singleton(row)); +// } +// +// if (openCache()) { +// putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); +// } +// } else { +// dealMissKey(input, resultFuture); +// if (openCache()) { +// putCache(key, CacheMissVal.getMissKeyObj()); +// } +// } +// +// // and close the connection +// connection.close(done -> { +// if (done.failed()) { +// throw new RuntimeException(done.cause()); +// } +// }); +// }); +// }); } @Override @@ -211,7 +231,13 @@ public Row fillData(Row input, Object line) { @Override public void close() throws Exception { super.close(); - mongoClient.close(); + try { + if (mongoClient != null) { + mongoClient.close(); + } + } catch (Exception e) { + throw new RuntimeException("[closeMongoDB]:" + e.getMessage()); + } } public String buildCacheKey(JsonArray jsonArray) { From 9b5c3561fb0e5e7198732ae303255dbc471dcf0f Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Fri, 9 Nov 2018 16:02:42 +0800 Subject: [PATCH 112/470] add mongo-async-side --- .../sql/side/mongo/MongoAsyncReqRow.java | 146 ++++++++---------- 1 file changed, 68 insertions(+), 78 deletions(-) diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index d854bb788..5fc689cbf 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -21,22 +21,23 @@ import com.dtstack.flink.sql.enums.ECacheContentType; import com.dtstack.flink.sql.side.AsyncReqRow; -import com.dtstack.flink.sql.side.CacheMissVal; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.mongo.table.MongoSideTableInfo; +import com.mongodb.BasicDBObject; +import com.mongodb.Block; import com.mongodb.MongoCredential; import com.mongodb.ServerAddress; +import com.mongodb.async.SingleResultCallback; +import com.mongodb.async.client.MongoClient; import com.mongodb.async.client.MongoClientSettings; import com.mongodb.async.client.MongoClients; +import com.mongodb.async.client.MongoCollection; import com.mongodb.async.client.MongoDatabase; import com.mongodb.connection.ClusterSettings; -import io.vertx.core.json.JsonArray; -import io.vertx.ext.sql.SQLClient; -import io.vertx.ext.sql.SQLConnection; -import io.vertx.core.json.JsonArray; +import com.mongodb.connection.ConnectionPoolSettings; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; @@ -44,14 +45,17 @@ import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; +import org.bson.Document; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.sql.Timestamp; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; /** * Reason: @@ -64,33 +68,26 @@ public class MongoAsyncReqRow extends AsyncReqRow { private static final Logger LOG = LoggerFactory.getLogger(MongoAsyncReqRow.class); - private transient SQLClient MongoClient; - - private final static String Mongo_DRIVER = "com.mongo.jdbc.Driver"; - - private final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 10; - - private final static int DEFAULT_VERTX_WORKER_POOL_SIZE = 20; - private final static int DEFAULT_MAX_DB_CONN_POOL_SIZE = 20; - private com.mongodb.async.client.MongoClient mongoClient; + private transient MongoClient mongoClient; private MongoDatabase db; + private MongoSideTableInfo MongoSideTableInfo; + public MongoAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(new MongoAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } - @Override public void open(Configuration parameters) throws Exception { super.open(parameters); + MongoSideTableInfo = (MongoSideTableInfo) sideInfo.getSideTableInfo(); connMongoDB(); } public void connMongoDB() throws Exception { - MongoSideTableInfo MongoSideTableInfo = (MongoSideTableInfo) sideInfo.getSideTableInfo(); MongoCredential mongoCredential; String[] servers = MongoSideTableInfo.getAddress().split(","); String host; @@ -104,13 +101,21 @@ public void connMongoDB() throws Exception { lists.add(new ServerAddress(host, port)); } ClusterSettings clusterSettings = ClusterSettings.builder().hosts(lists).build(); + ConnectionPoolSettings connectionPoolSettings = ConnectionPoolSettings.builder() + .maxSize(DEFAULT_MAX_DB_CONN_POOL_SIZE) + .build(); if (!StringUtils.isEmpty(MongoSideTableInfo.getUserName()) || !StringUtils.isEmpty(MongoSideTableInfo.getPassword())) { mongoCredential = MongoCredential.createCredential(MongoSideTableInfo.getUserName(), MongoSideTableInfo.getDatabase(), MongoSideTableInfo.getPassword().toCharArray()); - MongoClientSettings settings = MongoClientSettings.builder().credential(mongoCredential).clusterSettings(clusterSettings).build(); + MongoClientSettings settings = MongoClientSettings.builder().credential(mongoCredential) + .clusterSettings(clusterSettings) + .connectionPoolSettings(connectionPoolSettings) + .build(); mongoClient = MongoClients.create(settings); } else { - MongoClientSettings settings = MongoClientSettings.builder().clusterSettings(clusterSettings).build(); + MongoClientSettings settings = MongoClientSettings.builder().clusterSettings(clusterSettings) + .connectionPoolSettings(connectionPoolSettings) + .build(); mongoClient = MongoClients.create(settings); } db = mongoClient.getDatabase(MongoSideTableInfo.getDatabase()); @@ -119,19 +124,17 @@ public void connMongoDB() throws Exception { @Override public void asyncInvoke(Row input, ResultFuture resultFuture) throws Exception { - JsonArray inputParams = new JsonArray(); - for (Integer conValIndex : sideInfo.getEqualValIndex()) { + BasicDBObject basicDBObject = new BasicDBObject(); + for (int i = 0; i < sideInfo.getEqualFieldList().size(); i++) { + Integer conValIndex = sideInfo.getEqualValIndex().get(i); Object equalObj = input.getField(conValIndex); if (equalObj == null) { resultFuture.complete(null); } - - inputParams.add(equalObj); + basicDBObject.put(sideInfo.getEqualFieldList().get(i), equalObj); } - String key = buildCacheKey(inputParams); - System.out.println("inputParams:"+inputParams); - System.out.println("key:"+key); + String key = buildCacheKey(basicDBObject.values()); if (openCache()) { CacheObj val = getFromCache(key); if (val != null) { @@ -153,59 +156,40 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except } } + AtomicInteger atomicInteger = new AtomicInteger(0); + MongoCollection dbCollection = db.getCollection(MongoSideTableInfo.getTableName(), Document.class); + List cacheContent = Lists.newArrayList(); + Block printDocumentBlock = new Block() { + @Override + public void apply(final Document document) { + atomicInteger.incrementAndGet(); + Row row = fillData(input, document); + if (openCache()) { + cacheContent.add(document); + } + resultFuture.complete(Collections.singleton(row)); + } + }; + SingleResultCallback callbackWhenFinished = new SingleResultCallback() { + @Override + public void onResult(final Void result, final Throwable t) { + if (atomicInteger.get() <= 0) { + LOG.warn("Cannot retrieve the data from the database"); + resultFuture.complete(null); + } else { + if (openCache()) { + putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); + } + } + } + }; -// MongoClient.getConnection(conn -> { -// if (conn.failed()) { -// //Treatment failures -// resultFuture.completeExceptionally(conn.cause()); -// return; -// } -// -// final SQLConnection connection = conn.result(); -// String sqlCondition = sideInfo.getSqlCondition(); -// connection.queryWithParams(sqlCondition, inputParams, rs -> { -// if (rs.failed()) { -// LOG.error("Cannot retrieve the data from the database"); -// LOG.error("", rs.cause()); -// resultFuture.complete(null); -// return; -// } -// -// List cacheContent = Lists.newArrayList(); -// -// int resultSize = rs.result().getResults().size(); -// if (resultSize > 0) { -// for (JsonArray line : rs.result().getResults()) { -// Row row = fillData(input, line); -// if (openCache()) { -// cacheContent.add(line); -// } -// resultFuture.complete(Collections.singleton(row)); -// } -// -// if (openCache()) { -// putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); -// } -// } else { -// dealMissKey(input, resultFuture); -// if (openCache()) { -// putCache(key, CacheMissVal.getMissKeyObj()); -// } -// } -// -// // and close the connection -// connection.close(done -> { -// if (done.failed()) { -// throw new RuntimeException(done.cause()); -// } -// }); -// }); -// }); + dbCollection.find(basicDBObject).forEach(printDocumentBlock, callbackWhenFinished); } @Override public Row fillData(Row input, Object line) { - JsonArray jsonArray = (JsonArray) line; + Document doc = (Document) line; Row row = new Row(sideInfo.getOutFieldInfoList().size()); for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { Object obj = input.getField(entry.getValue()); @@ -219,10 +203,10 @@ public Row fillData(Row input, Object line) { } for (Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()) { - if (jsonArray == null) { + if (doc == null) { row.setField(entry.getKey(), null); } else { - row.setField(entry.getKey(), jsonArray.getValue(entry.getValue())); + row.setField(entry.getKey(), doc.get(sideInfo.getSideFieldNameIndex().get(entry.getKey()))); } } @@ -232,12 +216,18 @@ public Row fillData(Row input, Object line) { @Override public void close() throws Exception { super.close(); - MongoClient.close(); + try { + if (mongoClient != null) { + mongoClient.close(); + } + } catch (Exception e) { + throw new RuntimeException("[closeMongoDB]:" + e.getMessage()); + } } - public String buildCacheKey(JsonArray jsonArray) { + public String buildCacheKey(Collection collection) { StringBuilder sb = new StringBuilder(); - for (Object ele : jsonArray.getList()) { + for (Object ele : collection) { sb.append(ele.toString()) .append("_"); } From 54e9748892b93260f74021a1640d582f78add49b Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Fri, 9 Nov 2018 16:09:56 +0800 Subject: [PATCH 113/470] add mongoSide.md --- docs/mongoSide.md | 78 +++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 78 insertions(+) create mode 100644 docs/mongoSide.md diff --git a/docs/mongoSide.md b/docs/mongoSide.md new file mode 100644 index 000000000..db557f441 --- /dev/null +++ b/docs/mongoSide.md @@ -0,0 +1,78 @@ + +## 1.格式: +``` + CREATE TABLE tableName( + colName cloType, + ... + PRIMARY KEY(keyInfo), + PERIOD FOR SYSTEM_TIME + )WITH( + type ='mongo', + address ='ip:port[,ip:port]', + userName='dbUserName', + password='dbPwd', + tableName='tableName', + database='database', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' + ); +``` + +# 2.支持版本 + mongo-3.8.2 + +## 3.表结构定义 + + |参数名称|含义| + |----|---| + | tableName | 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同)| + | colName | 列名称| + | colType | 列类型 [colType支持的类型](colType.md)| + | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| + | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| + +## 4.参数 + + |参数名称|含义|是否必填|默认值| + |----|---|---|----| + | type |表明 输出表类型 mongo|是|| + | address | 连接mongo数据库 jdbcUrl |是|| + | userName | mongo连接用户名|否|| + | password | mongo连接密码|否|| + | tableName | mongo表名称|是|| + | database | mongo表名称|是|| + | cache | 维表缓存策略(NONE/LRU)|否|NONE| + | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| + + ---------- + > 缓存策略 + * NONE: 不做内存缓存 + * LRU: + * cacheSize: 缓存的条目数量 + * cacheTTLMs:缓存的过期时间(ms) + + +## 5.样例 +``` +create table sideTable( + CHANNEL varchar, + XCCOUNT int, + PRIMARY KEY(channel), + PERIOD FOR SYSTEM_TIME + )WITH( + type ='mongo', + address ='172.21.32.1:27017,172.21.32.1:27017', + database ='test', + tableName ='sidetest', + cache ='LRU', + parallelism ='1', + partitionedJoin='false' + ); + + +``` + + From 55d90f6c3dec812ac81e1ffe4ee020b130ac8f53 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Fri, 9 Nov 2018 22:06:28 +0800 Subject: [PATCH 114/470] add redis sidetable --- pom.xml | 1 + redis5/pom.xml | 20 ++ redis5/redis5-side/pom.xml | 32 +++ redis5/redis5-side/redis-all-side/pom.xml | 18 ++ .../flink/sql/side/redis/RedisAllReqRow.java | 22 ++ .../sql/side/redis/RedisAllSideInfo.java | 22 ++ redis5/redis5-side/redis-async-side/pom.xml | 18 ++ .../sql/side/redis/RedisAsyncReqRow.java | 22 ++ .../sql/side/redis/RedisAsyncSideInfo.java | 22 ++ redis5/redis5-side/redis-side-core/pom.xml | 18 ++ .../sql/side/redis/table/RedisSideParser.java | 49 +++++ .../side/redis/table/RedisSideTableInfo.java | 78 +++++++ redis5/redis5-sink/pom.xml | 94 ++++++++ .../sql/sink/redis/RedisOutputFormat.java | 205 ++++++++++++++++++ .../flink/sql/sink/redis/RedisSink.java | 108 +++++++++ .../sql/sink/redis/table/RedisSinkParser.java | 40 ++++ .../sql/sink/redis/table/RedisTableInfo.java | 93 ++++++++ 17 files changed, 862 insertions(+) create mode 100644 redis5/pom.xml create mode 100644 redis5/redis5-side/pom.xml create mode 100644 redis5/redis5-side/redis-all-side/pom.xml create mode 100644 redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java create mode 100644 redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java create mode 100644 redis5/redis5-side/redis-async-side/pom.xml create mode 100644 redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java create mode 100644 redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java create mode 100644 redis5/redis5-side/redis-side-core/pom.xml create mode 100644 redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java create mode 100644 redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideTableInfo.java create mode 100644 redis5/redis5-sink/pom.xml create mode 100644 redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java create mode 100644 redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java create mode 100644 redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java create mode 100644 redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java diff --git a/pom.xml b/pom.xml index b2df3e78e..07694589c 100644 --- a/pom.xml +++ b/pom.xml @@ -14,6 +14,7 @@ hbase elasticsearch5 mongo + redis5 launcher pom diff --git a/redis5/pom.xml b/redis5/pom.xml new file mode 100644 index 000000000..894f786ef --- /dev/null +++ b/redis5/pom.xml @@ -0,0 +1,20 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + sql.redis + pom + + + redis5-sink + redis5-side + + + + \ No newline at end of file diff --git a/redis5/redis5-side/pom.xml b/redis5/redis5-side/pom.xml new file mode 100644 index 000000000..6f98376c9 --- /dev/null +++ b/redis5/redis5-side/pom.xml @@ -0,0 +1,32 @@ + + + 4.0.0 + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.redis + 1.0-SNAPSHOT + + + com.dtstack.flink + redis5-side + 1.0-SNAPSHOT + + + redis-side-core + redis-async-side + redis-all-side + + + pom + + \ No newline at end of file diff --git a/redis5/redis5-side/redis-all-side/pom.xml b/redis5/redis5-side/redis-all-side/pom.xml new file mode 100644 index 000000000..5dc304135 --- /dev/null +++ b/redis5/redis5-side/redis-all-side/pom.xml @@ -0,0 +1,18 @@ + + + 4.0.0 + + + com.dtstack.flink + redis5-side + 1.0-SNAPSHOT + + + com.dtstack.flink + redis-all-side + 1.0-SNAPSHOT + + + \ No newline at end of file diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java new file mode 100644 index 000000000..2b09e518b --- /dev/null +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java @@ -0,0 +1,22 @@ +/* + * 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 com.dtstack.flink.sql.side.redis; + +public class RedisAllReqRow { +} diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java new file mode 100644 index 000000000..7b15278f9 --- /dev/null +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java @@ -0,0 +1,22 @@ +/* + * 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 com.dtstack.flink.sql.side.redis; + +public class RedisAllSideInfo { +} diff --git a/redis5/redis5-side/redis-async-side/pom.xml b/redis5/redis5-side/redis-async-side/pom.xml new file mode 100644 index 000000000..1f1fcc921 --- /dev/null +++ b/redis5/redis5-side/redis-async-side/pom.xml @@ -0,0 +1,18 @@ + + + 4.0.0 + + + com.dtstack.flink + redis5-side + 1.0-SNAPSHOT + + + com.dtstack.flink + redis-async-side + 1.0-SNAPSHOT + + + \ No newline at end of file diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java new file mode 100644 index 000000000..946149a67 --- /dev/null +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -0,0 +1,22 @@ +/* + * 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 com.dtstack.flink.sql.side.redis; + +public class RedisAsyncReqRow { +} diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java new file mode 100644 index 000000000..6a89c884c --- /dev/null +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java @@ -0,0 +1,22 @@ +/* + * 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 com.dtstack.flink.sql.side.redis; + +public class RedisAsyncSideInfo { +} diff --git a/redis5/redis5-side/redis-side-core/pom.xml b/redis5/redis5-side/redis-side-core/pom.xml new file mode 100644 index 000000000..5876c2e48 --- /dev/null +++ b/redis5/redis5-side/redis-side-core/pom.xml @@ -0,0 +1,18 @@ + + + 4.0.0 + + + com.dtstack.flink + redis5-side + 1.0-SNAPSHOT + + + com.dtstack.flink + redis-side-core + 1.0-SNAPSHOT + + + \ No newline at end of file diff --git a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java new file mode 100644 index 000000000..ae3f2abe0 --- /dev/null +++ b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java @@ -0,0 +1,49 @@ +/* + * 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 com.dtstack.flink.sql.side.redis.table; + +import com.dtstack.flink.sql.table.AbsSideTableParser; +import com.dtstack.flink.sql.table.TableInfo; + +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class RedisSideParser extends AbsSideTableParser { + + private final static String SIDE_SIGN_KEY = "sideSignKey"; + + private final static Pattern SIDE_TABLE_SIGN = Pattern.compile("(?i)^PERIOD\\s+FOR\\s+SYSTEM_TIME$"); + + static { + keyPatternMap.put(SIDE_SIGN_KEY, SIDE_TABLE_SIGN); + keyHandlerMap.put(SIDE_SIGN_KEY, RedisSideParser::dealSideSign); + } + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + RedisSideTableInfo redisSideTableInfo = new RedisSideTableInfo(); + + + return redisSideTableInfo; + } + + private static void dealSideSign(Matcher matcher, TableInfo tableInfo){ + } +} diff --git a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideTableInfo.java b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideTableInfo.java new file mode 100644 index 000000000..b82d142ef --- /dev/null +++ b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideTableInfo.java @@ -0,0 +1,78 @@ +/* + * 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 com.dtstack.flink.sql.side.redis.table; + +import com.dtstack.flink.sql.side.SideTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +public class RedisSideTableInfo extends SideTableInfo { + + private static final long serialVersionUID = -1L; + + private static final String CURR_TYPE = "redis"; + + public static final String URL_KEY = "url"; + + public static final String USER_NAME_KEY = "database"; + + public static final String PASSWORD_KEY = "password"; + + private String url; + + private String database; + + private String password; + + public RedisSideTableInfo(){ + setType(CURR_TYPE); + } + + + @Override + public boolean check() { + Preconditions.checkNotNull(url, "redis of URL is required"); + Preconditions.checkNotNull(database, "redis of database is required"); + Preconditions.checkNotNull(password, "redis of password is required"); + return true; + } + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public String getDatabase() { + return database; + } + + public void setDatabase(String database) { + this.database = database; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } +} diff --git a/redis5/redis5-sink/pom.xml b/redis5/redis5-sink/pom.xml new file mode 100644 index 000000000..ae54fec10 --- /dev/null +++ b/redis5/redis5-sink/pom.xml @@ -0,0 +1,94 @@ + + + + sql.redis + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.redis + jar + + redis-sink + http://maven.apache.org + + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + + + redis.clients + jedis + 2.8.0 + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java new file mode 100644 index 000000000..acae676a3 --- /dev/null +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java @@ -0,0 +1,205 @@ +/* + * 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 com.dtstack.flink.sql.sink.redis; + +import org.apache.commons.pool2.impl.GenericObjectPoolConfig; +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.types.Row; +import redis.clients.jedis.*; + +import java.io.IOException; +import java.util.*; + +public class RedisOutputFormat extends RichOutputFormat { + + private String url; + + private String database; + + private String password; + + protected String[] fieldNames; + + protected TypeInformation[] fieldTypes; + + protected List primaryKeys; + + protected int timeout; + + private JedisPool pool; + + private Jedis jedis; + + private JedisSentinelPool jedisSentinelPool; + + private GenericObjectPoolConfig poolConfig; + + private RedisOutputFormat(){ + } + @Override + public void configure(Configuration parameters) { + + } + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + establishConnection(); + } + + private void establishConnection() { + poolConfig = new GenericObjectPoolConfig(); + String[] nodes = url.split(","); + if (nodes.length > 1){ + //cluster + Set addresses = new HashSet<>(); + Set ipPorts = new HashSet<>(); + for (String ipPort : nodes) { + ipPorts.add(ipPort); + String[] ipPortPair = ipPort.split(":"); + addresses.add(new HostAndPort(ipPortPair[0].trim(), Integer.valueOf(ipPortPair[1].trim()))); + } + jedisSentinelPool = new JedisSentinelPool("Master", ipPorts, poolConfig, timeout, password, Integer.parseInt(database)); + jedis = jedisSentinelPool.getResource(); + } else { + String[] ipPortPair = nodes[0].split(":"); + String ip = ipPortPair[0]; + String port = ipPortPair[1]; + pool = new JedisPool(poolConfig, ip, Integer.parseInt(port), timeout, password, Integer.parseInt(database)); + jedis = pool.getResource(); + } + } + + @Override + public void writeRecord(Tuple2 record) throws IOException { + Tuple2 tupleTrans = record; + Boolean retract = tupleTrans.getField(0); + if (!retract) { + return; + } + Row row = tupleTrans.getField(1); + if (record.getArity() != fieldNames.length) { + return; + } + + HashMap map = new HashMap<>(); + + for (String primaryKey : primaryKeys){ + for (int i=0; i kvList = new LinkedList<>(); + for (String primaryKey : primaryKeys){ + StringBuilder primaryKV = new StringBuilder(); + int index = map.get(primaryKey).intValue(); + primaryKV.append(primaryKey).append(":").append((String) record.getField(index)); + kvList.add(primaryKV.toString()); + } + + String perKey = String.join(":", kvList); + + + for (int i = 0; i < fieldNames.length; i++) { + //key 表名:主键名:主键值:列名 + StringBuilder key = new StringBuilder(); + key.append(perKey).append(fieldNames[i]); + jedis.append(key.toString(), (String) record.getField(i)); + } + } + + @Override + public void close() throws IOException { + if (jedisSentinelPool != null) { + jedisSentinelPool.close(); + } + if (pool != null) { + pool.close(); + } + + } + + public static RedisOutputFormatBuilder buildRedisOutputFormat(){ + return new RedisOutputFormatBuilder(); + } + + public static class RedisOutputFormatBuilder{ + private final RedisOutputFormat redisOutputFormat; + + protected RedisOutputFormatBuilder(){ + this.redisOutputFormat = new RedisOutputFormat(); + } + + public RedisOutputFormatBuilder setUrl(String url){ + redisOutputFormat.url = url; + return this; + } + + public RedisOutputFormatBuilder setDatabase(String database){ + redisOutputFormat.database = database; + return this; + } + + public RedisOutputFormatBuilder setPassword(String password){ + redisOutputFormat.password = password; + return this; + } + + public RedisOutputFormatBuilder setFieldNames(String[] fieldNames){ + redisOutputFormat.fieldNames = fieldNames; + return this; + } + + public RedisOutputFormatBuilder setFieldTypes(TypeInformation[] fieldTypes){ + redisOutputFormat.fieldTypes = fieldTypes; + return this; + } + + public RedisOutputFormatBuilder setPrimaryKeys(List primaryKeys){ + redisOutputFormat.primaryKeys = primaryKeys; + return this; + } + + public RedisOutputFormatBuilder setTimeout(int timeout){ + redisOutputFormat.timeout = timeout; + return this; + } + + public RedisOutputFormat finish(){ + if (redisOutputFormat.url == null){ + throw new IllegalArgumentException("No URL supplied."); + } + + if (redisOutputFormat.database == null){ + throw new IllegalArgumentException("No database supplied."); + } + + if (redisOutputFormat.password == null){ + throw new IllegalArgumentException("No password supplied."); + } + + return redisOutputFormat; + } + } +} diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java new file mode 100644 index 000000000..ca9841e23 --- /dev/null +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java @@ -0,0 +1,108 @@ +/* + * 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 com.dtstack.flink.sql.sink.redis; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.redis.table.RedisTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.table.sinks.RetractStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +import java.util.List; + +public class RedisSink implements RetractStreamTableSink, IStreamSinkGener { + + protected String[] fieldNames; + + protected TypeInformation[] fieldTypes; + + protected String url; + + protected String database; + + protected String password; + + protected List primaryKeys; + + protected int timeout; + + public RedisSink(){ + + } + + @Override + public RedisSink genStreamSink(TargetTableInfo targetTableInfo) { + RedisTableInfo redisTableInfo = (RedisTableInfo) targetTableInfo; + this.url = redisTableInfo.getUrl(); + this.database = redisTableInfo.getDatabase(); + this.password = redisTableInfo.getPassword(); + this.primaryKeys = targetTableInfo.getPrimaryKeys(); + return this; + } + + @Override + public TypeInformation getRecordType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public void emitDataStream(DataStream> dataStream) { + RedisOutputFormat.RedisOutputFormatBuilder builder = RedisOutputFormat.buildRedisOutputFormat(); + builder.setUrl(this.url) + .setDatabase(this.database) + .setPassword(this.password) + .setFieldNames(this.fieldNames) + .setFieldTypes(this.fieldTypes) + .setPrimaryKeys(this.primaryKeys) + .setTimeout(this.timeout); + RedisOutputFormat redisOutputFormat = builder.finish(); + RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(redisOutputFormat); + dataStream.addSink(richSinkFunction); + } + + @Override + public TupleTypeInfo> getOutputType() { + return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), getRecordType()); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + @Override + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } +} diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java new file mode 100644 index 000000000..8890471dd --- /dev/null +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.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 com.dtstack.flink.sql.sink.redis.table; + +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +public class RedisSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + RedisTableInfo redisTableInfo = new RedisTableInfo(); + parseFieldsInfo(fieldsInfo, redisTableInfo); + redisTableInfo.setUrl(MathUtil.getString(props.get(RedisTableInfo.URL_KEY))); + redisTableInfo.setDatabase(MathUtil.getString(props.get(RedisTableInfo.DATABASE_KEY))); + redisTableInfo.setPassword(MathUtil.getString(props.get(RedisTableInfo.PASSWORD_KEY))); + if (props.get(RedisTableInfo.TIMEOUT) != null){ + redisTableInfo.setTimeout(Integer.parseInt(MathUtil.getString(props.get(RedisTableInfo.TIMEOUT)))); + } + return redisTableInfo; + } +} diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java new file mode 100644 index 000000000..1190610a1 --- /dev/null +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java @@ -0,0 +1,93 @@ +/* + * 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 com.dtstack.flink.sql.sink.redis.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +public class RedisTableInfo extends TargetTableInfo { + + private static final String CURR_TYPE = "redis"; + + public static final String URL_KEY = "url"; + + public static final String DATABASE_KEY = "database"; + + public static final String PASSWORD_KEY = "password"; + + public static final String TIMEOUT = "timeout"; + + public RedisTableInfo(){ + setType(CURR_TYPE); + } + + private String url; + + private String database; + + private String password; + + private int timeout = 1000; + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public String getDatabase() { + return database; + } + + public void setDatabase(String database) { + this.database = database; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } + + public int getTimeout() { + return timeout; + } + + public void setTimeout(int timeout) { + this.timeout = timeout; + } + + @Override + public boolean check() { + Preconditions.checkNotNull(url, "redis field of URL is required"); + Preconditions.checkNotNull(database, "redis field of database is required"); + Preconditions.checkNotNull(password, "redis field of password is required"); + return true; + } + + @Override + public String getType() { + return super.getType().toLowerCase(); + } + +} From b6eb612d50ba2774cb3dfde74416e8ba9c8fbedb Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Mon, 12 Nov 2018 12:44:57 +0800 Subject: [PATCH 115/470] modify README.md --- README.md | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 9a9ab8ad3..d53277157 100644 --- a/README.md +++ b/README.md @@ -6,14 +6,13 @@ # 已支持 * 源表:kafka 0.9,1.x版本 - * 维表:mysql,hbase - * 结果表:mysql,hbase,elasticsearch5.x + * 维表:mysql,hbase,mongo + * 结果表:mysql,hbase,elasticsearch5.x,mongo # 后续开发计划 * 增加全局缓存功能 * 增加临时表功能 * 增加redis维表,结果表功能 - * 增加mongodb维表,结果表功能 * 增加oracle维表,结果表功能 * 增加SQlServer维表,结果表功能 * 增加kafka结果表功能 @@ -130,10 +129,12 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * [elasticsearch 结果表插件](docs/elasticsearchSink.md) * [hbase 结果表插件](docs/hbaseSink.md) * [mysql 结果表插件](docs/mysqlSink.md) +* [mongo 结果表插件](docs/mongoSink.md) ### 2.3 维表插件 * [hbase 维表插件](docs/hbaseSide.md) * [mysql 维表插件](docs/mysqlSide.md) +* [mongo 维表插件](docs/mongoSide.md) ## 3 样例 From c1469324e23166ff422de292d13cb65b9d0fcf1c Mon Sep 17 00:00:00 2001 From: ambition119 <1269223860> Date: Mon, 12 Nov 2018 14:48:02 +0800 Subject: [PATCH 116/470] =?UTF-8?q?SQL=20CEP=20=E5=AE=9E=E7=8E=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/pom.xml | 25 +- .../flink/sql/parser/InsertSqlParser.java | 11 +- .../main/scala/com/dtstack/flink/App.scala | 11 + .../calcite/RelTimeIndicatorConverter.scala | 482 +++++ .../flink/table/codegen/CodeGenerator.scala | 1920 +++++++++++++++++ .../table/codegen/MatchCodeGenerator.scala | 574 +++++ .../flink/table/codegen/generated.scala | 100 + .../nodes/datastream/DataStreamMatch.scala | 309 +++ .../nodes/logical/FlinkLogicalMatch.scala | 115 + .../table/plan/rules/FlinkRuleSets.scala | 209 ++ .../datastream/DataStreamMatchRule.scala | 46 + .../table/runtime/cepmatch/ConvertToRow.scala | 14 + .../cepmatch/IterativeConditionRunner.scala | 40 + .../table/runtime/cepmatch/MatchUtil.scala | 99 + .../PatternFlatSelectFunctionRunner.scala | 47 + .../PatternSelectFunctionRunner.scala | 45 + .../table/validate/FunctionCatalog.scala | 534 +++++ .../flink/sql/side/SideSqlExecTest.java | 48 +- .../dtstack/flink/sql/sink/mysql/DBSink.java | 3 + 19 files changed, 4622 insertions(+), 10 deletions(-) create mode 100644 core/src/main/scala/com/dtstack/flink/App.scala create mode 100644 core/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala create mode 100644 core/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala create mode 100644 core/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala create mode 100644 core/src/main/scala/org/apache/flink/table/codegen/generated.scala create mode 100644 core/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala create mode 100644 core/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala create mode 100644 core/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala create mode 100644 core/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala create mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/ConvertToRow.scala create mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/IterativeConditionRunner.scala create mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/MatchUtil.scala create mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternFlatSelectFunctionRunner.scala create mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternSelectFunctionRunner.scala create mode 100644 core/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala diff --git a/core/pom.xml b/core/pom.xml index 1040fcea6..ba5970f91 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -28,9 +28,9 @@ - org.apache.flink - flink-table_2.11 - ${flink.version} + joda-time + joda-time + 2.5 @@ -45,6 +45,25 @@ ${flink.version} + + org.apache.flink + flink-table_2.11 + ${flink.version} + + + + org.apache.calcite + calcite-server + + 1.16.0 + + + + org.apache.flink + flink-cep-scala_2.11 + ${flink.version} + + org.apache.flink flink-scala_2.11 diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index e3d65f420..3e3a75880 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -20,12 +20,7 @@ package com.dtstack.flink.sql.parser; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlInsert; -import org.apache.calcite.sql.SqlJoin; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlSelect; +import org.apache.calcite.sql.*; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; import org.apache.commons.lang3.StringUtils; @@ -113,6 +108,10 @@ private static void parseNode(SqlNode sqlNode, SqlParseResult sqlParseResult){ sqlParseResult.addSourceTable(identifierNode.toString()); } break; + case MATCH_RECOGNIZE: + SqlMatchRecognize node = (SqlMatchRecognize) sqlNode; + sqlParseResult.addSourceTable(node.getTableRef().toString()); + break; default: //do nothing break; diff --git a/core/src/main/scala/com/dtstack/flink/App.scala b/core/src/main/scala/com/dtstack/flink/App.scala new file mode 100644 index 000000000..e74ccc28a --- /dev/null +++ b/core/src/main/scala/com/dtstack/flink/App.scala @@ -0,0 +1,11 @@ +package com.dtstack.flink + +/** + * Hello world! + * + */ +object App { + def main(args: Array[String]): Unit = { + println( "Hello World!" ) + } +} diff --git a/core/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala b/core/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala new file mode 100644 index 000000000..e120addb3 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala @@ -0,0 +1,482 @@ +package org.apache.flink.table.calcite + +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core._ +import org.apache.calcite.rel.logical._ +import org.apache.calcite.rel.{RelNode, RelShuttle} +import org.apache.calcite.rex._ +import org.apache.calcite.sql.fun.SqlStdOperatorTable +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo +import org.apache.flink.table.api.{TableException, ValidationException} +import org.apache.flink.table.calcite.FlinkTypeFactory.{isRowtimeIndicatorType, _} +import org.apache.flink.table.functions.sql.ProctimeSqlFunction +import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate +import org.apache.flink.table.plan.schema.TimeIndicatorRelDataType +import org.apache.flink.table.validate.BasicOperatorTable + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable + +/** + * Traverses a [[RelNode]] tree and converts fields with [[TimeIndicatorRelDataType]] type. If a + * time attribute is accessed for a calculation, it will be materialized. Forwarding is allowed in + * some cases, but not all. + */ +class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { + + private val timestamp = rexBuilder + .getTypeFactory + .asInstanceOf[FlinkTypeFactory] + .createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP, isNullable = false) + + override def visit(intersect: LogicalIntersect): RelNode = + throw new TableException("Logical intersect in a stream environment is not supported yet.") + + override def visit(union: LogicalUnion): RelNode = { + // visit children and update inputs + val inputs = union.getInputs.map(_.accept(this)) + + // make sure that time indicator types match + val inputTypes = inputs.map(_.getRowType) + + val head = inputTypes.head.getFieldList.map(_.getType) + + val isValid = inputTypes.forall { t => + val fieldTypes = t.getFieldList.map(_.getType) + + fieldTypes.zip(head).forall { case (l, r) => + // check if time indicators match + if (isTimeIndicatorType(l) && isTimeIndicatorType(r)) { + val leftTime = l.asInstanceOf[TimeIndicatorRelDataType].isEventTime + val rightTime = r.asInstanceOf[TimeIndicatorRelDataType].isEventTime + leftTime == rightTime + } + // one side is not an indicator + else if (isTimeIndicatorType(l) || isTimeIndicatorType(r)) { + false + } + // uninteresting types + else { + true + } + } + } + + if (!isValid) { + throw new ValidationException( + "Union fields with time attributes have different types.") + } + + LogicalUnion.create(inputs, union.all) + } + + override def visit(aggregate: LogicalAggregate): RelNode = convertAggregate(aggregate) + + override def visit(minus: LogicalMinus): RelNode = + throw new TableException("Logical minus in a stream environment is not supported yet.") + + override def visit(sort: LogicalSort): RelNode = { + + val input = sort.getInput.accept(this) + LogicalSort.create(input, sort.collation, sort.offset, sort.fetch) + } + + // override def visit(`match`: LogicalMatch): RelNode = + // throw new TableException("Logical match in a stream environment is not supported yet.") + // and cpe entry + override def visit(`match`: LogicalMatch): RelNode = { + val rowType = `match`.getInput.getRowType + + val materializer = new RexTimeIndicatorMaterializer( + rexBuilder, + rowType.getFieldList.map(_.getType)) + + val patternDefinitions = + `match`.getPatternDefinitions.foldLeft(mutable.Map[String, RexNode]()) { + case (m, (k, v)) => m += k -> v.accept(materializer) + } + + val measures = `match`.getMeasures.foldLeft(mutable.Map[String, RexNode]()) { + case (m, (k, v)) => m += k -> v.accept(materializer) + } + + val outputTypeBuilder = rexBuilder + .getTypeFactory + .asInstanceOf[FlinkTypeFactory] + .builder() + + `match`.getRowType.getFieldList.asScala + .foreach(x => measures.get(x.getName) match { + case Some(measure) => outputTypeBuilder.add(x.getName, measure.getType) + case None => outputTypeBuilder.add(x) + }) + + LogicalMatch.create( + `match`.getInput, + outputTypeBuilder.build(), + `match`.getPattern, + `match`.isStrictStart, + `match`.isStrictEnd, + patternDefinitions, + measures, + `match`.getAfter, + `match`.getSubsets.asInstanceOf[java.util.Map[String, java.util.TreeSet[String]]], + `match`.isAllRows, + `match`.getPartitionKeys, + `match`.getOrderKeys, + `match`.getInterval) + } + + + override def visit(other: RelNode): RelNode = other match { + + case uncollect: Uncollect => + // visit children and update inputs + val input = uncollect.getInput.accept(this) + Uncollect.create(uncollect.getTraitSet, input, uncollect.withOrdinality) + + case scan: LogicalTableFunctionScan => + scan + + case aggregate: LogicalWindowAggregate => + val convAggregate = convertAggregate(aggregate) + + LogicalWindowAggregate.create( + aggregate.getWindow, + aggregate.getNamedProperties, + convAggregate) + + case _ => + throw new TableException(s"Unsupported logical operator: ${other.getClass.getSimpleName}") + } + + + override def visit(exchange: LogicalExchange): RelNode = + throw new TableException("Logical exchange in a stream environment is not supported yet.") + + override def visit(scan: TableScan): RelNode = scan + + override def visit(scan: TableFunctionScan): RelNode = + throw new TableException("Table function scan in a stream environment is not supported yet.") + + override def visit(values: LogicalValues): RelNode = values + + override def visit(filter: LogicalFilter): RelNode = { + // visit children and update inputs + val input = filter.getInput.accept(this) + + // We do not materialize time indicators in conditions because they can be locally evaluated. + // Some conditions are evaluated by special operators (e.g., time window joins). + // Time indicators in remaining conditions are materialized by Calc before the code generation. + LogicalFilter.create(input, filter.getCondition) + } + + override def visit(project: LogicalProject): RelNode = { + // visit children and update inputs + val input = project.getInput.accept(this) + + // check if input field contains time indicator type + // materialize field if no time indicator is present anymore + // if input field is already materialized, change to timestamp type + val materializer = new RexTimeIndicatorMaterializer( + rexBuilder, + input.getRowType.getFieldList.map(_.getType)) + + val projects = project.getProjects.map(_.accept(materializer)) + val fieldNames = project.getRowType.getFieldNames + LogicalProject.create(input, projects, fieldNames) + } + + override def visit(join: LogicalJoin): RelNode = { + val left = join.getLeft.accept(this) + val right = join.getRight.accept(this) + + LogicalJoin.create(left, right, join.getCondition, join.getVariablesSet, join.getJoinType) + + } + + + override def visit(correlate: LogicalCorrelate): RelNode = { + // visit children and update inputs + val inputs = correlate.getInputs.map(_.accept(this)) + + val right = inputs(1) match { + case scan: LogicalTableFunctionScan => + // visit children and update inputs + val scanInputs = scan.getInputs.map(_.accept(this)) + + // check if input field contains time indicator type + // materialize field if no time indicator is present anymore + // if input field is already materialized, change to timestamp type + val materializer = new RexTimeIndicatorMaterializer( + rexBuilder, + inputs.head.getRowType.getFieldList.map(_.getType)) + + val call = scan.getCall.accept(materializer) + LogicalTableFunctionScan.create( + scan.getCluster, + scanInputs, + call, + scan.getElementType, + scan.getRowType, + scan.getColumnMappings) + + case _ => + inputs(1) + } + + LogicalCorrelate.create( + inputs.head, + right, + correlate.getCorrelationId, + correlate.getRequiredColumns, + correlate.getJoinType) + } + + + + + private def convertAggregate(aggregate: Aggregate): LogicalAggregate = { + // visit children and update inputs + val input = aggregate.getInput.accept(this) + + // add a project to materialize aggregation arguments/grouping keys + + val refIndices = mutable.Set[Int]() + + // check arguments of agg calls + aggregate.getAggCallList.foreach(call => if (call.getArgList.size() == 0) { + // count(*) has an empty argument list + (0 until input.getRowType.getFieldCount).foreach(refIndices.add) + } else { + // for other aggregations + call.getArgList.map(_.asInstanceOf[Int]).foreach(refIndices.add) + }) + + // check grouping sets + aggregate.getGroupSets.foreach(set => + set.asList().map(_.asInstanceOf[Int]).foreach(refIndices.add) + ) + + val needsMaterialization = refIndices.exists(idx => + isTimeIndicatorType(input.getRowType.getFieldList.get(idx).getType)) + + // create project if necessary + val projectedInput = if (needsMaterialization) { + + // insert or merge with input project if + // a time attribute is accessed and needs to be materialized + input match { + + // merge + case lp: LogicalProject => + val projects = lp.getProjects.zipWithIndex.map { case (expr, idx) => + if (isTimeIndicatorType(expr.getType) && refIndices.contains(idx)) { + if (isRowtimeIndicatorType(expr.getType)) { + // cast rowtime indicator to regular timestamp + rexBuilder.makeAbstractCast(timestamp, expr) + } else { + // generate proctime access + rexBuilder.makeCall(ProctimeSqlFunction, expr) + } + } else { + expr + } + } + + LogicalProject.create( + lp.getInput, + projects, + input.getRowType.getFieldNames) + + // new project + case _ => + val projects = input.getRowType.getFieldList.map { field => + if (isTimeIndicatorType(field.getType) && refIndices.contains(field.getIndex)) { + if (isRowtimeIndicatorType(field.getType)) { + // cast rowtime indicator to regular timestamp + rexBuilder.makeAbstractCast( + timestamp, + new RexInputRef(field.getIndex, field.getType)) + } else { + // generate proctime access + rexBuilder.makeCall( + ProctimeSqlFunction, + new RexInputRef(field.getIndex, field.getType)) + } + } else { + new RexInputRef(field.getIndex, field.getType) + } + } + + LogicalProject.create( + input, + projects, + input.getRowType.getFieldNames) + } + } else { + // no project necessary + input + } + + // remove time indicator type as agg call return type + val updatedAggCalls = aggregate.getAggCallList.map { call => + val callType = if (isTimeIndicatorType(call.getType)) { + timestamp + } else { + call.getType + } + AggregateCall.create( + call.getAggregation, + call.isDistinct, + call.getArgList, + call.filterArg, + callType, + call.name) + } + + LogicalAggregate.create( + projectedInput, + aggregate.indicator, + aggregate.getGroupSet, + aggregate.getGroupSets, + updatedAggCalls) + } + +} + +object RelTimeIndicatorConverter { + + def convert(rootRel: RelNode, rexBuilder: RexBuilder): RelNode = { + val converter = new RelTimeIndicatorConverter(rexBuilder) + val convertedRoot = rootRel.accept(converter) + + var needsConversion = false + + // materialize remaining proctime indicators + val projects = convertedRoot.getRowType.getFieldList.map(field => + if (isProctimeIndicatorType(field.getType)) { + needsConversion = true + rexBuilder.makeCall( + ProctimeSqlFunction, + new RexInputRef(field.getIndex, field.getType)) + } else { + new RexInputRef(field.getIndex, field.getType) + } + ) + + // add final conversion if necessary + if (needsConversion) { + LogicalProject.create( + convertedRoot, + projects, + convertedRoot.getRowType.getFieldNames) + } else { + convertedRoot + } + } + + /** + * Materializes time indicator accesses in an expression. + * + * @param expr The expression in which time indicators are materialized. + * @param rowType The input schema of the expression. + * @param rexBuilder A RexBuilder. + * + * @return The expression with materialized time indicators. + */ + def convertExpression(expr: RexNode, rowType: RelDataType, rexBuilder: RexBuilder): RexNode = { + val materializer = new RexTimeIndicatorMaterializer( + rexBuilder, + rowType.getFieldList.map(_.getType)) + + expr.accept(materializer) + } +} + +class RexTimeIndicatorMaterializer( + private val rexBuilder: RexBuilder, + private val input: Seq[RelDataType]) + extends RexShuttle { + + private val timestamp = rexBuilder + .getTypeFactory + .asInstanceOf[FlinkTypeFactory] + .createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP, isNullable = false) + + override def visitInputRef(inputRef: RexInputRef): RexNode = { + // reference is interesting + if (isTimeIndicatorType(inputRef.getType)) { + val resolvedRefType = input(inputRef.getIndex) + // input is a valid time indicator + if (isTimeIndicatorType(resolvedRefType)) { + inputRef + } + // input has been materialized + else { + new RexInputRef(inputRef.getIndex, resolvedRefType) + } + } + // reference is a regular field + else { + super.visitInputRef(inputRef) + } + } + + override def visitCall(call: RexCall): RexNode = { + val updatedCall = super.visitCall(call).asInstanceOf[RexCall] + + // materialize operands with time indicators + val materializedOperands = updatedCall.getOperator match { + + // skip materialization for special operators + case BasicOperatorTable.SESSION | BasicOperatorTable.HOP | BasicOperatorTable.TUMBLE => + updatedCall.getOperands.toList + + case _ => + updatedCall.getOperands.map { o => + if (isTimeIndicatorType(o.getType)) { + if (isRowtimeIndicatorType(o.getType)) { + // cast rowtime indicator to regular timestamp + rexBuilder.makeAbstractCast(timestamp, o) + } else { + // generate proctime access + rexBuilder.makeCall(ProctimeSqlFunction, o) + } + } else { + o + } + } + } + + // remove time indicator return type + updatedCall.getOperator match { + + // we do not modify AS if operand has not been materialized + case SqlStdOperatorTable.AS if + isTimeIndicatorType(updatedCall.getOperands.get(0).getType) => + updatedCall + + // do not modify window time attributes + case BasicOperatorTable.TUMBLE_ROWTIME | + BasicOperatorTable.TUMBLE_PROCTIME | + BasicOperatorTable.HOP_ROWTIME | + BasicOperatorTable.HOP_PROCTIME | + BasicOperatorTable.SESSION_ROWTIME | + BasicOperatorTable.SESSION_PROCTIME + // since we materialize groupings on time indicators, + // we cannot check the operands anymore but the return type at least + if isTimeIndicatorType(updatedCall.getType) => + updatedCall + + // materialize function's result and operands + case _ if isTimeIndicatorType(updatedCall.getType) => + updatedCall.clone(timestamp, materializedOperands) + + // materialize function's operands only + case _ => + updatedCall.clone(updatedCall.getType, materializedOperands) + } + } +} diff --git a/core/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala b/core/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala new file mode 100644 index 000000000..439e3a7f2 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala @@ -0,0 +1,1920 @@ +package org.apache.flink.table.codegen + +import java.math.{BigDecimal => JBigDecimal} + +import org.apache.calcite.avatica.util.DateTimeUtils +import org.apache.calcite.rex._ +import org.apache.calcite.sql.SqlOperator +import org.apache.calcite.sql.`type`.SqlTypeName._ +import org.apache.calcite.sql.`type`.{ReturnTypes, SqlTypeName} +import org.apache.calcite.sql.fun.SqlStdOperatorTable.{ROW, _} +import org.apache.commons.lang3.StringEscapeUtils +import org.apache.flink.api.common.functions._ +import org.apache.flink.api.common.typeinfo._ +import org.apache.flink.api.common.typeutils.CompositeType +import org.apache.flink.api.java.typeutils._ +import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.table.api.{TableConfig, TableException} +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.codegen.CodeGenUtils._ +import org.apache.flink.table.codegen.GeneratedExpression.{ALWAYS_NULL, NEVER_NULL, NO_CODE} +import org.apache.flink.table.codegen.calls.ScalarOperators._ +import org.apache.flink.table.codegen.calls.{CurrentTimePointCallGen, FunctionGenerator} +import org.apache.flink.table.functions.sql.{ProctimeSqlFunction, ScalarSqlFunctions, StreamRecordTimestampSqlFunction} +import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils +import org.apache.flink.table.functions.{FunctionContext, UserDefinedFunction} +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo +import org.apache.flink.table.typeutils.TypeCheckUtils._ +import org.joda.time.format.DateTimeFormatter + +import scala.collection.JavaConversions._ +import scala.collection.mutable + +/** + * [[CodeGenerator]] is the base code generator for generating Flink + * [[org.apache.flink.api.common.functions.Function]]s. + * It is responsible for expression generation and tracks the context (member variables etc). + * + * @param config configuration that determines runtime behavior + * @param nullableInput input(s) can be null. + * @param input1 type information about the first input of the Function + * @param input2 type information about the second input if the Function is binary + * @param input1FieldMapping additional mapping information for input1. + * POJO types have no deterministic field order and some input fields might not be read. + * The input1FieldMapping is also used to inject time indicator attributes. + * @param input2FieldMapping additional mapping information for input2. + * POJO types have no deterministic field order and some input fields might not be read. + */ +abstract class CodeGenerator( + config: TableConfig, + nullableInput: Boolean, + input1: TypeInformation[_ <: Any], + input2: Option[TypeInformation[_ <: Any]] = None, + input1FieldMapping: Option[Array[Int]] = None, + input2FieldMapping: Option[Array[Int]] = None) + extends RexVisitor[GeneratedExpression] { + + // check if nullCheck is enabled when inputs can be null + if (nullableInput && !config.getNullCheck) { + throw new CodeGenException("Null check must be enabled if entire rows can be null.") + } + + // check for POJO input1 mapping + input1 match { + case pt: PojoTypeInfo[_] => + input1FieldMapping.getOrElse( + throw new CodeGenException("No input mapping is specified for input1 of type POJO.")) + case _ => // ok + } + + // check for POJO input2 mapping + input2 match { + case Some(pt: PojoTypeInfo[_]) => + input2FieldMapping.getOrElse( + throw new CodeGenException("No input mapping is specified for input2 of type POJO.")) + case _ => // ok + } + + protected val input1Mapping: Array[Int] = input1FieldMapping match { + case Some(mapping) => mapping + case _ => (0 until input1.getArity).toArray + } + + protected val input2Mapping: Array[Int] = input2FieldMapping match { + case Some(mapping) => mapping + case _ => input2 match { + case Some(input) => (0 until input.getArity).toArray + case _ => Array[Int]() + } + } + + // set of member statements that will be added only once + // we use a LinkedHashSet to keep the insertion order + protected val reusableMemberStatements: mutable.LinkedHashSet[String] = + mutable.LinkedHashSet[String]() + + // set of constructor statements that will be added only once + // we use a LinkedHashSet to keep the insertion order + protected val reusableInitStatements: mutable.LinkedHashSet[String] = + mutable.LinkedHashSet[String]() + + // set of open statements for RichFunction that will be added only once + // we use a LinkedHashSet to keep the insertion order + protected val reusableOpenStatements: mutable.LinkedHashSet[String] = + mutable.LinkedHashSet[String]() + + // set of close statements for RichFunction that will be added only once + // we use a LinkedHashSet to keep the insertion order + protected val reusableCloseStatements: mutable.LinkedHashSet[String] = + mutable.LinkedHashSet[String]() + + // set of statements that will be added only once per record; + // code should only update member variables because local variables are not accessible if + // the code needs to be split; + // we use a LinkedHashSet to keep the insertion order + protected val reusablePerRecordStatements: mutable.LinkedHashSet[String] = + mutable.LinkedHashSet[String]() + + // map of initial input unboxing expressions that will be added only once + // (inputTerm, index) -> expr + protected val reusableInputUnboxingExprs: mutable.Map[(String, Int), GeneratedExpression] = + mutable.Map[(String, Int), GeneratedExpression]() + + // set of constructor statements that will be added only once + // we use a LinkedHashSet to keep the insertion order + protected val reusableConstructorStatements: mutable.LinkedHashSet[(String, String)] = + mutable.LinkedHashSet[(String, String)]() + + /** + * Flag that indicates that the generated code needed to be split into several methods. + */ + protected var hasCodeSplits: Boolean = false + + /** + * @return code block of statements that need to be placed in the member area of the Function + * (e.g. member variables and their initialization) + */ + def reuseMemberCode(): String = { + reusableMemberStatements.mkString("", "\n", "\n") + } + + /** + * @return code block of statements that need to be placed in the constructor of the Function + */ + def reuseInitCode(): String = { + reusableInitStatements.mkString("", "\n", "\n") + } + + /** + * @return code block of statements that need to be placed in the open() method of RichFunction + */ + def reuseOpenCode(): String = { + reusableOpenStatements.mkString("", "\n", "\n") + } + + /** + * @return code block of statements that need to be placed in the close() method of RichFunction + */ + def reuseCloseCode(): String = { + reusableCloseStatements.mkString("", "\n", "\n") + } + + /** + * @return code block of statements that need to be placed in the SAM of the Function + */ + def reusePerRecordCode(): String = { + reusablePerRecordStatements.mkString("", "\n", "\n") + } + + /** + * @return code block of statements that unbox input variables to a primitive variable + * and a corresponding null flag variable + */ + def reuseInputUnboxingCode(): String = { + reusableInputUnboxingExprs.values.map(_.code).mkString("", "\n", "\n") + } + + /** + * @return code block of constructor statements for the Function + */ + def reuseConstructorCode(className: String): String = { + reusableConstructorStatements.map { case (params, body) => + s""" + |public $className($params) throws Exception { + | this(); + | $body + |} + |""".stripMargin + }.mkString("", "\n", "\n") + } + + /** + * @return term of the (casted and possibly boxed) first input + */ + var input1Term = "in1" + + /** + * @return term of the (casted and possibly boxed) second input + */ + var input2Term = "in2" + + /** + * @return term of the (casted) output collector + */ + var collectorTerm = "c" + + /** + * @return term of the output record (possibly defined in the member area e.g. Row, Tuple) + */ + var outRecordTerm = "out" + + /** + * @return term of the [[ProcessFunction]]'s context + */ + var contextTerm = "ctx" + + /** + * @return returns if null checking is enabled + */ + def nullCheck: Boolean = config.getNullCheck + + /** + * Generates an expression from a RexNode. If objects or variables can be reused, they will be + * added to reusable code sections internally. + * + * @param rex Calcite row expression + * @return instance of GeneratedExpression + */ + def generateExpression(rex: RexNode): GeneratedExpression = { + rex.accept(this) + } + + /** + * Generates an expression that converts the first input (and second input) into the given type. + * If two inputs are converted, the second input is appended. If objects or variables can + * be reused, they will be added to reusable code sections internally. The evaluation result + * may be stored in the global result variable (see [[outRecordTerm]]). + * + * @param returnType conversion target type. Inputs and output must have the same arity. + * @param resultFieldNames result field names necessary for a mapping to POJO fields. + * @param rowtimeExpression an expression to extract the value of a rowtime field from + * the input data. Required if the field indices include a rowtime + * marker. + * @return instance of GeneratedExpression + */ + def generateConverterResultExpression( + returnType: TypeInformation[_ <: Any], + resultFieldNames: Seq[String], + rowtimeExpression: Option[RexNode] = None) + : GeneratedExpression = { + + val input1AccessExprs = input1Mapping.map { + case TimeIndicatorTypeInfo.ROWTIME_STREAM_MARKER | + TimeIndicatorTypeInfo.ROWTIME_BATCH_MARKER if rowtimeExpression.isDefined => + // generate rowtime attribute from expression + generateExpression(rowtimeExpression.get) + case TimeIndicatorTypeInfo.ROWTIME_STREAM_MARKER | + TimeIndicatorTypeInfo.ROWTIME_BATCH_MARKER => + throw TableException("Rowtime extraction expression missing. Please report a bug.") + case TimeIndicatorTypeInfo.PROCTIME_STREAM_MARKER => + // attribute is proctime indicator. + // we use a null literal and generate a timestamp when we need it. + generateNullLiteral(TimeIndicatorTypeInfo.PROCTIME_INDICATOR) + case TimeIndicatorTypeInfo.PROCTIME_BATCH_MARKER => + // attribute is proctime field in a batch query. + // it is initialized with the current time. + generateCurrentTimestamp() + case idx => + generateInputAccess(input1, input1Term, idx) + } + + val input2AccessExprs = input2 match { + case Some(ti) => + input2Mapping.map(idx => generateInputAccess(ti, input2Term, idx)).toSeq + case None => Seq() // add nothing + } + + generateResultExpression(input1AccessExprs ++ input2AccessExprs, returnType, resultFieldNames) + } + + /** + * Generates an expression from the left input and the right table function. + */ + def generateCorrelateAccessExprs: (Seq[GeneratedExpression], Seq[GeneratedExpression]) = { + val input1AccessExprs = input1Mapping.map { idx => + generateInputAccess(input1, input1Term, idx) + } + + val input2AccessExprs = input2 match { + case Some(ti) => + // use generateFieldAccess instead of generateInputAccess to avoid the generated table + // function's field access code is put on the top of function body rather than + // the while loop + input2Mapping.map { idx => + generateFieldAccess(ti, input2Term, idx) + }.toSeq + case None => throw new CodeGenException("Type information of input2 must not be null.") + } + (input1AccessExprs, input2AccessExprs) + } + + /** + * Generates an expression from a sequence of RexNode. If objects or variables can be reused, + * they will be added to reusable code sections internally. The evaluation result + * may be stored in the global result variable (see [[outRecordTerm]]). + * + * @param returnType conversion target type. Type must have the same arity than rexNodes. + * @param resultFieldNames result field names necessary for a mapping to POJO fields. + * @param rexNodes sequence of RexNode to be converted + * @return instance of GeneratedExpression + */ + def generateResultExpression( + returnType: TypeInformation[_ <: Any], + resultFieldNames: Seq[String], + rexNodes: Seq[RexNode]) + : GeneratedExpression = { + val fieldExprs = rexNodes.map(generateExpression) + generateResultExpression(fieldExprs, returnType, resultFieldNames) + } + + /** + * Generates an expression from a sequence of other expressions. If objects or variables can + * be reused, they will be added to reusable code sections internally. The evaluation result + * may be stored in the global result variable (see [[outRecordTerm]]). + * + * @param fieldExprs field expressions to be converted + * @param returnType conversion target type. Type must have the same arity than fieldExprs. + * @param resultFieldNames result field names necessary for a mapping to POJO fields. + * @return instance of GeneratedExpression + */ + def generateResultExpression( + fieldExprs: Seq[GeneratedExpression], + returnType: TypeInformation[_ <: Any], + resultFieldNames: Seq[String]) + : GeneratedExpression = { + // initial type check + if (returnType.getArity != fieldExprs.length) { + throw new CodeGenException( + s"Arity [${returnType.getArity}] of result type [$returnType] does not match " + + s"number [${fieldExprs.length}] of expressions [$fieldExprs].") + } + if (resultFieldNames.length != fieldExprs.length) { + throw new CodeGenException( + s"Arity [${resultFieldNames.length}] of result field names [$resultFieldNames] does not " + + s"match number [${fieldExprs.length}] of expressions [$fieldExprs].") + } + // type check + returnType match { + case pt: PojoTypeInfo[_] => + fieldExprs.zipWithIndex foreach { + case (fieldExpr, i) if fieldExpr.resultType != pt.getTypeAt(resultFieldNames(i)) => + throw new CodeGenException( + s"Incompatible types of expression and result type. Expression [$fieldExpr] type is" + + s" [${fieldExpr.resultType}], result type is [${pt.getTypeAt(resultFieldNames(i))}]") + + case _ => // ok + } + + case ct: CompositeType[_] => + fieldExprs.zipWithIndex foreach { + case (fieldExpr, i) if fieldExpr.resultType != ct.getTypeAt(i) => + throw new CodeGenException( + s"Incompatible types of expression and result type. Expression[$fieldExpr] type is " + + s"[${fieldExpr.resultType}], result type is [${ct.getTypeAt(i)}]") + case _ => // ok + } + + case t: TypeInformation[_] if t != fieldExprs.head.resultType => + throw new CodeGenException( + s"Incompatible types of expression and result type. Expression [${fieldExprs.head}] " + + s"type is [${fieldExprs.head.resultType}], result type is [$t]") + + case _ => // ok + } + + val returnTypeTerm = boxedTypeTermForTypeInfo(returnType) + val boxedFieldExprs = fieldExprs.map(generateOutputFieldBoxing) + + // generate result expression + returnType match { + case ri: RowTypeInfo => + addReusableOutRecord(ri) + val resultSetters = boxedFieldExprs.zipWithIndex map { + case (fieldExpr, i) => + if (nullCheck) { + s""" + |${fieldExpr.code} + |if (${fieldExpr.nullTerm}) { + | $outRecordTerm.setField($i, null); + |} + |else { + | $outRecordTerm.setField($i, ${fieldExpr.resultTerm}); + |} + |""".stripMargin + } + else { + s""" + |${fieldExpr.code} + |$outRecordTerm.setField($i, ${fieldExpr.resultTerm}); + |""".stripMargin + } + } + + val code = generateCodeSplits(resultSetters) + + GeneratedExpression(outRecordTerm, NEVER_NULL, code, returnType) + + case pt: PojoTypeInfo[_] => + addReusableOutRecord(pt) + val resultSetters = boxedFieldExprs.zip(resultFieldNames) map { + case (fieldExpr, fieldName) => + val accessor = getFieldAccessor(pt.getTypeClass, fieldName) + + accessor match { + // Reflective access of primitives/Objects + case ObjectPrivateFieldAccessor(field) => + val fieldTerm = addReusablePrivateFieldAccess(pt.getTypeClass, fieldName) + + val defaultIfNull = if (isFieldPrimitive(field)) { + primitiveDefaultValue(fieldExpr.resultType) + } else { + "null" + } + + if (nullCheck) { + s""" + |${fieldExpr.code} + |if (${fieldExpr.nullTerm}) { + | ${reflectiveFieldWriteAccess( + fieldTerm, + field, + outRecordTerm, + defaultIfNull)}; + |} + |else { + | ${reflectiveFieldWriteAccess( + fieldTerm, + field, + outRecordTerm, + fieldExpr.resultTerm)}; + |} + |""".stripMargin + } + else { + s""" + |${fieldExpr.code} + |${reflectiveFieldWriteAccess( + fieldTerm, + field, + outRecordTerm, + fieldExpr.resultTerm)}; + |""".stripMargin + } + + // primitive or Object field access (implicit boxing) + case _ => + if (nullCheck) { + s""" + |${fieldExpr.code} + |if (${fieldExpr.nullTerm}) { + | $outRecordTerm.$fieldName = null; + |} + |else { + | $outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; + |} + |""".stripMargin + } + else { + s""" + |${fieldExpr.code} + |$outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; + |""".stripMargin + } + } + } + + val code = generateCodeSplits(resultSetters) + + GeneratedExpression(outRecordTerm, NEVER_NULL, code, returnType) + + case tup: TupleTypeInfo[_] => + addReusableOutRecord(tup) + val resultSetters = boxedFieldExprs.zipWithIndex map { + case (fieldExpr, i) => + val fieldName = "f" + i + if (nullCheck) { + s""" + |${fieldExpr.code} + |if (${fieldExpr.nullTerm}) { + | throw new NullPointerException("Null result cannot be stored in a Tuple."); + |} + |else { + | $outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; + |} + |""".stripMargin + } + else { + s""" + |${fieldExpr.code} + |$outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; + |""".stripMargin + } + } + + val code = generateCodeSplits(resultSetters) + + GeneratedExpression(outRecordTerm, NEVER_NULL, code, returnType) + + case _: CaseClassTypeInfo[_] => + val fieldCodes: String = boxedFieldExprs.map(_.code).mkString("\n") + val constructorParams: String = boxedFieldExprs.map(_.resultTerm).mkString(", ") + val resultTerm = newName(outRecordTerm) + + val nullCheckCode = if (nullCheck) { + boxedFieldExprs map { (fieldExpr) => + s""" + |if (${fieldExpr.nullTerm}) { + | throw new NullPointerException("Null result cannot be stored in a Case Class."); + |} + |""".stripMargin + } mkString "\n" + } else { + "" + } + + val resultCode = + s""" + |$fieldCodes + |$nullCheckCode + |$returnTypeTerm $resultTerm = new $returnTypeTerm($constructorParams); + |""".stripMargin + + // case classes are not splittable + GeneratedExpression(resultTerm, NEVER_NULL, resultCode, returnType) + + case _: TypeInformation[_] => + val fieldExpr = boxedFieldExprs.head + val nullCheckCode = if (nullCheck) { + s""" + |if (${fieldExpr.nullTerm}) { + | throw new NullPointerException("Null result cannot be used for atomic types."); + |} + |""".stripMargin + } else { + "" + } + val resultCode = + s""" + |${fieldExpr.code} + |$nullCheckCode + |""".stripMargin + + // other types are not splittable + GeneratedExpression(fieldExpr.resultTerm, fieldExpr.nullTerm, resultCode, returnType) + + case _ => + throw new CodeGenException(s"Unsupported result type: $returnType") + } + } + + // ---------------------------------------------------------------------------------------------- + // RexVisitor methods + // ---------------------------------------------------------------------------------------------- + + override def visitInputRef(inputRef: RexInputRef): GeneratedExpression = { + // if inputRef index is within size of input1 we work with input1, input2 otherwise + val input = if (inputRef.getIndex < input1.getArity) { + (input1, input1Term) + } else { + (input2.getOrElse(throw new CodeGenException("Invalid input access.")), input2Term) + } + + val index = if (input._2 == input1Term) { + inputRef.getIndex + } else { + inputRef.getIndex - input1.getArity + } + + generateInputAccess(input._1, input._2, index) + } + + override def visitTableInputRef(rexTableInputRef: RexTableInputRef): GeneratedExpression = + visitInputRef(rexTableInputRef) + + override def visitFieldAccess(rexFieldAccess: RexFieldAccess): GeneratedExpression = { + val refExpr = rexFieldAccess.getReferenceExpr.accept(this) + val index = rexFieldAccess.getField.getIndex + generateFieldAccess(refExpr, index) + } + + override def visitLiteral(literal: RexLiteral): GeneratedExpression = { + val resultType = FlinkTypeFactory.toTypeInfo(literal.getType) + val value = literal.getValue3 + // null value with type + if (value == null) { + return generateNullLiteral(resultType) + } + // non-null values + literal.getType.getSqlTypeName match { + + case BOOLEAN => + generateNonNullLiteral(resultType, literal.getValue3.toString) + + case TINYINT => + val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) + if (decimal.isValidByte) { + generateNonNullLiteral(resultType, decimal.byteValue().toString) + } + else { + throw new CodeGenException("Decimal can not be converted to byte.") + } + + case SMALLINT => + val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) + if (decimal.isValidShort) { + generateNonNullLiteral(resultType, decimal.shortValue().toString) + } + else { + throw new CodeGenException("Decimal can not be converted to short.") + } + + case INTEGER => + val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) + if (decimal.isValidInt) { + generateNonNullLiteral(resultType, decimal.intValue().toString) + } + else { + throw new CodeGenException("Decimal can not be converted to integer.") + } + + case BIGINT => + val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) + if (decimal.isValidLong) { + generateNonNullLiteral(resultType, decimal.longValue().toString + "L") + } + else { + throw new CodeGenException("Decimal can not be converted to long.") + } + + case FLOAT => + val floatValue = value.asInstanceOf[JBigDecimal].floatValue() + floatValue match { + case Float.NaN => generateNonNullLiteral(resultType, "java.lang.Float.NaN") + case Float.NegativeInfinity => + generateNonNullLiteral(resultType, "java.lang.Float.NEGATIVE_INFINITY") + case Float.PositiveInfinity => + generateNonNullLiteral(resultType, "java.lang.Float.POSITIVE_INFINITY") + case _ => generateNonNullLiteral(resultType, floatValue.toString + "f") + } + + case DOUBLE => + val doubleValue = value.asInstanceOf[JBigDecimal].doubleValue() + doubleValue match { + case Double.NaN => generateNonNullLiteral(resultType, "java.lang.Double.NaN") + case Double.NegativeInfinity => + generateNonNullLiteral(resultType, "java.lang.Double.NEGATIVE_INFINITY") + case Double.PositiveInfinity => + generateNonNullLiteral(resultType, "java.lang.Double.POSITIVE_INFINITY") + case _ => generateNonNullLiteral(resultType, doubleValue.toString + "d") + } + case DECIMAL => + val decimalField = addReusableDecimal(value.asInstanceOf[JBigDecimal]) + generateNonNullLiteral(resultType, decimalField) + + case VARCHAR | CHAR => + val escapedValue = StringEscapeUtils.escapeJava( + StringEscapeUtils.unescapeJava(value.toString) + ) + generateNonNullLiteral(resultType, "\"" + escapedValue + "\"") + + case SYMBOL => + generateSymbol(value.asInstanceOf[Enum[_]]) + + case DATE => + generateNonNullLiteral(resultType, value.toString) + + case TIME => + generateNonNullLiteral(resultType, value.toString) + + case TIMESTAMP => + generateNonNullLiteral(resultType, value.toString + "L") + + case typeName if YEAR_INTERVAL_TYPES.contains(typeName) => + val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) + if (decimal.isValidInt) { + generateNonNullLiteral(resultType, decimal.intValue().toString) + } else { + throw new CodeGenException( + s"Decimal '$decimal' can not be converted to interval of months.") + } + + case typeName if DAY_INTERVAL_TYPES.contains(typeName) => + val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) + if (decimal.isValidLong) { + generateNonNullLiteral(resultType, decimal.longValue().toString + "L") + } else { + throw new CodeGenException( + s"Decimal '$decimal' can not be converted to interval of milliseconds.") + } + + case t@_ => + throw new CodeGenException(s"Type not supported: $t") + } + } + + override def visitCorrelVariable(correlVariable: RexCorrelVariable): GeneratedExpression = { + GeneratedExpression(input1Term, NEVER_NULL, NO_CODE, input1) + } + + override def visitLocalRef(localRef: RexLocalRef): GeneratedExpression = + throw new CodeGenException("Local variables are not supported yet.") + + override def visitRangeRef(rangeRef: RexRangeRef): GeneratedExpression = + throw new CodeGenException("Range references are not supported yet.") + + override def visitDynamicParam(dynamicParam: RexDynamicParam): GeneratedExpression = + throw new CodeGenException("Dynamic parameter references are not supported yet.") + + override def visitCall(call: RexCall): GeneratedExpression = { + + // special case: time materialization + if (call.getOperator == ProctimeSqlFunction) { + return generateProctimeTimestamp() + } + + val resultType = FlinkTypeFactory.toTypeInfo(call.getType) + + // convert operands and help giving untyped NULL literals a type + val operands = call.getOperands.zipWithIndex.map { + + // this helps e.g. for AS(null) + // we might need to extend this logic in case some rules do not create typed NULLs + case (operandLiteral: RexLiteral, 0) if + operandLiteral.getType.getSqlTypeName == SqlTypeName.NULL && + call.getOperator.getReturnTypeInference == ReturnTypes.ARG0 => + generateNullLiteral(resultType) + + case (o@_, _) => + o.accept(this) + } + generateCall(call, operands, resultType) + } + def generateCall( + call: RexCall, + operands: Seq[GeneratedExpression], + resultType: TypeInformation[_]) + : GeneratedExpression = { + + call.getOperator match { + // arithmetic + case PLUS if isNumeric(resultType) => + val left = operands.head + val right = operands(1) + requireNumeric(left) + requireNumeric(right) + generateArithmeticOperator("+", nullCheck, resultType, left, right, config) + + case PLUS | DATETIME_PLUS if isTemporal(resultType) => + val left = operands.head + val right = operands(1) + requireTemporal(left) + requireTemporal(right) + generateTemporalPlusMinus(plus = true, nullCheck, left, right, config) + + case MINUS if isNumeric(resultType) => + val left = operands.head + val right = operands(1) + requireNumeric(left) + requireNumeric(right) + generateArithmeticOperator("-", nullCheck, resultType, left, right, config) + + case MINUS | MINUS_DATE if isTemporal(resultType) => + val left = operands.head + val right = operands(1) + requireTemporal(left) + requireTemporal(right) + generateTemporalPlusMinus(plus = false, nullCheck, left, right, config) + + case MULTIPLY if isNumeric(resultType) => + val left = operands.head + val right = operands(1) + requireNumeric(left) + requireNumeric(right) + generateArithmeticOperator("*", nullCheck, resultType, left, right, config) + + case MULTIPLY if isTimeInterval(resultType) => + val left = operands.head + val right = operands(1) + requireTimeInterval(left) + requireNumeric(right) + generateArithmeticOperator("*", nullCheck, resultType, left, right, config) + + case DIVIDE | DIVIDE_INTEGER if isNumeric(resultType) => + val left = operands.head + val right = operands(1) + requireNumeric(left) + requireNumeric(right) + generateArithmeticOperator("/", nullCheck, resultType, left, right, config) + + case MOD if isNumeric(resultType) => + val left = operands.head + val right = operands(1) + requireNumeric(left) + requireNumeric(right) + generateArithmeticOperator("%", nullCheck, resultType, left, right, config) + + case UNARY_MINUS if isNumeric(resultType) => + val operand = operands.head + requireNumeric(operand) + generateUnaryArithmeticOperator("-", nullCheck, resultType, operand) + + case UNARY_MINUS if isTimeInterval(resultType) => + val operand = operands.head + requireTimeInterval(operand) + generateUnaryIntervalPlusMinus(plus = false, nullCheck, operand) + + case UNARY_PLUS if isNumeric(resultType) => + val operand = operands.head + requireNumeric(operand) + generateUnaryArithmeticOperator("+", nullCheck, resultType, operand) + + case UNARY_PLUS if isTimeInterval(resultType) => + val operand = operands.head + requireTimeInterval(operand) + generateUnaryIntervalPlusMinus(plus = true, nullCheck, operand) + + // comparison + case EQUALS => + val left = operands.head + val right = operands(1) + generateEquals(nullCheck, left, right) + + case NOT_EQUALS => + val left = operands.head + val right = operands(1) + generateNotEquals(nullCheck, left, right) + + case GREATER_THAN => + val left = operands.head + val right = operands(1) + requireComparable(left) + requireComparable(right) + generateComparison(">", nullCheck, left, right) + + case GREATER_THAN_OR_EQUAL => + val left = operands.head + val right = operands(1) + requireComparable(left) + requireComparable(right) + generateComparison(">=", nullCheck, left, right) + + case LESS_THAN => + val left = operands.head + val right = operands(1) + requireComparable(left) + requireComparable(right) + generateComparison("<", nullCheck, left, right) + + case LESS_THAN_OR_EQUAL => + val left = operands.head + val right = operands(1) + requireComparable(left) + requireComparable(right) + generateComparison("<=", nullCheck, left, right) + + case IS_NULL => + val operand = operands.head + generateIsNull(nullCheck, operand) + + case IS_NOT_NULL => + val operand = operands.head + generateIsNotNull(nullCheck, operand) + + // logic + case AND => + operands.reduceLeft { (left: GeneratedExpression, right: GeneratedExpression) => + requireBoolean(left) + requireBoolean(right) + generateAnd(nullCheck, left, right) + } + + case OR => + operands.reduceLeft { (left: GeneratedExpression, right: GeneratedExpression) => + requireBoolean(left) + requireBoolean(right) + generateOr(nullCheck, left, right) + } + + case NOT => + val operand = operands.head + requireBoolean(operand) + generateNot(nullCheck, operand) + + case CASE => + generateIfElse(nullCheck, operands, resultType) + + case IS_TRUE => + val operand = operands.head + requireBoolean(operand) + generateIsTrue(operand) + + case IS_NOT_TRUE => + val operand = operands.head + requireBoolean(operand) + generateIsNotTrue(operand) + + case IS_FALSE => + val operand = operands.head + requireBoolean(operand) + generateIsFalse(operand) + + case IS_NOT_FALSE => + val operand = operands.head + requireBoolean(operand) + generateIsNotFalse(operand) + + case IN => + val left = operands.head + val right = operands.tail + generateIn(this, left, right) + + // casting + case CAST | REINTERPRET => + val operand = operands.head + generateCast(nullCheck, operand, resultType) + + // as / renaming + case AS => + operands.head + + // string arithmetic + case CONCAT => + val left = operands.head + val right = operands(1) + requireString(left) + generateArithmeticOperator("+", nullCheck, resultType, left, right, config) + + // rows + case ROW => + generateRow(this, resultType, operands) + + // arrays + case ARRAY_VALUE_CONSTRUCTOR => + generateArray(this, resultType, operands) + + // maps + case MAP_VALUE_CONSTRUCTOR => + generateMap(this, resultType, operands) + + case ITEM => + operands.head.resultType match { + case t: TypeInformation[_] if isArray(t) => + val array = operands.head + val index = operands(1) + requireInteger(index) + generateArrayElementAt(this, array, index) + + case t: TypeInformation[_] if isMap(t) => + val key = operands(1) + generateMapGet(this, operands.head, key) + + case _ => throw new CodeGenException("Expect an array or a map.") + } + + case CARDINALITY => + operands.head.resultType match { + case t: TypeInformation[_] if isArray(t) => + val array = operands.head + generateArrayCardinality(nullCheck, array) + + case t: TypeInformation[_] if isMap(t) => + val map = operands.head + generateMapCardinality(nullCheck, map) + + case _ => throw new CodeGenException("Expect an array or a map.") + } + + case ELEMENT => + val array = operands.head + requireArray(array) + generateArrayElement(this, array) + + case DOT => + // Due to https://issues.apache.org/jira/browse/CALCITE-2162, expression such as + // "array[1].a.b" won't work now. + if (operands.size > 2) { + throw new CodeGenException( + "A DOT operator with more than 2 operands is not supported yet.") + } + val fieldName = call.operands.get(1).asInstanceOf[RexLiteral].getValueAs(classOf[String]) + val fieldIdx = operands + .head + .resultType + .asInstanceOf[CompositeType[_]] + .getFieldIndex(fieldName) + generateFieldAccess(operands.head, fieldIdx) + + case ScalarSqlFunctions.CONCAT => + generateConcat(this.nullCheck, operands) + + case ScalarSqlFunctions.CONCAT_WS => + generateConcatWs(operands) + + case StreamRecordTimestampSqlFunction => + generateStreamRecordRowtimeAccess() + + // advanced scalar functions + case sqlOperator: SqlOperator => + val callGen = FunctionGenerator.getCallGenerator( + sqlOperator, + operands.map(_.resultType), + resultType) + callGen + .getOrElse(throw new CodeGenException(s"Unsupported call: $sqlOperator \n" + + s"If you think this function should be supported, " + + s"you can create an issue and start a discussion for it.")) + .generate(this, operands) + + // unknown or invalid + case call@_ => + throw new CodeGenException(s"Unsupported call: $call") + } + } + + override def visitOver(over: RexOver): GeneratedExpression = + throw new CodeGenException("Aggregate functions over windows are not supported yet.") + + override def visitSubQuery(subQuery: RexSubQuery): GeneratedExpression = + throw new CodeGenException("Subqueries are not supported yet.") + + override def visitPatternFieldRef(fieldRef: RexPatternFieldRef): GeneratedExpression = + throw new CodeGenException("Pattern field references are not supported yet.") + + // ---------------------------------------------------------------------------------------------- + // generator helping methods + // ---------------------------------------------------------------------------------------------- + + private def generateCodeSplits(splits: Seq[String]): String = { + val totalLen = splits.map(_.length + 1).sum // 1 for a line break + + // split + if (totalLen > config.getMaxGeneratedCodeLength) { + + hasCodeSplits = true + + // add input unboxing to member area such that all split functions can access it + reusableInputUnboxingExprs.foreach { case (_, expr) => + + // declaration + val resultTypeTerm = primitiveTypeTermForTypeInfo(expr.resultType) + if (nullCheck) { + reusableMemberStatements.add(s"private boolean ${expr.nullTerm};") + } + reusableMemberStatements.add(s"private $resultTypeTerm ${expr.resultTerm};") + + // assignment + if (nullCheck) { + reusablePerRecordStatements.add(s"this.${expr.nullTerm} = ${expr.nullTerm};") + } + reusablePerRecordStatements.add(s"this.${expr.resultTerm} = ${expr.resultTerm};") + } + + // add split methods to the member area and return the code necessary to call those methods + val methodCalls = splits.map { split => + val methodName = newName(s"split") + + val method = + s""" + |private final void $methodName() throws Exception { + | $split + |} + |""".stripMargin + reusableMemberStatements.add(method) + + // create method call + s"$methodName();" + } + + methodCalls.mkString("\n") + } + // don't split + else { + splits.mkString("\n") + } + } + + def generateFieldAccess(refExpr: GeneratedExpression, index: Int): GeneratedExpression = { + + val fieldAccessExpr = generateFieldAccess( + refExpr.resultType, + refExpr.resultTerm, + index) + + val resultTerm = newName("result") + val nullTerm = newName("isNull") + val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldAccessExpr.resultType) + val defaultValue = primitiveDefaultValue(fieldAccessExpr.resultType) + val resultCode = if (nullCheck) { + s""" + |${refExpr.code} + |$resultTypeTerm $resultTerm; + |boolean $nullTerm; + |if (${refExpr.nullTerm}) { + | $resultTerm = $defaultValue; + | $nullTerm = true; + |} + |else { + | ${fieldAccessExpr.code} + | $resultTerm = ${fieldAccessExpr.resultTerm}; + | $nullTerm = ${fieldAccessExpr.nullTerm}; + |} + |""".stripMargin + } else { + s""" + |${refExpr.code} + |${fieldAccessExpr.code} + |$resultTypeTerm $resultTerm = ${fieldAccessExpr.resultTerm}; + |""".stripMargin + } + + GeneratedExpression(resultTerm, nullTerm, resultCode, fieldAccessExpr.resultType) + } + + private def generateInputAccess( + inputType: TypeInformation[_ <: Any], + inputTerm: String, + index: Int) + : GeneratedExpression = { + // if input has been used before, we can reuse the code that + // has already been generated + val inputExpr = reusableInputUnboxingExprs.get((inputTerm, index)) match { + // input access and unboxing has already been generated + case Some(expr) => + expr + + // generate input access and unboxing if necessary + case None => + val expr = if (nullableInput) { + generateNullableInputFieldAccess(inputType, inputTerm, index) + } else { + generateFieldAccess(inputType, inputTerm, index) + } + + reusableInputUnboxingExprs((inputTerm, index)) = expr + expr + } + // hide the generated code as it will be executed only once + GeneratedExpression(inputExpr.resultTerm, inputExpr.nullTerm, "", inputExpr.resultType) + } + + private def generateNullableInputFieldAccess( + inputType: TypeInformation[_ <: Any], + inputTerm: String, + index: Int) + : GeneratedExpression = { + val resultTerm = newName("result") + val nullTerm = newName("isNull") + + val fieldType = inputType match { + case ct: CompositeType[_] => ct.getTypeAt(index) + case t: TypeInformation[_] => t + } + val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType) + val defaultValue = primitiveDefaultValue(fieldType) + val fieldAccessExpr = generateFieldAccess(inputType, inputTerm, index) + + val inputCheckCode = + s""" + |$resultTypeTerm $resultTerm; + |boolean $nullTerm; + |if ($inputTerm == null) { + | $resultTerm = $defaultValue; + | $nullTerm = true; + |} + |else { + | ${fieldAccessExpr.code} + | $resultTerm = ${fieldAccessExpr.resultTerm}; + | $nullTerm = ${fieldAccessExpr.nullTerm}; + |} + |""".stripMargin + + GeneratedExpression(resultTerm, nullTerm, inputCheckCode, fieldType) + } + + def generateFieldAccess( + inputType: TypeInformation[_], + inputTerm: String, + index: Int) + : GeneratedExpression = { + inputType match { + + case ct: CompositeType[_] => + val accessor = fieldAccessorFor(ct, index) + val fieldType: TypeInformation[Any] = ct.getTypeAt(index) + val fieldTypeTerm = boxedTypeTermForTypeInfo(fieldType) + + accessor match { + case ObjectFieldAccessor(field) => + // primitive + if (isFieldPrimitive(field)) { + generateTerm(fieldType, s"$inputTerm.${field.getName}") + } + // Object + else { + generateInputFieldUnboxing( + fieldType, + s"($fieldTypeTerm) $inputTerm.${field.getName}") + } + + case ObjectGenericFieldAccessor(fieldName) => + // Object + val inputCode = s"($fieldTypeTerm) $inputTerm.$fieldName" + generateInputFieldUnboxing(fieldType, inputCode) + + case ObjectMethodAccessor(methodName) => + // Object + val inputCode = s"($fieldTypeTerm) $inputTerm.$methodName()" + generateInputFieldUnboxing(fieldType, inputCode) + + case ProductAccessor(i) => + // Object + val inputCode = s"($fieldTypeTerm) $inputTerm.getField($i)" + generateInputFieldUnboxing(fieldType, inputCode) + + case ObjectPrivateFieldAccessor(field) => + val fieldTerm = addReusablePrivateFieldAccess(ct.getTypeClass, field.getName) + val reflectiveAccessCode = reflectiveFieldReadAccess(fieldTerm, field, inputTerm) + // primitive + if (isFieldPrimitive(field)) { + generateTerm(fieldType, reflectiveAccessCode) + } + // Object + else { + generateInputFieldUnboxing(fieldType, reflectiveAccessCode) + } + } + + case t: TypeInformation[_] => + val fieldTypeTerm = boxedTypeTermForTypeInfo(t) + val inputCode = s"($fieldTypeTerm) $inputTerm" + generateInputFieldUnboxing(t, inputCode) + } + } + + private def generateNullLiteral(resultType: TypeInformation[_]): GeneratedExpression = { + val resultTerm = newName("result") + val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType) + val defaultValue = primitiveDefaultValue(resultType) + + if (nullCheck) { + val wrappedCode = s""" + |$resultTypeTerm $resultTerm = $defaultValue; + |""".stripMargin + + // mark this expression as a constant literal + GeneratedExpression(resultTerm, ALWAYS_NULL, wrappedCode, resultType, literal = true) + } else { + throw new CodeGenException("Null literals are not allowed if nullCheck is disabled.") + } + } + + private[flink] def generateNonNullLiteral( + literalType: TypeInformation[_], + literalCode: String) + : GeneratedExpression = { + + // mark this expression as a constant literal + generateTerm(literalType, literalCode).copy(literal = true) + } + + private[flink] def generateSymbol(enum: Enum[_]): GeneratedExpression = { + GeneratedExpression( + qualifyEnum(enum), + NEVER_NULL, + NO_CODE, + new GenericTypeInfo(enum.getDeclaringClass)) + } + + /** + * Generates access to a term (e.g. a field) that does not require unboxing logic. + * + * @param fieldType type of field + * @param fieldTerm expression term of field (already unboxed) + * @return internal unboxed field representation + */ + private[flink] def generateTerm( + fieldType: TypeInformation[_], + fieldTerm: String) + : GeneratedExpression = { + val resultTerm = newName("result") + val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType) + + val resultCode = s""" + |$resultTypeTerm $resultTerm = $fieldTerm; + |""".stripMargin + + GeneratedExpression(resultTerm, NEVER_NULL, resultCode, fieldType) + } + + /** + * Converts the external boxed format to an internal mostly primitive field representation. + * Wrapper types can autoboxed to their corresponding primitive type (Integer -> int). External + * objects are converted to their internal representation (Timestamp -> internal timestamp + * in long). + * + * @param fieldType type of field + * @param fieldTerm expression term of field to be unboxed + * @return internal unboxed field representation + */ + private[flink] def generateInputFieldUnboxing( + fieldType: TypeInformation[_], + fieldTerm: String) + : GeneratedExpression = { + val resultTerm = newName("result") + val nullTerm = newName("isNull") + val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType) + val defaultValue = primitiveDefaultValue(fieldType) + + // explicit unboxing + val unboxedFieldCode = if (isTimePoint(fieldType)) { + timePointToInternalCode(fieldType, fieldTerm) + } else { + fieldTerm + } + + val wrappedCode = if (nullCheck && !isReference(fieldType)) { + // assumes that fieldType is a boxed primitive. + s""" + |boolean $nullTerm = $fieldTerm == null; + |$resultTypeTerm $resultTerm; + |if ($nullTerm) { + | $resultTerm = $defaultValue; + |} + |else { + | $resultTerm = $fieldTerm; + |} + |""".stripMargin + } else if (nullCheck) { + s""" + |boolean $nullTerm = $fieldTerm == null; + |$resultTypeTerm $resultTerm; + |if ($nullTerm) { + | $resultTerm = $defaultValue; + |} + |else { + | $resultTerm = ($resultTypeTerm) $unboxedFieldCode; + |} + |""".stripMargin + } else { + s""" + |$resultTypeTerm $resultTerm = ($resultTypeTerm) $unboxedFieldCode; + |""".stripMargin + } + + GeneratedExpression(resultTerm, nullTerm, wrappedCode, fieldType) + } + + /** + * Converts the internal mostly primitive field representation to an external boxed format. + * Primitive types can autoboxed to their corresponding object type (int -> Integer). Internal + * representations are converted to their external objects (internal timestamp + * in long -> Timestamp). + * + * @param expr expression to be boxed + * @return external boxed field representation + */ + private[flink] def generateOutputFieldBoxing(expr: GeneratedExpression): GeneratedExpression = { + expr.resultType match { + // convert internal date/time/timestamp to java.sql.* objects + case SqlTimeTypeInfo.DATE | SqlTimeTypeInfo.TIME | SqlTimeTypeInfo.TIMESTAMP => + val resultTerm = newName("result") + val resultTypeTerm = boxedTypeTermForTypeInfo(expr.resultType) + val convMethod = internalToTimePointCode(expr.resultType, expr.resultTerm) + + val resultCode = if (nullCheck) { + s""" + |${expr.code} + |$resultTypeTerm $resultTerm; + |if (${expr.nullTerm}) { + | $resultTerm = null; + |} + |else { + | $resultTerm = $convMethod; + |} + |""".stripMargin + } else { + s""" + |${expr.code} + |$resultTypeTerm $resultTerm = $convMethod; + |""".stripMargin + } + + GeneratedExpression(resultTerm, expr.nullTerm, resultCode, expr.resultType) + + // other types are autoboxed or need no boxing + case _ => expr + } + } + + private[flink] def generateNullableOutputBoxing( + expr: GeneratedExpression, + typeInfo: TypeInformation[_]) + : GeneratedExpression = { + val boxedExpr = generateOutputFieldBoxing(generateCast(nullCheck, expr, typeInfo)) + val boxedTypeTerm = boxedTypeTermForTypeInfo(typeInfo) + val exprOrNull: String = if (nullCheck) { + s"${boxedExpr.nullTerm} ? null : ($boxedTypeTerm) ${boxedExpr.resultTerm}" + } else { + boxedExpr.resultTerm + } + boxedExpr.copy(resultTerm = exprOrNull) + } + + private[flink] def generateStreamRecordRowtimeAccess(): GeneratedExpression = { + val resultTerm = newName("result") + val nullTerm = newName("isNull") + + val accessCode = + s""" + |Long $resultTerm = $contextTerm.timestamp(); + |if ($resultTerm == null) { + | throw new RuntimeException("Rowtime timestamp is null. Please make sure that a proper " + + | "TimestampAssigner is defined and the stream environment uses the EventTime time " + + | "characteristic."); + |} + |boolean $nullTerm = false; + """.stripMargin + + GeneratedExpression(resultTerm, nullTerm, accessCode, Types.LONG) + } + + private[flink] def generateProctimeTimestamp(): GeneratedExpression = { + val resultTerm = newName("result") + + val resultCode = + s""" + |long $resultTerm = $contextTerm.timerService().currentProcessingTime(); + |""".stripMargin + GeneratedExpression(resultTerm, NEVER_NULL, resultCode, SqlTimeTypeInfo.TIMESTAMP) + } + + private[flink] def generateCurrentTimestamp(): GeneratedExpression = { + new CurrentTimePointCallGen(Types.SQL_TIMESTAMP, false).generate(this, Seq()) + } + + // ---------------------------------------------------------------------------------------------- + // Reusable code snippets + // ---------------------------------------------------------------------------------------------- + + /** + * Adds a reusable output record to the member area of the generated [[Function]]. + * The passed [[TypeInformation]] defines the type class to be instantiated. + * + * @param ti type information of type class to be instantiated during runtime + * @return member variable term + */ + def addReusableOutRecord(ti: TypeInformation[_]): Unit = { + val statement = ti match { + case rt: RowTypeInfo => + s""" + |final ${ti.getTypeClass.getCanonicalName} $outRecordTerm = + | new ${ti.getTypeClass.getCanonicalName}(${rt.getArity}); + |""".stripMargin + case _ => + s""" + |final ${ti.getTypeClass.getCanonicalName} $outRecordTerm = + | new ${ti.getTypeClass.getCanonicalName}(); + |""".stripMargin + } + reusableMemberStatements.add(statement) + } + + /** + * Adds a reusable [[java.lang.reflect.Field]] to the member area of the generated [[Function]]. + * The field can be used for accessing POJO fields more efficiently during runtime, however, + * the field does not have to be public. + * + * @param clazz class of containing field + * @param fieldName name of field to be extracted and instantiated during runtime + * @return member variable term + */ + def addReusablePrivateFieldAccess(clazz: Class[_], fieldName: String): String = { + val fieldTerm = s"field_${clazz.getCanonicalName.replace('.', '$')}_$fieldName" + val fieldExtraction = + s""" + |final java.lang.reflect.Field $fieldTerm = + | org.apache.flink.api.java.typeutils.TypeExtractor.getDeclaredField( + | ${clazz.getCanonicalName}.class, "$fieldName"); + |""".stripMargin + reusableMemberStatements.add(fieldExtraction) + + val fieldAccessibility = + s""" + |$fieldTerm.setAccessible(true); + |""".stripMargin + reusableInitStatements.add(fieldAccessibility) + + fieldTerm + } + + /** + * Adds a reusable [[java.math.BigDecimal]] to the member area of the generated [[Function]]. + * + * @param decimal decimal object to be instantiated during runtime + * @return member variable term + */ + def addReusableDecimal(decimal: JBigDecimal): String = decimal match { + case JBigDecimal.ZERO => "java.math.BigDecimal.ZERO" + case JBigDecimal.ONE => "java.math.BigDecimal.ONE" + case JBigDecimal.TEN => "java.math.BigDecimal.TEN" + case _ => + val fieldTerm = newName("decimal") + val fieldDecimal = + s""" + |final java.math.BigDecimal $fieldTerm = + | new java.math.BigDecimal("${decimal.toString}"); + |""".stripMargin + reusableMemberStatements.add(fieldDecimal) + fieldTerm + } + + /** + * Adds a reusable [[java.util.Random]] to the member area of the generated [[Function]]. + * + * The seed parameter must be a literal/constant expression. + * + * @return member variable term + */ + def addReusableRandom(seedExpr: Option[GeneratedExpression]): String = { + val fieldTerm = newName("random") + + val field = + s""" + |final java.util.Random $fieldTerm; + |""".stripMargin + reusableMemberStatements.add(field) + + val fieldInit = seedExpr match { + case Some(s) if nullCheck => + s""" + |${s.code} + |if(!${s.nullTerm}) { + | $fieldTerm = new java.util.Random(${s.resultTerm}); + |} + |else { + | $fieldTerm = new java.util.Random(); + |} + |""".stripMargin + case Some(s) => + s""" + |${s.code} + |$fieldTerm = new java.util.Random(${s.resultTerm}); + |""".stripMargin + case _ => + s""" + |$fieldTerm = new java.util.Random(); + |""".stripMargin + } + + reusableInitStatements.add(fieldInit) + fieldTerm + } + + /** + * Adds a reusable DateFormatter to the member area of the generated [[Function]]. + * + * @return member variable term + */ + def addReusableDateFormatter(format: GeneratedExpression): String = { + val fieldTerm = newName("dateFormatter") + + val field = + s""" + |final ${classOf[DateTimeFormatter].getCanonicalName} $fieldTerm; + |""".stripMargin + reusableMemberStatements.add(field) + + val fieldInit = + s""" + |${format.code} + |$fieldTerm = org.apache.flink.table.runtime.functions. + |DateTimeFunctions$$.MODULE$$.createDateTimeFormatter(${format.resultTerm}); + |""".stripMargin + + reusableInitStatements.add(fieldInit) + fieldTerm + } + + /** + * Adds a reusable [[UserDefinedFunction]] to the member area of the generated [[Function]]. + * + * @param function [[UserDefinedFunction]] object to be instantiated during runtime + * @param contextTerm [[RuntimeContext]] term to access the [[RuntimeContext]] + * @return member variable term + */ + def addReusableFunction(function: UserDefinedFunction, contextTerm: String = null): String = { + val classQualifier = function.getClass.getCanonicalName + val functionSerializedData = UserDefinedFunctionUtils.serialize(function) + val fieldTerm = s"function_${function.functionIdentifier}" + + val fieldFunction = + s""" + |final $classQualifier $fieldTerm; + |""".stripMargin + reusableMemberStatements.add(fieldFunction) + + val functionDeserialization = + s""" + |$fieldTerm = ($classQualifier) + |${UserDefinedFunctionUtils.getClass.getName.stripSuffix("$")} + |.deserialize("$functionSerializedData"); + """.stripMargin + + reusableInitStatements.add(functionDeserialization) + + val openFunction = if (contextTerm != null) { + s""" + |$fieldTerm.open(new ${classOf[FunctionContext].getCanonicalName}($contextTerm)); + """.stripMargin + } else { + s""" + |$fieldTerm.open(new ${classOf[FunctionContext].getCanonicalName}(getRuntimeContext())); + """.stripMargin + } + reusableOpenStatements.add(openFunction) + + val closeFunction = + s""" + |$fieldTerm.close(); + """.stripMargin + reusableCloseStatements.add(closeFunction) + + fieldTerm + } + + /** + * Adds a reusable constructor statement with the given parameter types. + * + * @param parameterTypes The parameter types to construct the function + * @return member variable terms + */ + def addReusableConstructor(parameterTypes: Class[_]*): Array[String] = { + val parameters = mutable.ListBuffer[String]() + val fieldTerms = mutable.ListBuffer[String]() + val body = mutable.ListBuffer[String]() + + parameterTypes.zipWithIndex.foreach { case (t, index) => + val classQualifier = t.getCanonicalName + val fieldTerm = newName(s"instance_${classQualifier.replace('.', '$')}") + val field = s"final $classQualifier $fieldTerm;" + reusableMemberStatements.add(field) + fieldTerms += fieldTerm + parameters += s"$classQualifier arg$index" + body += s"$fieldTerm = arg$index;" + } + + reusableConstructorStatements.add((parameters.mkString(","), body.mkString("", "\n", "\n"))) + + fieldTerms.toArray + } + + /** + * Adds a reusable [[org.apache.flink.types.Row]] + * to the member area of the generated [[Function]]. + */ + def addReusableRow(arity: Int): String = { + val fieldTerm = newName("row") + val fieldRow = + s""" + |final org.apache.flink.types.Row $fieldTerm = + | new org.apache.flink.types.Row($arity); + |""".stripMargin + reusableMemberStatements.add(fieldRow) + fieldTerm + } + + /** + * Adds a reusable array to the member area of the generated [[Function]]. + */ + def addReusableArray(clazz: Class[_], size: Int): String = { + val fieldTerm = newName("array") + val classQualifier = clazz.getCanonicalName // works also for int[] etc. + val initArray = classQualifier.replaceFirst("\\[", s"[$size") + val fieldArray = + s""" + |final $classQualifier $fieldTerm = + | new $initArray; + |""".stripMargin + reusableMemberStatements.add(fieldArray) + fieldTerm + } + + /** + * Adds a reusable hash map to the member area of the generated [[Function]]. + */ + def addReusableMap(): String = { + val fieldTerm = newName("map") + val classQualifier = "java.util.Map" + val initMap = "java.util.HashMap()" + val fieldMap = + s""" + |final $classQualifier $fieldTerm = + | new $initMap; + |""".stripMargin + reusableMemberStatements.add(fieldMap) + fieldTerm + } + + /** + * Adds a reusable timestamp to the beginning of the SAM of the generated [[Function]]. + */ + def addReusableTimestamp(): String = { + val fieldTerm = s"timestamp" + + // declaration + reusableMemberStatements.add(s"private long $fieldTerm;") + + // assignment + val field = + s""" + |$fieldTerm = java.lang.System.currentTimeMillis(); + |""".stripMargin + reusablePerRecordStatements.add(field) + fieldTerm + } + + /** + * Adds a reusable local timestamp to the beginning of the SAM of the generated [[Function]]. + */ + def addReusableLocalTimestamp(): String = { + val fieldTerm = s"localtimestamp" + + val timestamp = addReusableTimestamp() + + // declaration + reusableMemberStatements.add(s"private long $fieldTerm;") + + // assignment + val field = + s""" + |$fieldTerm = $timestamp + java.util.TimeZone.getDefault().getOffset($timestamp); + |""".stripMargin + reusablePerRecordStatements.add(field) + fieldTerm + } + + /** + * Adds a reusable time to the beginning of the SAM of the generated [[Function]]. + */ + def addReusableTime(): String = { + val fieldTerm = s"time" + + val timestamp = addReusableTimestamp() + + // declaration + reusableMemberStatements.add(s"private int $fieldTerm;") + + // assignment + // adopted from org.apache.calcite.runtime.SqlFunctions.currentTime() + val field = + s""" + |$fieldTerm = (int) ($timestamp % ${DateTimeUtils.MILLIS_PER_DAY}); + |if (time < 0) { + | time += ${DateTimeUtils.MILLIS_PER_DAY}; + |} + |""".stripMargin + reusablePerRecordStatements.add(field) + fieldTerm + } + + /** + * Adds a reusable local time to the beginning of the SAM of the generated [[Function]]. + */ + def addReusableLocalTime(): String = { + val fieldTerm = s"localtime" + + val localtimestamp = addReusableLocalTimestamp() + + // declaration + reusableMemberStatements.add(s"private int $fieldTerm;") + + // assignment + // adopted from org.apache.calcite.runtime.SqlFunctions.localTime() + val field = + s""" + |$fieldTerm = (int) ($localtimestamp % ${DateTimeUtils.MILLIS_PER_DAY}); + |""".stripMargin + reusablePerRecordStatements.add(field) + fieldTerm + } + + + /** + * Adds a reusable date to the beginning of the SAM of the generated [[Function]]. + */ + def addReusableDate(): String = { + val fieldTerm = s"date" + + val timestamp = addReusableTimestamp() + val time = addReusableTime() + + // declaration + reusableMemberStatements.add(s"private int $fieldTerm;") + + // assignment + // adopted from org.apache.calcite.runtime.SqlFunctions.currentDate() + val field = + s""" + |$fieldTerm = (int) ($timestamp / ${DateTimeUtils.MILLIS_PER_DAY}); + |if ($time < 0) { + | $fieldTerm -= 1; + |} + |""".stripMargin + reusablePerRecordStatements.add(field) + fieldTerm + } + + /** + * Adds a reusable [[java.util.HashSet]] to the member area of the generated [[Function]]. + * + * @param elements elements to be added to the set (including null) + * @return member variable term + */ + def addReusableSet(elements: Seq[GeneratedExpression]): String = { + val fieldTerm = newName("set") + + val field = + s""" + |final java.util.Set $fieldTerm; + |""".stripMargin + reusableMemberStatements.add(field) + + val init = + s""" + |$fieldTerm = new java.util.HashSet(); + |""".stripMargin + reusableInitStatements.add(init) + + elements.foreach { element => + val content = + s""" + |${element.code} + |if (${element.nullTerm}) { + | $fieldTerm.add(null); + |} else { + | $fieldTerm.add(${element.resultTerm}); + |} + |""".stripMargin + + reusableInitStatements.add(content) + } + + fieldTerm + } + + /** + * Adds a reusable constant to the member area of the generated [[Function]]. + * + * @param constant constant expression + * @return member variable term + */ + def addReusableBoxedConstant(constant: GeneratedExpression): String = { + require(constant.literal, "Literal expected") + + val fieldTerm = newName("constant") + + val boxed = generateOutputFieldBoxing(constant) + val boxedType = boxedTypeTermForTypeInfo(boxed.resultType) + + val field = + s""" + |final $boxedType $fieldTerm; + |""".stripMargin + reusableMemberStatements.add(field) + + val init = + s""" + |${boxed.code} + |$fieldTerm = ${boxed.resultTerm}; + |""".stripMargin + reusableInitStatements.add(init) + + fieldTerm + } + + /** + * Adds a reusable MessageDigest to the member area of the generated [[Function]]. + * + * @return member variable term + */ + def addReusableMessageDigest(algorithm: String): String = { + val fieldTerm = newName("messageDigest") + + val field = + s""" + |final java.security.MessageDigest $fieldTerm; + |""".stripMargin + reusableMemberStatements.add(field) + + val fieldInit = + s""" + |try { + | $fieldTerm = java.security.MessageDigest.getInstance("$algorithm"); + |} catch (java.security.NoSuchAlgorithmException e) { + | throw new RuntimeException("Algorithm for '$algorithm' is not available.", e); + |} + |""".stripMargin + + reusableInitStatements.add(fieldInit) + fieldTerm + } + + def addReusableInitStatement(initStatement: String): Unit = { + reusableInitStatements.add(initStatement) + } + + def addReusableMemberStatement(memberStatement: String): Unit = { + reusableMemberStatements.add(memberStatement) + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala b/core/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala new file mode 100644 index 000000000..d08dd570f --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala @@ -0,0 +1,574 @@ +package org.apache.flink.table.codegen + +import java.math.{BigDecimal => JBigDecimal} +import java.util + +import org.apache.calcite.rel.RelCollation +import org.apache.calcite.rex._ +import org.apache.calcite.sql.fun.SqlStdOperatorTable.{CLASSIFIER, FINAL, FIRST, LAST, MATCH_NUMBER, NEXT, PREV, RUNNING} +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.cep.{PatternFlatSelectFunction, PatternSelectFunction} +import org.apache.flink.cep.pattern.conditions.IterativeCondition +import org.apache.flink.table.api.TableConfig +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.codegen.Indenter.toISC +import org.apache.flink.table.codegen.CodeGenUtils.{boxedTypeTermForTypeInfo, newName, primitiveDefaultValue} +import org.apache.flink.table.plan.schema.RowSchema +import org.apache.flink.types.Row + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +/** + * A code generator for generating CEP related functions. + * + * @param config configuration that determines runtime behavior + * @param nullableInput input(s) can be null. + * @param input type information about the first input of the Function + * @param patternNames the names of patterns + * @param generateCondition whether the code generator is generating [[IterativeCondition]] + * @param patternName the name of current pattern + */ +class MatchCodeGenerator( + config: TableConfig, + nullableInput: Boolean, + input: TypeInformation[_ <: Any], + patternNames: Seq[String], + generateCondition: Boolean, + patternName: Option[String] = None) + extends CodeGenerator(config,nullableInput,input){ + /** + * @return term of pattern names + */ + private val patternNameListTerm = newName("patternNameList") + + /** + * @return term of current pattern which is processing + */ + private val currPatternTerm = newName("currPattern") + + /** + * @return term of current event which is processing + */ + private val currEventTerm = newName("currEvent") + + private val buildPatternNameList: String = { + for (patternName <- patternNames) yield + s""" + |$patternNameListTerm.add("$patternName"); + |""".stripMargin + }.mkString("\n") + + def addReusableStatements(): Unit = { + val eventTypeTerm = boxedTypeTermForTypeInfo(input) + val memberStatement = + s""" + |$eventTypeTerm $currEventTerm = null; + |String $currPatternTerm = null; + |java.util.List $patternNameListTerm = new java.util.ArrayList(); + |""".stripMargin + addReusableMemberStatement(memberStatement) + + addReusableInitStatement(buildPatternNameList) + } + + /** + * Generates a [[IterativeCondition]] that can be passed to Java compiler. + * + * @param name Class name of the function. Must not be unique but has to be a + * valid Java class identifier. + * @param bodyCode body code for the function + * @return a GeneratedIterativeCondition + */ + def generateIterativeCondition( + name: String, + bodyCode: String) + : GeneratedIterativeCondition = { + + val funcName = newName(name) + val inputTypeTerm = boxedTypeTermForTypeInfo(input) + + val funcCode = j""" + public class $funcName + extends ${classOf[IterativeCondition[_]].getCanonicalName} { + + ${reuseMemberCode()} + + public $funcName() throws Exception { + ${reuseInitCode()} + } + + @Override + public boolean filter( + Object _in1, ${classOf[IterativeCondition.Context[_]].getCanonicalName} $contextTerm) + throws Exception { + + $inputTypeTerm $input1Term = ($inputTypeTerm) _in1; + ${reusePerRecordCode()} + ${reuseInputUnboxingCode()} + $bodyCode + } + } + """.stripMargin + + GeneratedIterativeCondition(funcName, funcCode) + } + + /** + * Generates a [[PatternSelectFunction]] that can be passed to Java compiler. + * + * @param name Class name of the function. Must not be unique but has to be a + * valid Java class identifier. + * @param bodyCode body code for the function + * @return a GeneratedPatternSelectFunction + */ + def generatePatternSelectFunction( + name: String, + bodyCode: String) + : GeneratedPatternSelectFunction = { + + val funcName = newName(name) + val inputTypeTerm = + classOf[java.util.Map[java.lang.String, java.util.List[Row]]].getCanonicalName + + val funcCode = j""" + public class $funcName + implements ${classOf[PatternSelectFunction[_, _]].getCanonicalName} { + + ${reuseMemberCode()} + + public $funcName() throws Exception { + ${reuseInitCode()} + } + + @Override + public Object select(java.util.Map> _in1) + throws Exception { + + $inputTypeTerm $input1Term = ($inputTypeTerm) _in1; + ${reusePerRecordCode()} + ${reuseInputUnboxingCode()} + $bodyCode + } + } + """.stripMargin + + GeneratedPatternSelectFunction(funcName, funcCode) + } + + /** + * Generates a [[PatternFlatSelectFunction]] that can be passed to Java compiler. + * + * @param name Class name of the function. Must not be unique but has to be a + * valid Java class identifier. + * @param bodyCode body code for the function + * @return a GeneratedPatternFlatSelectFunction + */ + def generatePatternFlatSelectFunction( + name: String, + bodyCode: String) + : GeneratedPatternFlatSelectFunction = { + + val funcName = newName(name) + val inputTypeTerm = + classOf[java.util.Map[java.lang.String, java.util.List[Row]]].getCanonicalName + + val funcCode = j""" + public class $funcName + implements ${classOf[PatternFlatSelectFunction[_, _]].getCanonicalName} { + + ${reuseMemberCode()} + + public $funcName() throws Exception { + ${reuseInitCode()} + } + + @Override + public void flatSelect(java.util.Map> _in1, + org.apache.flink.util.Collector $collectorTerm) + throws Exception { + + $inputTypeTerm $input1Term = ($inputTypeTerm) _in1; + ${reusePerRecordCode()} + ${reuseInputUnboxingCode()} + $bodyCode + } + } + """.stripMargin + + GeneratedPatternFlatSelectFunction(funcName, funcCode) + } + + def generateSelectOutputExpression( + partitionKeys: util.List[RexNode], + measures: util.Map[String, RexNode], + returnType: RowSchema) + : GeneratedExpression = { + + val eventNameTerm = newName("event") + val eventTypeTerm = boxedTypeTermForTypeInfo(input) + + // For "ONE ROW PER MATCH", the output columns include: + // 1) the partition columns; + // 2) the columns defined in the measures clause. + val resultExprs = + partitionKeys.asScala.map { case inputRef: RexInputRef => + generateFieldAccess(input, eventNameTerm, inputRef.getIndex) + } ++ returnType.fieldNames.filter(measures.containsKey(_)).map { fieldName => + generateExpression(measures.get(fieldName)) + } + + val resultExpression = generateResultExpression( + resultExprs, + returnType.typeInfo, + returnType.fieldNames) + + val resultCode = + s""" + |$eventTypeTerm $eventNameTerm = null; + |if (${partitionKeys.size()} > 0) { + | for (java.util.Map.Entry entry : $input1Term.entrySet()) { + | java.util.List value = (java.util.List) entry.getValue(); + | if (value != null && value.size() > 0) { + | $eventNameTerm = ($eventTypeTerm) value.get(0); + | break; + | } + | } + |} + | + |${resultExpression.code} + |""".stripMargin + + resultExpression.copy(code = resultCode) + } + + def generateFlatSelectOutputExpression( + partitionKeys: util.List[RexNode], + orderKeys: RelCollation, + measures: util.Map[String, RexNode], + returnType: RowSchema) + : GeneratedExpression = { + + val patternNameTerm = newName("patternName") + val eventNameTerm = newName("event") + val eventNameListTerm = newName("eventList") + val eventTypeTerm = boxedTypeTermForTypeInfo(input) + val listTypeTerm = classOf[java.util.List[_]].getCanonicalName + + // For "ALL ROWS PER MATCH", the output columns include: + // 1) the partition columns; + // 2) the ordering columns; + // 3) the columns defined in the measures clause; + // 4) any remaining columns defined of the input. + val fieldsAccessed = mutable.Set[Int]() + val resultExprs = + partitionKeys.asScala.map { case inputRef: RexInputRef => + fieldsAccessed += inputRef.getIndex + generateFieldAccess(input, eventNameTerm, inputRef.getIndex) + } ++ orderKeys.getFieldCollations.asScala.map { fieldCollation => + fieldsAccessed += fieldCollation.getFieldIndex + generateFieldAccess(input, eventNameTerm, fieldCollation.getFieldIndex) + } ++ (0 until input.getArity).filterNot(fieldsAccessed.contains).map { idx => + generateFieldAccess(input, eventNameTerm, idx) + } ++ returnType.fieldNames.filter(measures.containsKey(_)).map { fieldName => + generateExpression(measures.get(fieldName)) + } + + val resultExpression = generateResultExpression( + resultExprs, + returnType.typeInfo, + returnType.fieldNames) + + val resultCode = + s""" + |for (String $patternNameTerm : $patternNameListTerm) { + | $currPatternTerm = $patternNameTerm; + | $listTypeTerm $eventNameListTerm = ($listTypeTerm) $input1Term.get($patternNameTerm); + | if ($eventNameListTerm != null) { + | for ($eventTypeTerm $eventNameTerm : $eventNameListTerm) { + | $currEventTerm = $eventNameTerm; + | ${resultExpression.code} + | $collectorTerm.collect(${resultExpression.resultTerm}); + | } + | } + |} + |$currPatternTerm = null; + |$currEventTerm = null; + |""".stripMargin + + GeneratedExpression("", "false", resultCode, null) + } + + override def visitCall(call: RexCall): GeneratedExpression = { + val resultType = FlinkTypeFactory.toTypeInfo(call.getType) + call.getOperator match { + case PREV => + val countLiteral = call.operands.get(1).asInstanceOf[RexLiteral] + val count = countLiteral.getValue3.asInstanceOf[JBigDecimal].intValue() + generatePrev( + call.operands.get(0), + count, + resultType) + + case NEXT | CLASSIFIER | MATCH_NUMBER => + throw new CodeGenException(s"Unsupported call: $call") + + case FIRST | LAST => + val countLiteral = call.operands.get(1).asInstanceOf[RexLiteral] + val count = countLiteral.getValue3.asInstanceOf[JBigDecimal].intValue() + generateFirstLast( + call.operands.get(0), + count, + resultType, + running = true, + call.getOperator == FIRST) + + case RUNNING | FINAL => + generateRunningFinal( + call.operands.get(0), + resultType, + call.getOperator == RUNNING) + + case _ => super.visitCall(call) + } + } + + private def generatePrev( + rexNode: RexNode, + count: Int, + resultType: TypeInformation[_]) + : GeneratedExpression = { + rexNode match { + case patternFieldRef: RexPatternFieldRef => + if (count == 0 && patternFieldRef.getAlpha == patternName.get) { + // return current one + return visitInputRef(patternFieldRef) + } + + val listName = newName("patternEvents") + val resultTerm = newName("result") + val nullTerm = newName("isNull") + val indexTerm = newName("eventIndex") + val visitedEventNumberTerm = newName("visitedEventNumber") + val eventTerm = newName("event") + val resultTypeTerm = boxedTypeTermForTypeInfo(resultType) + val defaultValue = primitiveDefaultValue(resultType) + + val eventTypeTerm = boxedTypeTermForTypeInfo(input) + + val patternNamesToVisit = patternNames + .take(patternNames.indexOf(patternFieldRef.getAlpha) + 1) + .reverse + def findEventByPhysicalPosition: String = { + val init: String = + s""" + |java.util.List $listName = new java.util.ArrayList(); + |""".stripMargin + + val getResult: String = { + for (tmpPatternName <- patternNamesToVisit) yield + s""" + |for ($eventTypeTerm $eventTerm : $contextTerm + | .getEventsForPattern("$tmpPatternName")) { + | $listName.add($eventTerm); + |} + | + |$indexTerm = $listName.size() - ($count - $visitedEventNumberTerm); + |if ($indexTerm >= 0) { + | $resultTerm = ($resultTypeTerm) (($eventTypeTerm) $listName.get($indexTerm)) + | .getField(${patternFieldRef.getIndex}); + | $nullTerm = false; + | break; + |} + | + |$visitedEventNumberTerm += $listName.size(); + |$listName.clear(); + |""".stripMargin + }.mkString("\n") + + s""" + |$init + |$getResult + |""".stripMargin + } + + val resultCode = + s""" + |int $visitedEventNumberTerm = 0; + |int $indexTerm; + |$resultTypeTerm $resultTerm = $defaultValue; + |boolean $nullTerm = true; + |do { + | $findEventByPhysicalPosition + |} while (false); + |""".stripMargin + + GeneratedExpression(resultTerm, nullTerm, resultCode, resultType) + + case rexCall: RexCall => + val operands = rexCall.operands.asScala.map { + operand => generatePrev( + operand, + count, + FlinkTypeFactory.toTypeInfo(operand.getType)) + } + + generateCall(rexCall, operands, resultType) + + case _ => + generateExpression(rexNode) + } + } + + private def generateFirstLast( + rexNode: RexNode, + count: Int, + resultType: TypeInformation[_], + running: Boolean, + first: Boolean) + : GeneratedExpression = { + rexNode match { + case patternFieldRef: RexPatternFieldRef => + + val eventNameTerm = newName("event") + val resultTerm = newName("result") + val listName = newName("patternEvents") + val nullTerm = newName("isNull") + val patternNameTerm = newName("patternName") + val eventNameListTerm = newName("eventNameList") + val resultTypeTerm = boxedTypeTermForTypeInfo(resultType) + val defaultValue = primitiveDefaultValue(resultType) + + val eventTypeTerm = boxedTypeTermForTypeInfo(input) + val listTypeTerm = classOf[java.util.List[_]].getCanonicalName + + def findEventByLogicalPosition: String = { + val init = + s""" + |java.util.List $listName = new java.util.ArrayList(); + |""".stripMargin + + val findEventsByPatterName = if (generateCondition) { + s""" + |for ($eventTypeTerm $eventNameTerm : $contextTerm + | .getEventsForPattern("${patternFieldRef.getAlpha}")) { + | $listName.add($eventNameTerm); + |} + |""".stripMargin + } else { + s""" + |for (String $patternNameTerm : $patternNameListTerm) { + | if ($patternNameTerm.equals("${patternFieldRef.getAlpha}") || + | ${patternFieldRef.getAlpha.equals("*")}) { + | boolean skipLoop = false; + | $listTypeTerm $eventNameListTerm = + | ($listTypeTerm) $input1Term.get($patternNameTerm); + | if ($eventNameListTerm != null) { + | for ($eventTypeTerm $eventNameTerm : $eventNameListTerm) { + | $listName.add($eventNameTerm); + | if ($running && $eventNameTerm == $currEventTerm) { + | skipLoop = true; + | break; + | } + | } + | } + | + | if (skipLoop) { + | break; + | } + | } + | + | if ($running && $patternNameTerm.equals($currPatternTerm)) { + | break; + | } + |} + |""".stripMargin + } + + val getResult = + s""" + |if ($listName.size() > $count) { + | if ($first) { + | $resultTerm = ($resultTypeTerm) (($eventTypeTerm) + | $listName.get($count)) + | .getField(${patternFieldRef.getIndex}); + | } else { + | $resultTerm = ($resultTypeTerm) (($eventTypeTerm) + | $listName.get($listName.size() - $count - 1)) + | .getField(${patternFieldRef.getIndex}); + | } + | $nullTerm = false; + |} + |""".stripMargin + + s""" + |$init + |$findEventsByPatterName + |$getResult + |""".stripMargin + } + + val resultCode = + s""" + |$resultTypeTerm $resultTerm = $defaultValue; + |boolean $nullTerm = true; + |$findEventByLogicalPosition + |""".stripMargin + + GeneratedExpression(resultTerm, nullTerm, resultCode, resultType) + + case rexCall: RexCall => + val operands = rexCall.operands.asScala.map { + operand => generateFirstLast( + operand, + count, + FlinkTypeFactory.toTypeInfo(operand.getType), + running, + first) + } + + generateCall(rexCall, operands, resultType) + + case _ => + generateExpression(rexNode) + } + } + + /* + + + */ + private def generateRunningFinal( + rexNode: RexNode, + resultType: TypeInformation[_], + running: Boolean): GeneratedExpression + = { + rexNode match { + case _: RexPatternFieldRef => + generateFirstLast(rexNode, 0, resultType, running, first = false) + + case rexCall: RexCall if rexCall.getOperator == FIRST || rexCall.getOperator == LAST => + val countLiteral = rexCall.operands.get(1).asInstanceOf[RexLiteral] + val count = countLiteral.getValue3.asInstanceOf[JBigDecimal].intValue() + generateFirstLast( + rexCall.operands.get(0), + count, + resultType, + running, + rexCall.getOperator == FIRST) + + case rexCall: RexCall => + val operands = rexCall.operands.asScala.map { + operand => generateRunningFinal( + operand, + FlinkTypeFactory.toTypeInfo(operand.getType), + running) + } + + generateCall(rexCall, operands, resultType) + + case _ => + generateExpression(rexNode) + } + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/flink/table/codegen/generated.scala b/core/src/main/scala/org/apache/flink/table/codegen/generated.scala new file mode 100644 index 000000000..3551d80ed --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/codegen/generated.scala @@ -0,0 +1,100 @@ +package org.apache.flink.table.codegen + +import org.apache.flink.api.common.functions +import org.apache.flink.api.common.functions.Function +import org.apache.flink.api.common.io.InputFormat +import org.apache.flink.api.common.typeinfo.TypeInformation + +/** + * Describes a generated expression. + * + * @param resultTerm term to access the result of the expression + * @param nullTerm boolean term that indicates if expression is null + * @param code code necessary to produce resultTerm and nullTerm + * @param resultType type of the resultTerm + * @param literal flag to indicate a constant expression do not reference input and can thus + * be used in the member area (e.g. as constructor parameter of a reusable + * instance) + */ +case class GeneratedExpression( + resultTerm: String, + nullTerm: String, + code: String, + resultType: TypeInformation[_], + literal: Boolean = false) + +object GeneratedExpression { + val ALWAYS_NULL = "true" + val NEVER_NULL = "false" + val NO_CODE = "" +} + +/** + * Describes a generated [[functions.Function]] + * + * @param name class name of the generated Function. + * @param returnType the type information of the result type + * @param code code of the generated Function. + * @tparam F type of function + * @tparam T type of function + */ +case class GeneratedFunction[F <: Function, T <: Any]( + name: String, + returnType: TypeInformation[T], + code: String) + +/** + * Describes a generated aggregate helper function + * + * @param name class name of the generated Function. + * @param code code of the generated Function. + */ +case class GeneratedAggregationsFunction( + name: String, + code: String) + +/** + * Describes a generated [[InputFormat]]. + * + * @param name class name of the generated input function. + * @param returnType the type information of the result type + * @param code code of the generated Function. + * @tparam F type of function + * @tparam T type of function + */ +case class GeneratedInput[F <: InputFormat[_, _], T <: Any]( + name: String, + returnType: TypeInformation[T], + code: String) + +/** + * Describes a generated [[org.apache.flink.util.Collector]]. + * + * @param name class name of the generated Collector. + * @param code code of the generated Collector. + */ +case class GeneratedCollector(name: String, code: String) + +/** + * Describes a generated [[org.apache.flink.cep.pattern.conditions.IterativeCondition]]. + * + * @param name class name of the generated IterativeCondition. + * @param code code of the generated IterativeCondition. + */ +case class GeneratedIterativeCondition(name: String, code: String) + +/** + * Describes a generated [[org.apache.flink.cep.PatternSelectFunction]]. + * + * @param name class name of the generated PatternSelectFunction. + * @param code code of the generated PatternSelectFunction. + */ +case class GeneratedPatternSelectFunction(name: String, code: String) + +/** + * Describes a generated [[org.apache.flink.cep.PatternFlatSelectFunction]]. + * + * @param name class name of the generated PatternFlatSelectFunction. + * @param code code of the generated PatternFlatSelectFunction. + */ +case class GeneratedPatternFlatSelectFunction(name: String, code: String) diff --git a/core/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala b/core/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala new file mode 100644 index 000000000..10f60ba88 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala @@ -0,0 +1,309 @@ +package org.apache.flink.table.plan.nodes.datastream + +import java.math.{BigDecimal => JBigDecimal} +import java.util + +import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} +import org.apache.calcite.rel._ +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rex._ +import org.apache.calcite.sql.`type`.SqlTypeName._ +import org.apache.calcite.sql.fun.SqlStdOperatorTable._ +import org.apache.flink.cep.{CEP, PatternStream} +import org.apache.flink.cep.pattern.Pattern +import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.streaming.api.windowing.time.Time +import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment, TableException} +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.plan.schema.RowSchema +import org.apache.flink.table.runtime.RowtimeProcessFunction +import org.apache.flink.table.runtime.cepmatch.{ConvertToRow, MatchUtil} +import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} +import org.apache.flink.types.Row + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ListBuffer + +/** + * Flink RelNode which matches along with LogicalMatch. + */ +class DataStreamMatch( + cluster: RelOptCluster, + traitSet: RelTraitSet, + input: RelNode, + pattern: RexNode, + strictStart: Boolean, + strictEnd: Boolean, + patternDefinitions: util.Map[String, RexNode], + measures: util.Map[String, RexNode], + after: RexNode, + subsets: util.Map[String, util.SortedSet[String]], + allRows: Boolean, + partitionKeys: util.List[RexNode], + orderKeys: RelCollation, + interval: RexNode, + schema: RowSchema, + inputSchema: RowSchema) + extends SingleRel(cluster, traitSet, input) + with DataStreamRel { + + override def deriveRowType(): RelDataType = schema.relDataType + + override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = { + new DataStreamMatch( + cluster, + traitSet, + inputs.get(0), + pattern, + strictStart, + strictEnd, + patternDefinitions, + measures, + after, + subsets, + allRows, + partitionKeys, + orderKeys, + interval, + schema, + inputSchema) + } + + override def toString: String = { + s"Match(${ + if (!partitionKeys.isEmpty) { + s"PARTITION BY: ${partitionKeys.toArray.map(_.toString).mkString(", ")}, " + } else { + "" + } + }${ + if (!orderKeys.getFieldCollations.isEmpty) { + s"ORDER BY: ${orderKeys.getFieldCollations.asScala.map { + x => inputSchema.relDataType.getFieldList.get(x.getFieldIndex).getName + }.mkString(", ")}, " + } else { + "" + } + }${ + if (!measures.isEmpty) { + s"MEASURES: ${measures.asScala.map { + case (k, v) => s"${v.toString} AS $k" + }.mkString(", ")}, " + } else { + "" + } + }${ + if (allRows) { + s"ALL ROWS PER MATCH, " + } else { + s"ONE ROW PER MATCH, " + } + }${ + s"${after.toString}, " + }${ + s"PATTERN: (${pattern.toString})" + }${ + if (interval != null) { + s"WITHIN INTERVAL: $interval, " + } else { + s", " + } + }${ + if (!subsets.isEmpty) { + s"SUBSET: ${subsets.asScala.map { + case (k, v) => s"$k = (${v.toArray.mkString(", ")})" + }.mkString(", ")}, " + } else { + "" + } + }${ + s"DEFINE: ${patternDefinitions.asScala.map { + case (k, v) => s"$k AS ${v.toString}" + }.mkString(", ")}" + })" + } + + override def explainTerms(pw: RelWriter): RelWriter = { + pw.input("input", getInput()) + .itemIf("partitionBy", + partitionKeys.toArray.map(_.toString).mkString(", "), + !partitionKeys.isEmpty) + .itemIf("orderBy", + orderKeys.getFieldCollations.asScala.map { + x => inputSchema.relDataType.getFieldList.get(x.getFieldIndex).getName + }.mkString(", "), + !orderKeys.getFieldCollations.isEmpty) + .itemIf("measures", + measures.asScala.map { case (k, v) => s"${v.toString} AS $k"}.mkString(", "), + !measures.isEmpty) + .item("allrows", allRows) + .item("after", after.toString) + .item("pattern", pattern.toString) + .itemIf("within interval", + if (interval != null) { + interval.toString + } else { + null + }, + interval != null) + .itemIf("subset", + subsets.asScala.map { case (k, v) => s"$k = (${v.toArray.mkString(", ")})"}.mkString(", "), + !subsets.isEmpty) + .item("define", + patternDefinitions.asScala.map { case (k, v) => s"$k AS ${v.toString}"}.mkString(", ")) + } + + override def translateToPlan( + tableEnv: StreamTableEnvironment, + queryConfig: StreamQueryConfig): DataStream[CRow] = { + + val config = tableEnv.config + val inputTypeInfo = inputSchema.typeInfo + + val crowInput: DataStream[CRow] = getInput + .asInstanceOf[DataStreamRel] + .translateToPlan(tableEnv, queryConfig) + + val rowtimeFields = inputSchema.relDataType + .getFieldList.asScala + .filter(f => FlinkTypeFactory.isRowtimeIndicatorType(f.getType)) + + //主要是针对rowtime做处理,对存在rowtime的将其复制到StreamRecord的时间戳字段内 + val timestampedInput = if (rowtimeFields.nonEmpty) { + // copy the rowtime field into the StreamRecord timestamp field + val timeIdx = rowtimeFields.head.getIndex + + crowInput + .process(new RowtimeProcessFunction(timeIdx, CRowTypeInfo(inputTypeInfo))) + .setParallelism(crowInput.getParallelism) + .name(s"rowtime field: (${rowtimeFields.head})") + } else { + crowInput + } + + val inputDS: DataStream[Row] = timestampedInput + .map(new ConvertToRow) + .setParallelism(timestampedInput.getParallelism) + .name("ConvertToRow") + .returns(inputTypeInfo) + + def translatePattern( + rexNode: RexNode, + currentPattern: Pattern[Row, Row], + patternNames: ListBuffer[String]): Pattern[Row, Row] = rexNode match { + //Conditions + case literal: RexLiteral => + val patternName = literal.getValue3.toString + patternNames += patternName + val newPattern = next(currentPattern, patternName) + + val patternDefinition = patternDefinitions.get(patternName) + if (patternDefinition != null) { + val condition = MatchUtil.generateIterativeCondition( + config, + inputSchema, + patternName, + patternNames, + patternDefinition, + inputTypeInfo) + + newPattern.where(condition) + } else { + newPattern + } + + case call: RexCall => + + call.getOperator match { + case PATTERN_CONCAT => + val left = call.operands.get(0) + val right = call.operands.get(1) + translatePattern(right, + translatePattern(left, currentPattern, patternNames), + patternNames) + // Quantifiers + case PATTERN_QUANTIFIER => + val name = call.operands.get(0).asInstanceOf[RexLiteral] + val newPattern = translatePattern(name, currentPattern, patternNames) + + val startNum = call.operands.get(1).asInstanceOf[RexLiteral] + .getValue3.asInstanceOf[JBigDecimal].intValue() + val endNum = call.operands.get(2).asInstanceOf[RexLiteral] + .getValue3.asInstanceOf[JBigDecimal].intValue() + + if (startNum == 0 && endNum == -1) { // zero or more + newPattern.oneOrMore().optional().consecutive() + } else if (startNum == 1 && endNum == -1) { // one or more + newPattern.oneOrMore().consecutive() + } else if (startNum == 0 && endNum == 1) { // optional + newPattern.optional() + } else if (endNum != -1) { // times + newPattern.times(startNum, endNum).consecutive() + } else { // times or more + newPattern.timesOrMore(startNum).consecutive() + } + + case PATTERN_ALTER => + throw TableException("Currently, CEP doesn't support branching patterns.") + + case PATTERN_PERMUTE => + throw TableException("Currently, CEP doesn't support PERMUTE patterns.") + + case PATTERN_EXCLUDE => + throw TableException("Currently, CEP doesn't support '{-' '-}' patterns.") + } + + case _ => + throw TableException("") + } + + val patternNames: ListBuffer[String] = ListBuffer() + val cepPattern = translatePattern(pattern, null, patternNames) + if (interval != null) { + val intervalLiteral = interval.asInstanceOf[RexLiteral] + val intervalValue = interval.asInstanceOf[RexLiteral].getValueAs(classOf[java.lang.Long]) + val intervalMs: Long = intervalLiteral.getTypeName match { + case INTERVAL_YEAR | INTERVAL_YEAR_MONTH | INTERVAL_MONTH => + // convert from months to milliseconds, suppose 1 month = 30 days + intervalValue * 30L * 24 * 3600 * 1000 + case _ => intervalValue + } + + cepPattern.within(Time.milliseconds(intervalMs)) + } + + val patternStream: PatternStream[Row] = CEP.pattern[Row](inputDS, cepPattern) + + val outTypeInfo = CRowTypeInfo(schema.typeInfo) + if (allRows) { + val patternFlatSelectFunction = + MatchUtil.generatePatternFlatSelectFunction( + config, + schema, + patternNames, + partitionKeys, + orderKeys, + measures, + inputTypeInfo) + patternStream.flatSelect[CRow](patternFlatSelectFunction, outTypeInfo) + } else { + val patternSelectFunction = + MatchUtil.generatePatternSelectFunction( + config, + schema, + patternNames, + partitionKeys, + measures, + inputTypeInfo) + patternStream.select[CRow](patternSelectFunction, outTypeInfo) + } + } + + private def next(currentPattern: Pattern[Row, Row], patternName: String): Pattern[Row, Row] = { + if (currentPattern == null) { + Pattern.begin(patternName) + } else { + currentPattern.next(patternName) + } + } +} diff --git a/core/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala b/core/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala new file mode 100644 index 000000000..693e4d314 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala @@ -0,0 +1,115 @@ +package org.apache.flink.table.plan.nodes.logical + +import java.util + +import org.apache.calcite.plan._ +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.convert.ConverterRule +import org.apache.calcite.rel.core.Match +import org.apache.calcite.rel.logical.LogicalMatch +import org.apache.calcite.rel.{RelCollation, RelNode} +import org.apache.calcite.rex.RexNode +import org.apache.flink.table.plan.nodes.FlinkConventions + + +class FlinkLogicalMatch( + cluster: RelOptCluster, + traitSet: RelTraitSet, + input: RelNode, + rowType: RelDataType, + pattern: RexNode, + strictStart: Boolean, + strictEnd: Boolean, + patternDefinitions: util.Map[String, RexNode], + measures: util.Map[String, RexNode], + after: RexNode, + subsets: util.Map[String, _ <: util.SortedSet[String]], + allRows: Boolean, + partitionKeys: util.List[RexNode], + orderKeys: RelCollation, + interval: RexNode) + extends Match( + cluster, + traitSet, + input, + rowType, + pattern, + strictStart, + strictEnd, + patternDefinitions, + measures, + after, + subsets, + allRows, + partitionKeys, + orderKeys, + interval) + with FlinkLogicalRel { + + override def copy( + input: RelNode, + rowType: RelDataType, + pattern: RexNode, + strictStart: Boolean, + strictEnd: Boolean, + patternDefinitions: util.Map[String, RexNode], + measures: util.Map[String, RexNode], + after: RexNode, + subsets: util.Map[String, _ <: util.SortedSet[String]], + allRows: Boolean, + partitionKeys: util.List[RexNode], + orderKeys: RelCollation, + interval: RexNode): Match = { + new FlinkLogicalMatch( + cluster, + traitSet, + input, + rowType, + pattern, + strictStart, + strictEnd, + patternDefinitions, + measures, + after, + subsets, + allRows, + partitionKeys, + orderKeys, + interval) + } +} + +private class FlinkLogicalMatchConverter + extends ConverterRule( + classOf[LogicalMatch], + Convention.NONE, + FlinkConventions.LOGICAL, + "FlinkLogicalMatchConverter") { + + override def convert(rel: RelNode): RelNode = { + val logicalMatch = rel.asInstanceOf[LogicalMatch] + val traitSet = rel.getTraitSet.replace(FlinkConventions.LOGICAL) + val newInput = RelOptRule.convert(logicalMatch.getInput, FlinkConventions.LOGICAL) + + new FlinkLogicalMatch( + rel.getCluster, + traitSet, + newInput, + logicalMatch.getRowType, + logicalMatch.getPattern, + logicalMatch.isStrictStart, + logicalMatch.isStrictEnd, + logicalMatch.getPatternDefinitions, + logicalMatch.getMeasures, + logicalMatch.getAfter, + logicalMatch.getSubsets, + logicalMatch.isAllRows, + logicalMatch.getPartitionKeys, + logicalMatch.getOrderKeys, + logicalMatch.getInterval) + } +} + +object FlinkLogicalMatch { + val CONVERTER: ConverterRule = new FlinkLogicalMatchConverter() +} diff --git a/core/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/core/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala new file mode 100644 index 000000000..5a0ff9d65 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala @@ -0,0 +1,209 @@ +package org.apache.flink.table.plan.rules + +import org.apache.calcite.rel.core.RelFactories +import org.apache.calcite.rel.rules._ +import org.apache.calcite.tools.{RuleSet, RuleSets} +import org.apache.flink.table.plan.rules.common._ +import org.apache.flink.table.plan.rules.logical._ +import org.apache.flink.table.plan.rules.dataSet._ +import org.apache.flink.table.plan.rules.datastream.{DataStreamMatchRule, _} +import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalMatch, _} + +object FlinkRuleSets { + + /** + * Convert sub-queries before query decorrelation. + */ + val TABLE_SUBQUERY_RULES: RuleSet = RuleSets.ofList( + SubQueryRemoveRule.FILTER, + SubQueryRemoveRule.PROJECT, + SubQueryRemoveRule.JOIN) + + /** + * Convert table references before query decorrelation. + */ + val TABLE_REF_RULES: RuleSet = RuleSets.ofList( + TableScanRule.INSTANCE, + EnumerableToLogicalTableScan.INSTANCE) + + val LOGICAL_OPT_RULES: RuleSet = RuleSets.ofList( + + // push a filter into a join + FilterJoinRule.FILTER_ON_JOIN, + // push filter into the children of a join + FilterJoinRule.JOIN, + // push filter through an aggregation + FilterAggregateTransposeRule.INSTANCE, + // push filter through set operation + FilterSetOpTransposeRule.INSTANCE, + // push project through set operation + ProjectSetOpTransposeRule.INSTANCE, + + // aggregation and projection rules + AggregateProjectMergeRule.INSTANCE, + AggregateProjectPullUpConstantsRule.INSTANCE, + // push a projection past a filter or vice versa + ProjectFilterTransposeRule.INSTANCE, + FilterProjectTransposeRule.INSTANCE, + // push a projection to the children of a join + // push all expressions to handle the time indicator correctly + new ProjectJoinTransposeRule(PushProjector.ExprCondition.FALSE, RelFactories.LOGICAL_BUILDER), + // merge projections + ProjectMergeRule.INSTANCE, + // remove identity project + ProjectRemoveRule.INSTANCE, + // reorder sort and projection + SortProjectTransposeRule.INSTANCE, + ProjectSortTransposeRule.INSTANCE, + + // join rules + JoinPushExpressionsRule.INSTANCE, + + // remove union with only a single child + UnionEliminatorRule.INSTANCE, + // convert non-all union into all-union + distinct + UnionToDistinctRule.INSTANCE, + + // remove aggregation if it does not aggregate and input is already distinct + AggregateRemoveRule.INSTANCE, + // push aggregate through join + AggregateJoinTransposeRule.EXTENDED, + // aggregate union rule + AggregateUnionAggregateRule.INSTANCE, + // expand distinct aggregate to normal aggregate with groupby + AggregateExpandDistinctAggregatesRule.JOIN, + + // reduce aggregate functions like AVG, STDDEV_POP etc. + AggregateReduceFunctionsRule.INSTANCE, + + // remove unnecessary sort rule + SortRemoveRule.INSTANCE, + + // prune empty results rules + PruneEmptyRules.AGGREGATE_INSTANCE, + PruneEmptyRules.FILTER_INSTANCE, + PruneEmptyRules.JOIN_LEFT_INSTANCE, + PruneEmptyRules.JOIN_RIGHT_INSTANCE, + PruneEmptyRules.PROJECT_INSTANCE, + PruneEmptyRules.SORT_INSTANCE, + PruneEmptyRules.UNION_INSTANCE, + + // calc rules + FilterCalcMergeRule.INSTANCE, + ProjectCalcMergeRule.INSTANCE, + FilterToCalcRule.INSTANCE, + ProjectToCalcRule.INSTANCE, + CalcMergeRule.INSTANCE, + + // scan optimization + PushProjectIntoTableSourceScanRule.INSTANCE, + PushFilterIntoTableSourceScanRule.INSTANCE, + + // unnest rule + LogicalUnnestRule.INSTANCE, + + // translate to flink logical rel nodes + FlinkLogicalAggregate.CONVERTER, + FlinkLogicalWindowAggregate.CONVERTER, + FlinkLogicalOverWindow.CONVERTER, + FlinkLogicalCalc.CONVERTER, + FlinkLogicalCorrelate.CONVERTER, + FlinkLogicalIntersect.CONVERTER, + FlinkLogicalJoin.CONVERTER, + FlinkLogicalMinus.CONVERTER, + FlinkLogicalSort.CONVERTER, + FlinkLogicalUnion.CONVERTER, + FlinkLogicalValues.CONVERTER, + FlinkLogicalTableSourceScan.CONVERTER, + FlinkLogicalTableFunctionScan.CONVERTER, + FlinkLogicalNativeTableScan.CONVERTER, + FlinkLogicalMatch.CONVERTER + ) + + + /** + * RuleSet to normalize plans for batch / DataSet execution + */ + val DATASET_NORM_RULES: RuleSet = RuleSets.ofList( + // simplify expressions rules + ReduceExpressionsRule.FILTER_INSTANCE, + ReduceExpressionsRule.PROJECT_INSTANCE, + ReduceExpressionsRule.CALC_INSTANCE, + ReduceExpressionsRule.JOIN_INSTANCE, + ProjectToWindowRule.PROJECT, + + // Transform grouping sets + DecomposeGroupingSetRule.INSTANCE, + // Transform window to LogicalWindowAggregate + DataSetLogicalWindowAggregateRule.INSTANCE, + WindowPropertiesRule.INSTANCE, + WindowPropertiesHavingRule.INSTANCE + ) + + /** + * RuleSet to optimize plans for batch / DataSet execution + */ + val DATASET_OPT_RULES: RuleSet = RuleSets.ofList( + // translate to Flink DataSet nodes + DataSetWindowAggregateRule.INSTANCE, + DataSetAggregateRule.INSTANCE, + DataSetDistinctRule.INSTANCE, + DataSetCalcRule.INSTANCE, + DataSetJoinRule.INSTANCE, + DataSetSingleRowJoinRule.INSTANCE, + DataSetScanRule.INSTANCE, + DataSetUnionRule.INSTANCE, + DataSetIntersectRule.INSTANCE, + DataSetMinusRule.INSTANCE, + DataSetSortRule.INSTANCE, + DataSetValuesRule.INSTANCE, + DataSetCorrelateRule.INSTANCE, + BatchTableSourceScanRule.INSTANCE + ) + + /** + * RuleSet to normalize plans for stream / DataStream execution + */ + val DATASTREAM_NORM_RULES: RuleSet = RuleSets.ofList( + // Transform window to LogicalWindowAggregate + DataStreamLogicalWindowAggregateRule.INSTANCE, + WindowPropertiesRule.INSTANCE, + WindowPropertiesHavingRule.INSTANCE, + + // simplify expressions rules + ReduceExpressionsRule.FILTER_INSTANCE, + ReduceExpressionsRule.PROJECT_INSTANCE, + ReduceExpressionsRule.CALC_INSTANCE, + ProjectToWindowRule.PROJECT + ) + + /** + * RuleSet to optimize plans for stream / DataStream execution + */ + val DATASTREAM_OPT_RULES: RuleSet = RuleSets.ofList( + // translate to DataStream nodes + DataStreamSortRule.INSTANCE, + DataStreamGroupAggregateRule.INSTANCE, + DataStreamOverAggregateRule.INSTANCE, + DataStreamGroupWindowAggregateRule.INSTANCE, + DataStreamCalcRule.INSTANCE, + DataStreamScanRule.INSTANCE, + DataStreamUnionRule.INSTANCE, + DataStreamValuesRule.INSTANCE, + DataStreamCorrelateRule.INSTANCE, + DataStreamWindowJoinRule.INSTANCE, + StreamTableSourceScanRule.INSTANCE, + DataStreamMatchRule.INSTANCE + ) + + /** + * RuleSet to decorate plans for stream / DataStream execution + */ + val DATASTREAM_DECO_RULES: RuleSet = RuleSets.ofList( + // retraction rules + DataStreamRetractionRules.DEFAULT_RETRACTION_INSTANCE, + DataStreamRetractionRules.UPDATES_AS_RETRACTION_INSTANCE, + DataStreamRetractionRules.ACCMODE_INSTANCE + ) + +} diff --git a/core/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala b/core/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala new file mode 100644 index 000000000..8bb052504 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala @@ -0,0 +1,46 @@ +package org.apache.flink.table.plan.rules.datastream + +import org.apache.calcite.plan.{RelOptRule, RelTraitSet} +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.convert.ConverterRule +import org.apache.flink.table.plan.nodes.FlinkConventions +import org.apache.flink.table.plan.nodes.datastream.DataStreamMatch +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalMatch +import org.apache.flink.table.plan.schema.RowSchema + +class DataStreamMatchRule + extends ConverterRule( + classOf[FlinkLogicalMatch], + FlinkConventions.LOGICAL, + FlinkConventions.DATASTREAM, + "DataStreamMatchRule") { + + override def convert(rel: RelNode): RelNode = { + val logicalMatch: FlinkLogicalMatch = rel.asInstanceOf[FlinkLogicalMatch] + val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASTREAM) + val convertInput: RelNode = + RelOptRule.convert(logicalMatch.getInput, FlinkConventions.DATASTREAM) + + new DataStreamMatch( + rel.getCluster, + traitSet, + convertInput, + logicalMatch.getPattern, + logicalMatch.isStrictStart, + logicalMatch.isStrictEnd, + logicalMatch.getPatternDefinitions, + logicalMatch.getMeasures, + logicalMatch.getAfter, + logicalMatch.getSubsets, + logicalMatch.isAllRows, + logicalMatch.getPartitionKeys, + logicalMatch.getOrderKeys, + logicalMatch.getInterval, + new RowSchema(logicalMatch.getRowType), + new RowSchema(logicalMatch.getInput.getRowType)) + } +} + +object DataStreamMatchRule { + val INSTANCE: RelOptRule = new DataStreamMatchRule +} diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/ConvertToRow.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/ConvertToRow.scala new file mode 100644 index 000000000..cfaf5fe81 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/ConvertToRow.scala @@ -0,0 +1,14 @@ +package org.apache.flink.table.runtime.cepmatch + +import org.apache.flink.api.common.functions.MapFunction +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.types.Row + +/** + * MapFunction convert CRow to Row. + */ +class ConvertToRow extends MapFunction[CRow, Row] { + override def map(value: CRow): Row = { + value.row + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/IterativeConditionRunner.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/IterativeConditionRunner.scala new file mode 100644 index 000000000..04a365ebb --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/IterativeConditionRunner.scala @@ -0,0 +1,40 @@ +package org.apache.flink.table.runtime.cepmatch + +import org.apache.flink.cep.pattern.conditions.IterativeCondition +import org.apache.flink.table.codegen.Compiler +import org.apache.flink.types.Row +import org.slf4j.LoggerFactory + +/** + * IterativeConditionRunner with [[Row]] value. + */ +class IterativeConditionRunner( + name: String, + code: String) + extends IterativeCondition[Row] + with Compiler[IterativeCondition[Row]]{ + + val LOG = LoggerFactory.getLogger(this.getClass) + + // IterativeCondition will be serialized as part of state, + // so make function as transient to avoid ClassNotFoundException when restore state, + // see FLINK-6939 for details + @transient private var function: IterativeCondition[Row] = _ + + def init(): Unit = { + LOG.debug(s"Compiling IterativeCondition: $name \n\n Code:\n$code") + // We cannot get user's classloader currently, see FLINK-6938 for details + val clazz = compile(Thread.currentThread().getContextClassLoader, name, code) + LOG.debug("Instantiating IterativeCondition.") + function = clazz.newInstance() + } + + override def filter(value: Row, ctx: IterativeCondition.Context[Row]): Boolean = { + + if (function == null) { + init() + } + + function.filter(value, ctx) + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/MatchUtil.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/MatchUtil.scala new file mode 100644 index 000000000..e0af24e93 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/MatchUtil.scala @@ -0,0 +1,99 @@ +package org.apache.flink.table.runtime.cepmatch + +import java.util + +import org.apache.calcite.rel.RelCollation +import org.apache.calcite.rex.RexNode +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.cep.{PatternFlatSelectFunction, PatternSelectFunction} +import org.apache.flink.cep.pattern.conditions.IterativeCondition +import org.apache.flink.table.api.TableConfig +import org.apache.flink.table.codegen.MatchCodeGenerator +import org.apache.flink.table.plan.schema.RowSchema +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.types.Row + +/** + * An util class to generate match functions. + * 1。IterativeCondition + * 2。PatternSelectFunction + * 3。PatternFlatSelectFunction + */ +object MatchUtil { + + private[flink] def generateIterativeCondition( + config: TableConfig, + inputType: RowSchema, + patternName: String, + patternNames: Seq[String], + patternDefinition: RexNode, + inputTypeInfo: TypeInformation[_]): IterativeCondition[Row] = { + + val generator = new MatchCodeGenerator( + config, false, inputTypeInfo, patternNames, true, Some(patternName)) + val condition = generator.generateExpression(patternDefinition) + val body = + s""" + |${condition.code} + |return ${condition.resultTerm}; + |""".stripMargin + + val genCondition = generator.generateIterativeCondition("MatchRecognizeCondition", body) + new IterativeConditionRunner(genCondition.name, genCondition.code) + } + + private[flink] def generatePatternSelectFunction( + config: TableConfig, + returnType: RowSchema, + patternNames: Seq[String], + partitionKeys: util.List[RexNode], + measures: util.Map[String, RexNode], + inputTypeInfo: TypeInformation[_]): PatternSelectFunction[Row, CRow] = { + + val generator = new MatchCodeGenerator(config, false, inputTypeInfo, patternNames, false) + + val resultExpression = generator.generateSelectOutputExpression( + partitionKeys, + measures, + returnType) + val body = + s""" + |${resultExpression.code} + |return ${resultExpression.resultTerm}; + |""".stripMargin + + generator.addReusableStatements() + val genFunction = generator.generatePatternSelectFunction( + "MatchRecognizePatternSelectFunction", + body) + new PatternSelectFunctionRunner(genFunction.name, genFunction.code) + } + + private[flink] def generatePatternFlatSelectFunction( + config: TableConfig, + returnType: RowSchema, + patternNames: Seq[String], + partitionKeys: util.List[RexNode], + orderKeys: RelCollation, + measures: util.Map[String, RexNode], + inputTypeInfo: TypeInformation[_]): PatternFlatSelectFunction[Row, CRow] = { + + val generator = new MatchCodeGenerator(config, false, inputTypeInfo, patternNames, false) + + val resultExpression = generator.generateFlatSelectOutputExpression( + partitionKeys, + orderKeys, + measures, + returnType) + val body = + s""" + |${resultExpression.code} + |""".stripMargin + + generator.addReusableStatements() + val genFunction = generator.generatePatternFlatSelectFunction( + "MatchRecognizePatternFlatSelectFunction", + body) + new PatternFlatSelectFunctionRunner(genFunction.name, genFunction.code) + } +} diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternFlatSelectFunctionRunner.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternFlatSelectFunctionRunner.scala new file mode 100644 index 000000000..f72296324 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternFlatSelectFunctionRunner.scala @@ -0,0 +1,47 @@ +package org.apache.flink.table.runtime.cepmatch + +import java.util + +import org.apache.flink.cep.PatternFlatSelectFunction +import org.apache.flink.table.codegen.Compiler +import org.apache.flink.table.runtime.CRowWrappingCollector +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.slf4j.LoggerFactory + +/** + * PatternFlatSelectFunctionRunner with [[Row]] input and [[CRow]] output. + */ +class PatternFlatSelectFunctionRunner( + name: String, + code: String) + extends PatternFlatSelectFunction[Row, CRow] + with Compiler[PatternFlatSelectFunction[Row, Row]] { + + val LOG = LoggerFactory.getLogger(this.getClass) + + private var cRowWrapper: CRowWrappingCollector = _ + + private var function: PatternFlatSelectFunction[Row, Row] = _ + + def init(): Unit = { + LOG.debug(s"Compiling PatternFlatSelectFunction: $name \n\n Code:\n$code") + val clazz = compile(Thread.currentThread().getContextClassLoader, name, code) + LOG.debug("Instantiating PatternFlatSelectFunction.") + function = clazz.newInstance() + + this.cRowWrapper = new CRowWrappingCollector() + } + + override def flatSelect( + pattern: util.Map[String, util.List[Row]], + out: Collector[CRow]): Unit = { + if (function == null) { + init() + } + + cRowWrapper.out = out + function.flatSelect(pattern, cRowWrapper) + } +} diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternSelectFunctionRunner.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternSelectFunctionRunner.scala new file mode 100644 index 000000000..d84c9e61d --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternSelectFunctionRunner.scala @@ -0,0 +1,45 @@ +package org.apache.flink.table.runtime.cepmatch + +import java.util + +import org.apache.flink.cep.PatternSelectFunction +import org.apache.flink.table.codegen.Compiler +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.types.Row +import org.slf4j.LoggerFactory + +/** + * PatternSelectFunctionRunner with [[Row]] input and [[CRow]] output. + */ +class PatternSelectFunctionRunner( + name: String, + code: String) + extends PatternSelectFunction[Row, CRow] + with Compiler[PatternSelectFunction[Row, Row]] { + + val LOG = LoggerFactory.getLogger(this.getClass) + + private var outCRow: CRow = _ + + private var function: PatternSelectFunction[Row, Row] = _ + + def init(): Unit = { + LOG.debug(s"Compiling PatternSelectFunction: $name \n\n Code:\n$code") + val clazz = compile(Thread.currentThread().getContextClassLoader, name, code) + LOG.debug("Instantiating PatternSelectFunction.") + function = clazz.newInstance() + } + + override def select(pattern: util.Map[String, util.List[Row]]): CRow = { + if (outCRow == null) { + outCRow = new CRow(null, true) + } + + if (function == null) { + init() + } + + outCRow.row = function.select(pattern) + outCRow + } +} diff --git a/core/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala b/core/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala new file mode 100644 index 000000000..f89c41d71 --- /dev/null +++ b/core/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala @@ -0,0 +1,534 @@ +package org.apache.flink.table.validate + +import org.apache.calcite.sql.`type`.{OperandTypes, ReturnTypes, SqlTypeTransforms} +import org.apache.calcite.sql.fun.SqlStdOperatorTable +import org.apache.calcite.sql.util.{ChainedSqlOperatorTable, ListSqlOperatorTable, ReflectiveSqlOperatorTable} +import org.apache.calcite.sql._ +import org.apache.flink.table.api._ +import org.apache.flink.table.expressions._ +import org.apache.flink.table.functions.sql.ScalarSqlFunctions +import org.apache.flink.table.functions.utils.{AggSqlFunction, ScalarSqlFunction, TableSqlFunction} +import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction} + +import _root_.scala.collection.JavaConversions._ +import _root_.scala.collection.mutable +import _root_.scala.util.{Failure, Success, Try} + +/** + * A catalog for looking up (user-defined) functions, used during validation phases + * of both Table API and SQL API. + */ +class FunctionCatalog { + + private val functionBuilders = mutable.HashMap.empty[String, Class[_]] + private val sqlFunctions = mutable.ListBuffer[SqlFunction]() + + def registerFunction(name: String, builder: Class[_]): Unit = + functionBuilders.put(name.toLowerCase, builder) + + def registerSqlFunction(sqlFunction: SqlFunction): Unit = { + sqlFunctions --= sqlFunctions.filter(_.getName == sqlFunction.getName) + sqlFunctions += sqlFunction + } + + def getUserDefinedFunctions: Seq[String] = { + sqlFunctions.map(_.getName) + } + + def getSqlOperatorTable: SqlOperatorTable = + ChainedSqlOperatorTable.of( + new BasicOperatorTable(), + new ListSqlOperatorTable(sqlFunctions) + ) + + /** + * Lookup and create an expression if we find a match. + */ + def lookupFunction(name: String, children: Seq[Expression]): Expression = { + val funcClass = functionBuilders + .getOrElse(name.toLowerCase, throw ValidationException(s"Undefined function: $name")) + + // Instantiate a function using the provided `children` + funcClass match { + + // user-defined scalar function call + case sf if classOf[ScalarFunction].isAssignableFrom(sf) => + val scalarSqlFunction = sqlFunctions + .find(f => f.getName.equalsIgnoreCase(name) && f.isInstanceOf[ScalarSqlFunction]) + .getOrElse(throw ValidationException(s"Undefined scalar function: $name")) + .asInstanceOf[ScalarSqlFunction] + ScalarFunctionCall(scalarSqlFunction.getScalarFunction, children) + + // user-defined table function call + case tf if classOf[TableFunction[_]].isAssignableFrom(tf) => + val tableSqlFunction = sqlFunctions + .find(f => f.getName.equalsIgnoreCase(name) && f.isInstanceOf[TableSqlFunction]) + .getOrElse(throw ValidationException(s"Undefined table function: $name")) + .asInstanceOf[TableSqlFunction] + val typeInfo = tableSqlFunction.getRowTypeInfo + val function = tableSqlFunction.getTableFunction + TableFunctionCall(name, function, children, typeInfo) + + // user-defined aggregate function call + case af if classOf[AggregateFunction[_, _]].isAssignableFrom(af) => + val aggregateFunction = sqlFunctions + .find(f => f.getName.equalsIgnoreCase(name) && f.isInstanceOf[AggSqlFunction]) + .getOrElse(throw ValidationException(s"Undefined table function: $name")) + .asInstanceOf[AggSqlFunction] + val function = aggregateFunction.getFunction + val returnType = aggregateFunction.returnType + val accType = aggregateFunction.accType + AggFunctionCall(function, returnType, accType, children) + + // general expression call + case expression if classOf[Expression].isAssignableFrom(expression) => + // try to find a constructor accepts `Seq[Expression]` + Try(funcClass.getDeclaredConstructor(classOf[Seq[_]])) match { + case Success(seqCtor) => + Try(seqCtor.newInstance(children).asInstanceOf[Expression]) match { + case Success(expr) => expr + case Failure(e) => throw new ValidationException(e.getMessage) + } + case Failure(_) => + Try(funcClass.getDeclaredConstructor(classOf[Expression], classOf[Seq[_]])) match { + case Success(ctor) => + Try(ctor.newInstance(children.head, children.tail).asInstanceOf[Expression]) match { + case Success(expr) => expr + case Failure(e) => throw new ValidationException(e.getMessage) + } + case Failure(_) => + val childrenClass = Seq.fill(children.length)(classOf[Expression]) + // try to find a constructor matching the exact number of children + Try(funcClass.getDeclaredConstructor(childrenClass: _*)) match { + case Success(ctor) => + Try(ctor.newInstance(children: _*).asInstanceOf[Expression]) match { + case Success(expr) => expr + case Failure(exception) => throw ValidationException(exception.getMessage) + } + case Failure(_) => + throw ValidationException( + s"Invalid number of arguments for function $funcClass") + } + } + } + case _ => + throw ValidationException("Unsupported function.") + } + } + + /** + * Drop a function and return if the function existed. + */ + def dropFunction(name: String): Boolean = + functionBuilders.remove(name.toLowerCase).isDefined + + /** + * Drop all registered functions. + */ + def clear(): Unit = functionBuilders.clear() +} + +object FunctionCatalog { + + val builtInFunctions: Map[String, Class[_]] = Map( + + // logic + "and" -> classOf[And], + "or" -> classOf[Or], + "not" -> classOf[Not], + "equals" -> classOf[EqualTo], + "greaterThan" -> classOf[GreaterThan], + "greaterThanOrEqual" -> classOf[GreaterThanOrEqual], + "lessThan" -> classOf[LessThan], + "lessThanOrEqual" -> classOf[LessThanOrEqual], + "notEquals" -> classOf[NotEqualTo], + "in" -> classOf[In], + "isNull" -> classOf[IsNull], + "isNotNull" -> classOf[IsNotNull], + "isTrue" -> classOf[IsTrue], + "isFalse" -> classOf[IsFalse], + "isNotTrue" -> classOf[IsNotTrue], + "isNotFalse" -> classOf[IsNotFalse], + "if" -> classOf[If], + + // aggregate functions + "avg" -> classOf[Avg], + "count" -> classOf[Count], + "max" -> classOf[Max], + "min" -> classOf[Min], + "sum" -> classOf[Sum], + "sum0" -> classOf[Sum0], + "stddevPop" -> classOf[StddevPop], + "stddevSamp" -> classOf[StddevSamp], + "varPop" -> classOf[VarPop], + "varSamp" -> classOf[VarSamp], + "collect" -> classOf[Collect], + + // string functions + "charLength" -> classOf[CharLength], + "initCap" -> classOf[InitCap], + "like" -> classOf[Like], + "concat" -> classOf[Plus], + "lower" -> classOf[Lower], + "lowerCase" -> classOf[Lower], + "similar" -> classOf[Similar], + "substring" -> classOf[Substring], + "trim" -> classOf[Trim], + "upper" -> classOf[Upper], + "upperCase" -> classOf[Upper], + "position" -> classOf[Position], + "overlay" -> classOf[Overlay], + "concat" -> classOf[Concat], + "concat_ws" -> classOf[ConcatWs], + "lpad" -> classOf[Lpad], + "rpad" -> classOf[Rpad], + + // math functions + "plus" -> classOf[Plus], + "minus" -> classOf[Minus], + "divide" -> classOf[Div], + "times" -> classOf[Mul], + "abs" -> classOf[Abs], + "ceil" -> classOf[Ceil], + "exp" -> classOf[Exp], + "floor" -> classOf[Floor], + "log10" -> classOf[Log10], + "ln" -> classOf[Ln], + "power" -> classOf[Power], + "mod" -> classOf[Mod], + "sqrt" -> classOf[Sqrt], + "minusPrefix" -> classOf[UnaryMinus], + "sin" -> classOf[Sin], + "cos" -> classOf[Cos], + "tan" -> classOf[Tan], + "cot" -> classOf[Cot], + "asin" -> classOf[Asin], + "acos" -> classOf[Acos], + "atan" -> classOf[Atan], + "degrees" -> classOf[Degrees], + "radians" -> classOf[Radians], + "sign" -> classOf[Sign], + "round" -> classOf[Round], + "pi" -> classOf[Pi], + "e" -> classOf[E], + "rand" -> classOf[Rand], + "randInteger" -> classOf[RandInteger], + "bin" -> classOf[Bin], + + // temporal functions + "extract" -> classOf[Extract], + "currentDate" -> classOf[CurrentDate], + "currentTime" -> classOf[CurrentTime], + "currentTimestamp" -> classOf[CurrentTimestamp], + "localTime" -> classOf[LocalTime], + "localTimestamp" -> classOf[LocalTimestamp], + "quarter" -> classOf[Quarter], + "temporalOverlaps" -> classOf[TemporalOverlaps], + "dateTimePlus" -> classOf[Plus], + "dateFormat" -> classOf[DateFormat], + + // item + "at" -> classOf[ItemAt], + + // cardinality + "cardinality" -> classOf[Cardinality], + + // array + "array" -> classOf[ArrayConstructor], + "element" -> classOf[ArrayElement], + + // map + "map" -> classOf[MapConstructor], + + // row + "row" -> classOf[RowConstructor], + + // window properties + "start" -> classOf[WindowStart], + "end" -> classOf[WindowEnd], + + // ordering + "asc" -> classOf[Asc], + "desc" -> classOf[Desc], + + // crypto hash + "md5" -> classOf[Md5], + "sha1" -> classOf[Sha1], + "sha256" -> classOf[Sha256] + ) + + /** + * Create a new function catalog with built-in functions. + */ + def withBuiltIns: FunctionCatalog = { + val catalog = new FunctionCatalog() + builtInFunctions.foreach { case (n, c) => catalog.registerFunction(n, c) } + catalog + } +} + +class BasicOperatorTable extends ReflectiveSqlOperatorTable { + + /** + * List of supported SQL operators / functions. + * + * This list should be kept in sync with [[SqlStdOperatorTable]]. + */ + private val builtInSqlOperators: Seq[SqlOperator] = Seq( + // SET OPERATORS + SqlStdOperatorTable.UNION, + SqlStdOperatorTable.UNION_ALL, + SqlStdOperatorTable.EXCEPT, + SqlStdOperatorTable.EXCEPT_ALL, + SqlStdOperatorTable.INTERSECT, + SqlStdOperatorTable.INTERSECT_ALL, + // BINARY OPERATORS + SqlStdOperatorTable.AND, + SqlStdOperatorTable.AS, + SqlStdOperatorTable.CONCAT, + SqlStdOperatorTable.DIVIDE, + SqlStdOperatorTable.DIVIDE_INTEGER, + SqlStdOperatorTable.DOT, + SqlStdOperatorTable.EQUALS, + SqlStdOperatorTable.GREATER_THAN, + SqlStdOperatorTable.IS_DISTINCT_FROM, + SqlStdOperatorTable.IS_NOT_DISTINCT_FROM, + SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, + SqlStdOperatorTable.LESS_THAN, + SqlStdOperatorTable.LESS_THAN_OR_EQUAL, + SqlStdOperatorTable.MINUS, + SqlStdOperatorTable.MULTIPLY, + SqlStdOperatorTable.NOT_EQUALS, + SqlStdOperatorTable.OR, + SqlStdOperatorTable.PLUS, + SqlStdOperatorTable.DATETIME_PLUS, + // POSTFIX OPERATORS + SqlStdOperatorTable.DESC, + SqlStdOperatorTable.NULLS_FIRST, + SqlStdOperatorTable.IS_NOT_NULL, + SqlStdOperatorTable.IS_NULL, + SqlStdOperatorTable.IS_NOT_TRUE, + SqlStdOperatorTable.IS_TRUE, + SqlStdOperatorTable.IS_NOT_FALSE, + SqlStdOperatorTable.IS_FALSE, + SqlStdOperatorTable.IS_NOT_UNKNOWN, + SqlStdOperatorTable.IS_UNKNOWN, + // PREFIX OPERATORS + SqlStdOperatorTable.NOT, + SqlStdOperatorTable.UNARY_MINUS, + SqlStdOperatorTable.UNARY_PLUS, + // GROUPING FUNCTIONS + SqlStdOperatorTable.GROUP_ID, + SqlStdOperatorTable.GROUPING, + SqlStdOperatorTable.GROUPING_ID, + // AGGREGATE OPERATORS + SqlStdOperatorTable.SUM, + SqlStdOperatorTable.SUM0, + SqlStdOperatorTable.COUNT, + SqlStdOperatorTable.COLLECT, + SqlStdOperatorTable.MIN, + SqlStdOperatorTable.MAX, + SqlStdOperatorTable.AVG, + SqlStdOperatorTable.STDDEV_POP, + SqlStdOperatorTable.STDDEV_SAMP, + SqlStdOperatorTable.VAR_POP, + SqlStdOperatorTable.VAR_SAMP, + // ARRAY OPERATORS + SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR, + SqlStdOperatorTable.ELEMENT, + // MAP OPERATORS + SqlStdOperatorTable.MAP_VALUE_CONSTRUCTOR, + // ARRAY MAP SHARED OPERATORS + SqlStdOperatorTable.ITEM, + SqlStdOperatorTable.CARDINALITY, + // SPECIAL OPERATORS + SqlStdOperatorTable.ROW, + SqlStdOperatorTable.OVERLAPS, + SqlStdOperatorTable.LITERAL_CHAIN, + SqlStdOperatorTable.BETWEEN, + SqlStdOperatorTable.SYMMETRIC_BETWEEN, + SqlStdOperatorTable.NOT_BETWEEN, + SqlStdOperatorTable.SYMMETRIC_NOT_BETWEEN, + SqlStdOperatorTable.NOT_LIKE, + SqlStdOperatorTable.LIKE, + SqlStdOperatorTable.NOT_SIMILAR_TO, + SqlStdOperatorTable.SIMILAR_TO, + SqlStdOperatorTable.CASE, + SqlStdOperatorTable.REINTERPRET, + SqlStdOperatorTable.EXTRACT, + SqlStdOperatorTable.IN, + // FUNCTIONS + SqlStdOperatorTable.SUBSTRING, + SqlStdOperatorTable.OVERLAY, + SqlStdOperatorTable.TRIM, + SqlStdOperatorTable.POSITION, + SqlStdOperatorTable.CHAR_LENGTH, + SqlStdOperatorTable.CHARACTER_LENGTH, + SqlStdOperatorTable.UPPER, + SqlStdOperatorTable.LOWER, + SqlStdOperatorTable.INITCAP, + SqlStdOperatorTable.POWER, + SqlStdOperatorTable.SQRT, + SqlStdOperatorTable.MOD, + SqlStdOperatorTable.LN, + SqlStdOperatorTable.LOG10, + SqlStdOperatorTable.ABS, + SqlStdOperatorTable.EXP, + SqlStdOperatorTable.NULLIF, + SqlStdOperatorTable.COALESCE, + SqlStdOperatorTable.FLOOR, + SqlStdOperatorTable.CEIL, + SqlStdOperatorTable.LOCALTIME, + SqlStdOperatorTable.LOCALTIMESTAMP, + SqlStdOperatorTable.CURRENT_TIME, + SqlStdOperatorTable.CURRENT_TIMESTAMP, + SqlStdOperatorTable.CURRENT_DATE, + ScalarSqlFunctions.DATE_FORMAT, + SqlStdOperatorTable.CAST, + SqlStdOperatorTable.EXTRACT, + SqlStdOperatorTable.QUARTER, + SqlStdOperatorTable.SCALAR_QUERY, + SqlStdOperatorTable.EXISTS, + SqlStdOperatorTable.SIN, + SqlStdOperatorTable.COS, + SqlStdOperatorTable.TAN, + SqlStdOperatorTable.COT, + SqlStdOperatorTable.ASIN, + SqlStdOperatorTable.ACOS, + SqlStdOperatorTable.ATAN, + SqlStdOperatorTable.DEGREES, + SqlStdOperatorTable.RADIANS, + SqlStdOperatorTable.SIGN, + SqlStdOperatorTable.ROUND, + SqlStdOperatorTable.PI, + ScalarSqlFunctions.E, + SqlStdOperatorTable.RAND, + SqlStdOperatorTable.RAND_INTEGER, + ScalarSqlFunctions.CONCAT, + ScalarSqlFunctions.CONCAT_WS, + ScalarSqlFunctions.BIN, + SqlStdOperatorTable.TIMESTAMP_ADD, + ScalarSqlFunctions.LOG, + ScalarSqlFunctions.LPAD, + ScalarSqlFunctions.RPAD, + ScalarSqlFunctions.MD5, + ScalarSqlFunctions.SHA1, + ScalarSqlFunctions.SHA256, + + // EXTENSIONS + BasicOperatorTable.TUMBLE, + BasicOperatorTable.HOP, + BasicOperatorTable.SESSION, + BasicOperatorTable.TUMBLE_START, + BasicOperatorTable.TUMBLE_END, + BasicOperatorTable.HOP_START, + BasicOperatorTable.HOP_END, + BasicOperatorTable.SESSION_START, + BasicOperatorTable.SESSION_END, + BasicOperatorTable.TUMBLE_PROCTIME, + BasicOperatorTable.TUMBLE_ROWTIME, + BasicOperatorTable.HOP_PROCTIME, + BasicOperatorTable.HOP_ROWTIME, + BasicOperatorTable.SESSION_PROCTIME, + BasicOperatorTable.SESSION_ROWTIME, + + // MATCH_RECOGNIZE + SqlStdOperatorTable.FIRST, + SqlStdOperatorTable.LAST, + SqlStdOperatorTable.PREV, + SqlStdOperatorTable.NEXT, + SqlStdOperatorTable.CLASSIFIER, + SqlStdOperatorTable.MATCH_NUMBER, + SqlStdOperatorTable.FINAL, + SqlStdOperatorTable.RUNNING + ) + + builtInSqlOperators.foreach(register) +} + +object BasicOperatorTable { + + /** + * We need custom group auxiliary functions in order to support nested windows. + */ + + val TUMBLE: SqlGroupedWindowFunction = new SqlGroupedWindowFunction( + SqlKind.TUMBLE, + null, + OperandTypes.or(OperandTypes.DATETIME_INTERVAL, OperandTypes.DATETIME_INTERVAL_TIME)) { + override def getAuxiliaryFunctions: _root_.java.util.List[SqlGroupedWindowFunction] = + Seq( + TUMBLE_START, + TUMBLE_END, + TUMBLE_ROWTIME, + TUMBLE_PROCTIME) + } + val TUMBLE_START: SqlGroupedWindowFunction = TUMBLE.auxiliary(SqlKind.TUMBLE_START) + val TUMBLE_END: SqlGroupedWindowFunction = TUMBLE.auxiliary(SqlKind.TUMBLE_END) + val TUMBLE_ROWTIME: SqlGroupedWindowFunction = + new SqlGroupedWindowFunction( + "TUMBLE_ROWTIME", + SqlKind.OTHER_FUNCTION, + TUMBLE, + // ensure that returned rowtime is always NOT_NULLABLE + ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NOT_NULLABLE), + null, + TUMBLE.getOperandTypeChecker, + SqlFunctionCategory.SYSTEM) + val TUMBLE_PROCTIME: SqlGroupedWindowFunction = + TUMBLE.auxiliary("TUMBLE_PROCTIME", SqlKind.OTHER_FUNCTION) + + val HOP: SqlGroupedWindowFunction = new SqlGroupedWindowFunction( + SqlKind.HOP, + null, + OperandTypes.or( + OperandTypes.DATETIME_INTERVAL_INTERVAL, + OperandTypes.DATETIME_INTERVAL_INTERVAL_TIME)) { + override def getAuxiliaryFunctions: _root_.java.util.List[SqlGroupedWindowFunction] = + Seq( + HOP_START, + HOP_END, + HOP_ROWTIME, + HOP_PROCTIME) + } + val HOP_START: SqlGroupedWindowFunction = HOP.auxiliary(SqlKind.HOP_START) + val HOP_END: SqlGroupedWindowFunction = HOP.auxiliary(SqlKind.HOP_END) + val HOP_ROWTIME: SqlGroupedWindowFunction = + new SqlGroupedWindowFunction( + "HOP_ROWTIME", + SqlKind.OTHER_FUNCTION, + HOP, + // ensure that returned rowtime is always NOT_NULLABLE + ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NOT_NULLABLE), + null, + HOP.getOperandTypeChecker, + SqlFunctionCategory.SYSTEM) + val HOP_PROCTIME: SqlGroupedWindowFunction = HOP.auxiliary("HOP_PROCTIME", SqlKind.OTHER_FUNCTION) + + val SESSION: SqlGroupedWindowFunction = new SqlGroupedWindowFunction( + SqlKind.SESSION, + null, + OperandTypes.or(OperandTypes.DATETIME_INTERVAL, OperandTypes.DATETIME_INTERVAL_TIME)) { + override def getAuxiliaryFunctions: _root_.java.util.List[SqlGroupedWindowFunction] = + Seq( + SESSION_START, + SESSION_END, + SESSION_ROWTIME, + SESSION_PROCTIME) + } + val SESSION_START: SqlGroupedWindowFunction = SESSION.auxiliary(SqlKind.SESSION_START) + val SESSION_END: SqlGroupedWindowFunction = SESSION.auxiliary(SqlKind.SESSION_END) + val SESSION_ROWTIME: SqlGroupedWindowFunction = + new SqlGroupedWindowFunction( + "SESSION_ROWTIME", + SqlKind.OTHER_FUNCTION, + SESSION, + // ensure that returned rowtime is always NOT_NULLABLE + ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NOT_NULLABLE), + null, + SESSION.getOperandTypeChecker, + SqlFunctionCategory.SYSTEM) + val SESSION_PROCTIME: SqlGroupedWindowFunction = + SESSION.auxiliary("SESSION_PROCTIME", SqlKind.OTHER_FUNCTION) + +} diff --git a/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java b/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java index e29277fee..9771f6dd7 100644 --- a/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java +++ b/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java @@ -39,6 +39,51 @@ public class SideSqlExecTest { + /** + * 参考阿里Blink的cep sql语法,文档https://help.aliyun.com/document_detail/73845.html?spm=a2c4g.11186623.6.637.5cba27efFHjOSs + * @throws Exception + */ + @Test + public void testCepSql() throws Exception { + List paramList = Lists.newArrayList(); + paramList.add("-sql"); + String sqlContext = "CREATE table source(" + + "name varchar, " + + "price float, " + + "tax float, " + + "tstamp timestamp) " + + "with (" + + " type = 'kafka09',bootstrapServers = 'kudu1:9092',zookeeperQuorum = '172.16.8.107:2181/kafka', offsetReset = 'latest',topic = 'tranflow_input',parallelism = '1' " + + ");" + + "CREATE table sink(" + + "start_tstamp timestamp, " + + "bottom_tstamp timestamp, " + + "end_tstamp timestamp, " + + "bottom_total float, " + + "end_total float ) " + + "with (" + + " type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/bank_test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 'max_deposit_acct_base',cache = 'LRU',cacheSize = '10000',cacheTTLMs = '60000',parallelism = '1' " + + ");" + + "insert into sink " + + "select * from source " + + "MATCH_RECOGNIZE (\n" + + " MEASURES\n" + + " STRT.tstamp AS start_tstamp,\n" + + " LAST(DOWN.tstamp) AS bottom_tstamp,\n" + + " LAST(UP.tstamp) AS end_tstamp,\n" + + " FIRST(DOWN.price + DOWN.tax + 1) AS bottom_total,\n" + + " FIRST(UP.price + UP.tax) AS end_total" + + " ONE ROW PER MATCH\n" + + " PATTERN (STRT DOWN+ UP+)\n" + + " DEFINE\n" + + " DOWN AS DOWN.price < PREV(DOWN.price),\n" + + " UP AS UP.price > PREV(UP.price) AND UP.tax > LAST(DOWN.tax)\n" + + ") AS T" + ; + test(sqlContext); + } + + @Test public void testRunSideSql() throws Exception { //String runParam = "-sql CREATE+TABLE+MyTable(channel+STRING%2c+pv+INT%2c+xctime+bigint%2c+timeLeng+as+CHARACTER_LENGTH(channel)%2c++WATERMARK+FOR+xctime+AS+withOffset(xctime%2c+1000))+WITH+(+type%3d%27kafka09%27%2c+bootstrapServers%3d%27172.16.8.198%3a9092%27%2c+offsetReset%3d%27latest%27%2ctopic%3d%27nbTest1%27)%3bCREATE+TABLE+MyResult(channel+STRING%2c+pv+INT)+WITH+(+type%3d%27mysql%27%2c+url%3d%27jdbc%3amysql%3a%2f%2f172.16.8.104%3a3306%2ftest%3fcharset%3dutf8%27%2cuserName%3d%27dtstack%27%2cpassword%3d%27abc123%27%2c+tableName%3d%27pv%27)%3bcreate+table+sideTable(channel+String%2c+count+int%2c+PERIOD+FOR+SYSTEM_TIME)+WITH+(+type%3d%27mysql%27%2c+url%3d%27jdbc%3amysql%3a%2f%2f172.16.8.104%3a3306%2ftest%3fcharset%3dutf8%27%2cuserName%3d%27dtstack%27%2cpassword%3d%27abc123%27%2c+tableName%3d%27pv%27)%3binsert+into+MyResult+select+a.channel%2cb.pv+from+MyTable+a+join+sideTable+b+on+a.channel%3db.channel%3b -name xc -localSqlPluginPath D:\\gitspace\\flink-sql-plugin\\plugins -mode local -remoteSqlPluginPath /opt/dtstack/flinkplugin -confProp %7b%22time.characteristic%22%3a%22EventTime%22%7d -addjar %5b%22D%3a%5c%5cgitspace%5c%5crdos-execution-engine%5c%5c..%5c%5ctmp140%5c%5cflink14Test-1.0-SNAPSHOT.jar%22%5d"; @@ -228,7 +273,8 @@ public void test(String sql) throws Exception { paramList.add("-name"); paramList.add("xc"); paramList.add("-localSqlPluginPath"); - paramList.add("D:\\gitspace\\flinkStreamSQL\\plugins"); +// paramList.add("D:\\gitspace\\flinkStreamSQL\\plugins"); + paramList.add("/Users/meitu/code/github/flinkStreamSQL/plugins"); paramList.add("-mode"); paramList.add("local"); paramList.add("-addjar"); diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java index 26ab2da73..c418360cb 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java @@ -32,6 +32,7 @@ import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; +import java.sql.Timestamp; import java.sql.Types; import java.util.List; @@ -113,6 +114,8 @@ protected void buildSqlTypes(List fieldTypeArray){ tmpFieldsType[i] = Types.BINARY; }else if(fieldType.equals(Float.class.getName()) || fieldType.equals(Double.class.getName())){ tmpFieldsType[i] = Types.DOUBLE; + }else if(fieldType.equals(Timestamp.class.getName())){ + tmpFieldsType[i] = Types.TIMESTAMP; }else{ throw new RuntimeException("no support field type for sql. the input type:" + fieldType); } From 03980b56f1b04decb5da3c6edc3b65b399800606 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 12 Nov 2018 21:07:06 +0800 Subject: [PATCH 117/470] kafka10,11 source metric register name --- core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java | 2 +- .../java/com/dtstack/flink/sql/source/kafka/KafkaSource.java | 5 ++++- .../java/com/dtstack/flink/sql/source/kafka/KafkaSource.java | 5 ++++- 3 files changed, 9 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java index b8940ffaf..181f6de5c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java @@ -101,7 +101,7 @@ public static void openCheckpoint(StreamExecutionEnvironment env, Properties pro } String cleanupModeStr = properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_CLEANUPMODE_KEY); - if(cleanupModeStr != null){//设置在cancle job情况下checkpoint是否被保存 + if(cleanupModeStr != null){//设置在cancel job情况下checkpoint是否被保存 if("true".equalsIgnoreCase(cleanupModeStr)){ env.getCheckpointConfig().enableExternalizedCheckpoints( CheckpointConfig.ExternalizedCheckpointCleanup.DELETE_ON_CANCELLATION); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index e884dcc37..ffc42f7ef 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -43,6 +43,8 @@ public class KafkaSource implements IStreamSourceGener
{ + private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; + /** * Get kafka data source, you need to provide the data field names, data types * If you do not specify auto.offset.reset, the default use groupoffset @@ -78,6 +80,7 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } String fields = StringUtils.join(kafka010SourceTableInfo.getFields(), ","); - return tableEnv.fromDataStream(env.addSource(kafkaSrc, typeInformation), fields); + String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); + return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); } } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 4c9acd1de..8bfec8107 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -43,6 +43,8 @@ public class KafkaSource implements IStreamSourceGener
{ + private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; + /** * Get kafka data source, you need to provide the data field names, data types * If you do not specify auto.offset.reset, the default use groupoffset @@ -78,6 +80,7 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } String fields = StringUtils.join(kafka011SourceTableInfo.getFields(), ","); - return tableEnv.fromDataStream(env.addSource(kafkaSrc, typeInformation), fields); + String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); + return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); } } From 0f7372ba539fd49e85e5929c2db658d1269bc669 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Mon, 12 Nov 2018 22:08:44 +0800 Subject: [PATCH 118/470] support sidetable of redis --- redis5/redis5-side/pom.xml | 5 + redis5/redis5-side/redis-all-side/pom.xml | 73 +++++- .../flink/sql/side/redis/RedisAllReqRow.java | 223 +++++++++++++++++- .../sql/side/redis/RedisAllSideInfo.java | 18 +- redis5/redis5-side/redis-side-core/pom.xml | 13 +- .../sql/side/redis/table/RedisSideParser.java | 9 +- .../side/redis/table/RedisSideTableInfo.java | 18 +- .../sql/sink/redis/table/RedisTableInfo.java | 2 +- 8 files changed, 350 insertions(+), 11 deletions(-) diff --git a/redis5/redis5-side/pom.xml b/redis5/redis5-side/pom.xml index 6f98376c9..3629735f7 100644 --- a/redis5/redis5-side/pom.xml +++ b/redis5/redis5-side/pom.xml @@ -9,6 +9,11 @@ sql.core 1.0-SNAPSHOT + + redis.clients + jedis + 2.8.0 + diff --git a/redis5/redis5-side/redis-all-side/pom.xml b/redis5/redis5-side/redis-all-side/pom.xml index 5dc304135..5d78be9f5 100644 --- a/redis5/redis5-side/redis-all-side/pom.xml +++ b/redis5/redis5-side/redis-all-side/pom.xml @@ -3,6 +3,13 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 + + + com.dtstack.flink + sql.side.redis.core + 1.0-SNAPSHOT + + com.dtstack.flink @@ -11,8 +18,72 @@ com.dtstack.flink - redis-all-side + sql.side.all.redis + redis-all-side 1.0-SNAPSHOT + jar + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java index 2b09e518b..85a5eb393 100644 --- a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java @@ -18,5 +18,226 @@ package com.dtstack.flink.sql.side.redis; -public class RedisAllReqRow { +import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.redis.table.RedisSideTableInfo; +import org.apache.calcite.sql.JoinType; +import org.apache.commons.pool2.impl.GenericObjectPoolConfig; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import redis.clients.jedis.HostAndPort; +import redis.clients.jedis.Jedis; +import redis.clients.jedis.JedisPool; +import redis.clients.jedis.JedisSentinelPool; + +import java.sql.SQLException; +import java.sql.Timestamp; +import java.util.*; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +public class RedisAllReqRow extends AllReqRow { + + private static final Logger LOG = LoggerFactory.getLogger(RedisAllReqRow.class); + + private static final int CONN_RETRY_NUM = 3; + + private static final int TIMEOUT = 1000; + + private JedisPool pool; + + private JedisSentinelPool jedisSentinelPool; + + private RedisSideTableInfo tableInfo; + + private AtomicReference>> cacheRef = new AtomicReference<>(); + + public RedisAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new RedisAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + @Override + protected Row fillData(Row input, Object sideInput) { + Map sideInputMap = (Map) sideInput; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ + obj = ((Timestamp)obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ + if(sideInputMap == null){ + row.setField(entry.getKey(), null); + }else{ + String key = sideInfo.getSideFieldNameIndex().get(entry.getKey()); + row.setField(entry.getKey(), sideInputMap.get(key)); + } + } + + return row; + } + + @Override + protected void initCache() throws SQLException { + tableInfo = (RedisSideTableInfo) sideInfo.getSideTableInfo(); + Map> newCache = Maps.newConcurrentMap(); + cacheRef.set(newCache); + loadData(newCache); + } + + @Override + protected void reloadCache() { + Map> newCache = Maps.newConcurrentMap(); + try { + loadData(newCache); + } catch (SQLException e) { + LOG.error("", e); + } + + cacheRef.set(newCache); + LOG.info("----- Redis all cacheRef reload end:{}", Calendar.getInstance()); + } + + @Override + public void flatMap(Row row, Collector out) throws Exception { + Map inputParams = Maps.newHashMap(); + for(Integer conValIndex : sideInfo.getEqualValIndex()){ + Object equalObj = row.getField(conValIndex); + if(equalObj == null){ + out.collect(null); + } + String columnName = sideInfo.getEqualFieldList().get(conValIndex); + inputParams.put(columnName, (String) equalObj); + } + String key = buildKey(inputParams); + + Map cacheMap = cacheRef.get().get(key); + + if (cacheMap == null){ + if(sideInfo.getJoinType() == JoinType.LEFT){ + Row data = fillData(row, null); + out.collect(data); + }else{ + return; + } + + return; + } + + Row newRow = fillData(row, cacheMap); + out.collect(newRow); + } + + private String buildKey(Map inputParams) { + String tableName = tableInfo.getTableName(); + StringBuilder key = new StringBuilder(); + for (int i=0; i> tmpCache) throws SQLException { + Jedis jedis = null; + + try { + for(int i=0; i keys = jedis.keys(perKey); + List newPerKeys = new LinkedList<>(); + for (String key : keys){ + String[] splitKey = key.split(":"); + String newKey = splitKey[0] + splitKey[1] + splitKey[2]; + newPerKeys.add(newKey); + } + List list = newPerKeys.stream().distinct().collect(Collectors.toList()); + for(String key : list){ + Map kv = Maps.newHashMap(); + String[] primaryKv = key.split(":"); + kv.put(primaryKv[1], primaryKv[2]); + + String pattern = key + "*"; + Set realKeys = jedis.keys(pattern); + for (String realKey : realKeys){ + kv.put(realKey.split(":")[3], jedis.get(realKey)); + } + tmpCache.put(key, kv); + } + + + } catch (Exception e){ + LOG.error("", e); + } finally { + if (jedis != null){ + jedis.close(); + } + if (jedisSentinelPool != null) { + jedisSentinelPool.close(); + } + if (pool != null) { + pool.close(); + } + } + } + + private void toCache(Map keyValue, Map>> tmpCache) { + Set setKey = keyValue.keySet(); + for (String key : setKey){ + String[] keys = key.split(":"); + String newKey = keys[0] + keys[1] + keys[2]; + } + + } + + private Jedis getJedis(String url, String password, String database){ + GenericObjectPoolConfig poolConfig = new GenericObjectPoolConfig(); + String[] nodes = url.split(","); + if (nodes.length > 1){ + //cluster + Set addresses = new HashSet<>(); + Set ipPorts = new HashSet<>(); + for (String ipPort : nodes) { + ipPorts.add(ipPort); + String[] ipPortPair = ipPort.split(":"); + addresses.add(new HostAndPort(ipPortPair[0].trim(), Integer.valueOf(ipPortPair[1].trim()))); + } + jedisSentinelPool = new JedisSentinelPool("Master", ipPorts, poolConfig, TIMEOUT, password, Integer.parseInt(database)); + return jedisSentinelPool.getResource(); + } else { + String[] ipPortPair = nodes[0].split(":"); + String ip = ipPortPair[0]; + String port = ipPortPair[1]; + pool = new JedisPool(poolConfig, ip, Integer.parseInt(port), TIMEOUT, password, Integer.parseInt(database)); + return pool.getResource(); + } + } } diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java index 7b15278f9..2f4fbd52c 100644 --- a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java @@ -18,5 +18,21 @@ package com.dtstack.flink.sql.side.redis; -public class RedisAllSideInfo { +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; + +import java.util.List; + +public class RedisAllSideInfo extends SideInfo { + public RedisAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + + } } diff --git a/redis5/redis5-side/redis-side-core/pom.xml b/redis5/redis5-side/redis-side-core/pom.xml index 5876c2e48..9f5f346b7 100644 --- a/redis5/redis5-side/redis-side-core/pom.xml +++ b/redis5/redis5-side/redis-side-core/pom.xml @@ -10,9 +10,16 @@ 1.0-SNAPSHOT - com.dtstack.flink - redis-side-core - 1.0-SNAPSHOT + sql.side.redis.core + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + jar \ No newline at end of file diff --git a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java index ae3f2abe0..d9a56d892 100644 --- a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java +++ b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java @@ -20,6 +20,7 @@ import com.dtstack.flink.sql.table.AbsSideTableParser; import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; import java.util.regex.Matcher; @@ -39,7 +40,13 @@ public class RedisSideParser extends AbsSideTableParser { @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { RedisSideTableInfo redisSideTableInfo = new RedisSideTableInfo(); - + redisSideTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, redisSideTableInfo); + parseCacheProp(redisSideTableInfo, props); + redisSideTableInfo.setUrl(MathUtil.getString(props.get(RedisSideTableInfo.URL_KEY))); + redisSideTableInfo.setPassword(MathUtil.getString(props.get(RedisSideTableInfo.PASSWORD_KEY))); + redisSideTableInfo.setDatabase(MathUtil.getString(props.get(RedisSideTableInfo.DATABASE_KEY))); + redisSideTableInfo.setTableName(MathUtil.getString(props.get(RedisSideTableInfo.TABLE_KEY))); return redisSideTableInfo; } diff --git a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideTableInfo.java b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideTableInfo.java index b82d142ef..3846e574e 100644 --- a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideTableInfo.java +++ b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideTableInfo.java @@ -29,14 +29,18 @@ public class RedisSideTableInfo extends SideTableInfo { public static final String URL_KEY = "url"; - public static final String USER_NAME_KEY = "database"; - public static final String PASSWORD_KEY = "password"; + public static final String DATABASE_KEY = "database"; + + public static final String TABLE_KEY = "table"; + private String url; private String database; + private String table; + private String password; public RedisSideTableInfo(){ @@ -48,7 +52,7 @@ public RedisSideTableInfo(){ public boolean check() { Preconditions.checkNotNull(url, "redis of URL is required"); Preconditions.checkNotNull(database, "redis of database is required"); - Preconditions.checkNotNull(password, "redis of password is required"); + //Preconditions.checkNotNull(password, "redis of password is required"); return true; } @@ -68,6 +72,14 @@ public void setDatabase(String database) { this.database = database; } + public String getTableName(){ + return table; + } + + public void setTableName(String table){ + this.table = table; + } + public String getPassword() { return password; } diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java index 1190610a1..eab0ee79b 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java @@ -81,7 +81,7 @@ public void setTimeout(int timeout) { public boolean check() { Preconditions.checkNotNull(url, "redis field of URL is required"); Preconditions.checkNotNull(database, "redis field of database is required"); - Preconditions.checkNotNull(password, "redis field of password is required"); + //Preconditions.checkNotNull(password, "redis field of password is required"); return true; } From 91cb0971a7dee2f9f1cd3e16a6d0f18be88ab6a1 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 13 Nov 2018 14:55:11 +0800 Subject: [PATCH 119/470] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E8=84=8F=E6=95=B0?= =?UTF-8?q?=E6=8D=AE=E8=BF=94=E5=9B=9E=E5=80=BC=E4=B8=BAnull?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/source/kafka/CustomerJsonDeserialization.java | 2 +- .../flink/sql/source/kafka/CustomerJsonDeserialization.java | 2 +- .../flink/sql/source/kafka/CustomerJsonDeserialization.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 9deb04b94..ec369b28f 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -124,7 +124,7 @@ public Row deserialize(byte[] message) throws IOException { } catch (Throwable t) { //add metric of dirty data dirtyDataCounter.inc(); - return new Row(fieldNames.length); + return null; } } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 4abcaf8c8..bcb54159a 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -124,7 +124,7 @@ public Row deserialize(byte[] message) throws IOException { } catch (Throwable t) { //add metric of dirty data dirtyDataCounter.inc(); - return new Row(fieldNames.length); + return null; } } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 09350bfc1..78f1c17ff 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -126,7 +126,7 @@ public Row deserialize(byte[] message) throws IOException { } catch (Throwable t) { //add metric of dirty data dirtyDataCounter.inc(); - return new Row(fieldNames.length); + return null; } } From da44f8787d8e0ddc0de72f6a88ead650a83d42b1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Tue, 13 Nov 2018 22:12:55 +0800 Subject: [PATCH 120/470] add redis side table --- .../flink/sql/side/redis/RedisAllReqRow.java | 4 +- .../sql/side/redis/RedisAllSideInfo.java | 3 + redis5/redis5-side/redis-async-side/pom.xml | 81 +++++++++- .../sql/side/redis/RedisAsyncReqRow.java | 139 +++++++++++++++++- .../sql/side/redis/RedisAsyncSideInfo.java | 40 ++++- 5 files changed, 261 insertions(+), 6 deletions(-) diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java index 85a5eb393..b6fa5f987 100644 --- a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java @@ -40,7 +40,9 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; -public class RedisAllReqRow extends AllReqRow { +public class RedisAllReqRow extends AllReqRow{ + + private static final long serialVersionUID = 7578879189085344807L; private static final Logger LOG = LoggerFactory.getLogger(RedisAllReqRow.class); diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java index 2f4fbd52c..4bcb77def 100644 --- a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java @@ -27,6 +27,9 @@ import java.util.List; public class RedisAllSideInfo extends SideInfo { + + private static final long serialVersionUID = 1998703966487857613L; + public RedisAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/redis5/redis5-side/redis-async-side/pom.xml b/redis5/redis5-side/redis-async-side/pom.xml index 1f1fcc921..d0e83c441 100644 --- a/redis5/redis5-side/redis-async-side/pom.xml +++ b/redis5/redis5-side/redis-async-side/pom.xml @@ -3,16 +3,91 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 + + + com.dtstack.flink + sql.side.redis.core + 1.0-SNAPSHOT + + + io.lettuce + lettuce-core + 5.0.5.RELEASE + + com.dtstack.flink redis5-side 1.0-SNAPSHOT + sql.side.async.redis - com.dtstack.flink - redis-async-side - 1.0-SNAPSHOT + redis-async-side + + jar + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index 946149a67..74c9b1b98 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -18,5 +18,142 @@ package com.dtstack.flink.sql.side.redis; -public class RedisAsyncReqRow { +import com.dtstack.flink.sql.enums.ECacheContentType; +import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.cache.CacheObj; +import com.dtstack.flink.sql.side.redis.table.RedisSideTableInfo; +import io.lettuce.core.RedisClient; +import io.lettuce.core.api.StatefulRedisConnection; +import io.lettuce.core.api.async.RedisAsyncCommands; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; + +import java.sql.Timestamp; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class RedisAsyncReqRow extends AsyncReqRow { + + private static final long serialVersionUID = -2079908694523987738L; + + private RedisClient redisClient; + + private StatefulRedisConnection connection; + + private RedisAsyncCommands async; + + private RedisSideTableInfo redisSideTableInfo; + + + public RedisAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new RedisAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + redisSideTableInfo = (RedisSideTableInfo) sideInfo.getSideTableInfo(); + StringBuilder uri = new StringBuilder(); + // TODO: 2018/11/13 根据redis模式,拼接uri + redisClient = RedisClient.create(uri.toString()); + connection = redisClient.connect(); + async = connection.async(); + } + + @Override + protected Row fillData(Row input, Object sideInput) { + Map keyValue = (Map) sideInput; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ + obj = ((Timestamp)obj).getTime(); + } + + row.setField(entry.getKey(), obj); + } + + // TODO: 2018/11/13 插入维表数据 + for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ + if(keyValue == null){ + row.setField(entry.getKey(), null); + }else{ + row.setField(entry.getKey(), keyValue.get(entry.getValue())); + } + } + + return row; + } + + @Override + public void asyncInvoke(Row input, ResultFuture resultFuture) throws Exception { + List keyData = Lists.newLinkedList(); + for (int i = 0; i < sideInfo.getEqualValIndex().size(); i++) { + Integer conValIndex = sideInfo.getEqualValIndex().get(i); + Object equalObj = input.getField(conValIndex); + if(equalObj == null){ + resultFuture.complete(null); + } + + keyData.add(sideInfo.getEqualFieldList().get(i)); + keyData.add((String) equalObj); + } + + String key = buildCacheKey(keyData); + + if(openCache()){ + CacheObj val = getFromCache(key); + if(val != null){ + + if(ECacheContentType.MissVal == val.getType()){ + dealMissKey(input, resultFuture); + return; + }else if(ECacheContentType.MultiLine == val.getType()){ + + for(Object jsonArray : (List)val.getContent()){ + Row row = fillData(input, jsonArray); + resultFuture.complete(Collections.singleton(row)); + } + + }else{ + throw new RuntimeException("not support cache obj type " + val.getType()); + } + return; + } + } + + // TODO: 2018/11/13 异步实现并缓存 + Map keyValue = Maps.newConcurrentMap(); + List keyList = async.keys(key).get(); + for (String aKey : keyList){ + keyValue.put(aKey, async.get(aKey).get()); + } + Row row = fillData(input, keyValue); + resultFuture.complete(Collections.singleton(row)); + + } + + private String buildCacheKey(List keyData) { + String kv = String.join(":", keyData); + String tableName = redisSideTableInfo.getTableName(); + StringBuilder preKey = new StringBuilder(); + preKey.append(tableName).append(":").append(kv); + return preKey.toString(); + } + + @Override + public void close() throws Exception { + super.close(); + connection.close(); + redisClient.shutdown(); + } + } diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java index 6a89c884c..abd12714a 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java @@ -18,5 +18,43 @@ package com.dtstack.flink.sql.side.redis; -public class RedisAsyncSideInfo { +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.redis.table.RedisSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.List; + +public class RedisAsyncSideInfo extends SideInfo { + private static final long serialVersionUID = -4851348392924455039L; + + public RedisAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + RedisSideTableInfo redisSideTableInfo = (RedisSideTableInfo) sideTableInfo; + + String sideTableName = joinInfo.getSideTableName(); + + SqlNode conditionNode = joinInfo.getCondition(); + + List sqlNodeList = Lists.newArrayList(); + if(conditionNode.getKind() == SqlKind.AND){ + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); + }else{ + sqlNodeList.add(conditionNode); + } + + for(SqlNode sqlNode : sqlNodeList){ + dealOneEqualCon(sqlNode, sideTableName); + } + } } From 5bc15673e5ed1c49f69c5c0e6fe58688a70bdda4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Wed, 14 Nov 2018 16:41:01 +0800 Subject: [PATCH 121/470] add redis async function --- redis5/redis5-side/pom.xml | 17 +++--- .../flink/sql/side/redis/RedisAllReqRow.java | 9 ---- .../sql/side/redis/RedisAsyncReqRow.java | 53 +++++++++++++------ redis5/redis5-side/redis-side-core/pom.xml | 3 +- 4 files changed, 46 insertions(+), 36 deletions(-) diff --git a/redis5/redis5-side/pom.xml b/redis5/redis5-side/pom.xml index 3629735f7..b350a62cc 100644 --- a/redis5/redis5-side/pom.xml +++ b/redis5/redis5-side/pom.xml @@ -2,6 +2,12 @@ + + sql.redis + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + 4.0.0 @@ -16,15 +22,8 @@ - - com.dtstack.flink - sql.redis - 1.0-SNAPSHOT - - - com.dtstack.flink - redis5-side - 1.0-SNAPSHOT + sql.side.redis + redis-side redis-side-core diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java index b6fa5f987..bc74843d6 100644 --- a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java @@ -211,15 +211,6 @@ private void loadData(Map> tmpCache) throws SQLExcep } } - private void toCache(Map keyValue, Map>> tmpCache) { - Set setKey = keyValue.keySet(); - for (String key : setKey){ - String[] keys = key.split(":"); - String newKey = keys[0] + keys[1] + keys[2]; - } - - } - private Jedis getJedis(String url, String password, String database){ GenericObjectPoolConfig poolConfig = new GenericObjectPoolConfig(); String[] nodes = url.split(","); diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index 74c9b1b98..cd612e7d1 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -37,6 +37,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.function.Consumer; public class RedisAsyncReqRow extends AsyncReqRow { @@ -60,7 +61,15 @@ public void open(Configuration parameters) throws Exception { super.open(parameters); redisSideTableInfo = (RedisSideTableInfo) sideInfo.getSideTableInfo(); StringBuilder uri = new StringBuilder(); - // TODO: 2018/11/13 根据redis模式,拼接uri + String url = redisSideTableInfo.getUrl(); + String password = redisSideTableInfo.getPassword(); + String database = redisSideTableInfo.getDatabase(); + if (url.split(",").length > 1){ + uri.append("redis-sentinel://").append(password).append("@") + .append(url).append("/").append(database).append("#").append(url.split(",")[0]); + } else { + uri.append("redis://").append(password).append("@").append(url).append("/").append(database); + } redisClient = RedisClient.create(uri.toString()); connection = redisClient.connect(); async = connection.async(); @@ -117,12 +126,8 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except dealMissKey(input, resultFuture); return; }else if(ECacheContentType.MultiLine == val.getType()){ - - for(Object jsonArray : (List)val.getContent()){ - Row row = fillData(input, jsonArray); - resultFuture.complete(Collections.singleton(row)); - } - + Row row = fillData(input, val.getContent()); + resultFuture.complete(Collections.singleton(row)); }else{ throw new RuntimeException("not support cache obj type " + val.getType()); } @@ -130,14 +135,24 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except } } - // TODO: 2018/11/13 异步实现并缓存 - Map keyValue = Maps.newConcurrentMap(); - List keyList = async.keys(key).get(); - for (String aKey : keyList){ - keyValue.put(aKey, async.get(aKey).get()); - } - Row row = fillData(input, keyValue); - resultFuture.complete(Collections.singleton(row)); + async.keys(key).whenComplete((value, e)-> { + for (String newKey : value){ + String[] splitKey = newKey.split(":"); + Map keyValue = Maps.newConcurrentMap(); + keyValue.put(splitKey[1], splitKey[2]); + async.get(newKey).thenAccept(new Consumer() { + @Override + public void accept(String s) { + keyValue.put(splitKey[3], s); + } + }); + Row row = fillData(input, keyValue); + resultFuture.complete(Collections.singleton(row)); + if(openCache()){ + putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, keyValue)); + } + } + }); } @@ -152,8 +167,12 @@ private String buildCacheKey(List keyData) { @Override public void close() throws Exception { super.close(); - connection.close(); - redisClient.shutdown(); + if (connection != null){ + connection.close(); + } + if (redisClient != null){ + redisClient.shutdown(); + } } } diff --git a/redis5/redis5-side/redis-side-core/pom.xml b/redis5/redis5-side/redis-side-core/pom.xml index 9f5f346b7..7fb45a291 100644 --- a/redis5/redis5-side/redis-side-core/pom.xml +++ b/redis5/redis5-side/redis-side-core/pom.xml @@ -5,9 +5,10 @@ 4.0.0 + sql.side.redis com.dtstack.flink - redis5-side 1.0-SNAPSHOT + ../pom.xml sql.side.redis.core From 1cfa3e36ca5f8a92fb7a28d1bcfc0e23d02bd06f Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 14 Nov 2018 20:29:39 +0800 Subject: [PATCH 122/470] redis pom.xml error --- redis5/redis5-side/pom.xml | 4 ++-- redis5/redis5-side/redis-all-side/pom.xml | 21 +++++++++----------- redis5/redis5-side/redis-async-side/pom.xml | 22 +++++++++------------ 3 files changed, 20 insertions(+), 27 deletions(-) diff --git a/redis5/redis5-side/pom.xml b/redis5/redis5-side/pom.xml index b350a62cc..c81f1c8c3 100644 --- a/redis5/redis5-side/pom.xml +++ b/redis5/redis5-side/pom.xml @@ -9,6 +9,8 @@ ../pom.xml 4.0.0 + sql.side.redis + redis-side com.dtstack.flink @@ -22,8 +24,6 @@ - sql.side.redis - redis-side redis-side-core diff --git a/redis5/redis5-side/redis-all-side/pom.xml b/redis5/redis5-side/redis-all-side/pom.xml index 5d78be9f5..7dd752b96 100644 --- a/redis5/redis5-side/redis-all-side/pom.xml +++ b/redis5/redis5-side/redis-all-side/pom.xml @@ -2,7 +2,16 @@ + + com.dtstack.flink + sql.side.redis + 1.0-SNAPSHOT + ../pom.xml + 4.0.0 + sql.side.all.redis + redis-all-side + jar com.dtstack.flink @@ -11,18 +20,6 @@ - - com.dtstack.flink - redis5-side - 1.0-SNAPSHOT - - - com.dtstack.flink - sql.side.all.redis - redis-all-side - 1.0-SNAPSHOT - jar - diff --git a/redis5/redis5-side/redis-async-side/pom.xml b/redis5/redis5-side/redis-async-side/pom.xml index d0e83c441..1aaacd4dc 100644 --- a/redis5/redis5-side/redis-async-side/pom.xml +++ b/redis5/redis5-side/redis-async-side/pom.xml @@ -2,6 +2,15 @@ + + com.dtstack.flink + sql.side.redis + 1.0-SNAPSHOT + ../pom.xml + + sql.side.async.redis + redis-async-side + jar 4.0.0 @@ -16,17 +25,6 @@ - - com.dtstack.flink - redis5-side - 1.0-SNAPSHOT - - sql.side.async.redis - - redis-async-side - - jar - @@ -88,6 +86,4 @@ - - \ No newline at end of file From 5df17370030d413b9e1cc4546c1c956ea2dcae51 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Wed, 14 Nov 2018 22:18:46 +0800 Subject: [PATCH 123/470] fix redis table bugs --- .../flink/sql/side/redis/RedisAllReqRow.java | 11 ++++------ .../sql/side/redis/RedisAllSideInfo.java | 18 ++++++++++++++++- .../sql/side/redis/RedisAsyncReqRow.java | 2 +- .../sql/side/redis/RedisAsyncSideInfo.java | 2 -- .../side/redis/table/RedisSideTableInfo.java | 11 +++++----- .../sql/sink/redis/RedisOutputFormat.java | 20 ++++++++++++++----- .../flink/sql/sink/redis/RedisSink.java | 4 ++++ .../sql/sink/redis/table/RedisSinkParser.java | 2 ++ .../sql/sink/redis/table/RedisTableInfo.java | 13 +++++++++++- 9 files changed, 61 insertions(+), 22 deletions(-) diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java index bc74843d6..5b9ebbe10 100644 --- a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java @@ -29,10 +29,7 @@ import org.apache.flink.util.Collector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import redis.clients.jedis.HostAndPort; -import redis.clients.jedis.Jedis; -import redis.clients.jedis.JedisPool; -import redis.clients.jedis.JedisSentinelPool; +import redis.clients.jedis.*; import java.sql.SQLException; import java.sql.Timestamp; @@ -48,7 +45,7 @@ public class RedisAllReqRow extends AllReqRow{ private static final int CONN_RETRY_NUM = 3; - private static final int TIMEOUT = 1000; + private static final int TIMEOUT = 10000; private JedisPool pool; @@ -178,7 +175,7 @@ private void loadData(Map> tmpCache) throws SQLExcep List newPerKeys = new LinkedList<>(); for (String key : keys){ String[] splitKey = key.split(":"); - String newKey = splitKey[0] + splitKey[1] + splitKey[2]; + String newKey = splitKey[0] + ":" + splitKey[1] + ":" + splitKey[2]; newPerKeys.add(newKey); } List list = newPerKeys.stream().distinct().collect(Collectors.toList()); @@ -212,7 +209,7 @@ private void loadData(Map> tmpCache) throws SQLExcep } private Jedis getJedis(String url, String password, String database){ - GenericObjectPoolConfig poolConfig = new GenericObjectPoolConfig(); + JedisPoolConfig poolConfig = new JedisPoolConfig(); String[] nodes = url.split(","); if (nodes.length > 1){ //cluster diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java index 4bcb77def..29564a46d 100644 --- a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java @@ -22,7 +22,11 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; import java.util.List; @@ -36,6 +40,18 @@ public RedisAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List sqlNodeList = Lists.newArrayList(); + if(conditionNode.getKind() == SqlKind.AND){ + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); + }else{ + sqlNodeList.add(conditionNode); + } + + for(SqlNode sqlNode : sqlNodeList){ + dealOneEqualCon(sqlNode, sideTableName); + } } } diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index cd612e7d1..61a10ad8f 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -161,7 +161,7 @@ private String buildCacheKey(List keyData) { String tableName = redisSideTableInfo.getTableName(); StringBuilder preKey = new StringBuilder(); preKey.append(tableName).append(":").append(kv); - return preKey.toString(); + return preKey.toString().toLowerCase(); } @Override diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java index abd12714a..a5cf0fcaf 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java @@ -40,8 +40,6 @@ public RedisAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List { private String database; + private String tableName; + private String password; protected String[] fieldNames; @@ -104,7 +106,7 @@ public void writeRecord(Tuple2 record) throws IOException { for (String primaryKey : primaryKeys){ for (int i=0; i, IStreamSinkGener< protected String database; + protected String tableName; + protected String password; protected List primaryKeys; @@ -60,6 +62,7 @@ public RedisSink genStreamSink(TargetTableInfo targetTableInfo) { this.url = redisTableInfo.getUrl(); this.database = redisTableInfo.getDatabase(); this.password = redisTableInfo.getPassword(); + this.tableName = redisTableInfo.getTablename(); this.primaryKeys = targetTableInfo.getPrimaryKeys(); return this; } @@ -74,6 +77,7 @@ public void emitDataStream(DataStream> dataStream) { RedisOutputFormat.RedisOutputFormatBuilder builder = RedisOutputFormat.buildRedisOutputFormat(); builder.setUrl(this.url) .setDatabase(this.database) + .setTableName(this.tableName) .setPassword(this.password) .setFieldNames(this.fieldNames) .setFieldTypes(this.fieldTypes) diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java index 8890471dd..1bd74eb65 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java @@ -28,10 +28,12 @@ public class RedisSinkParser extends AbsTableParser { @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { RedisTableInfo redisTableInfo = new RedisTableInfo(); + redisTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, redisTableInfo); redisTableInfo.setUrl(MathUtil.getString(props.get(RedisTableInfo.URL_KEY))); redisTableInfo.setDatabase(MathUtil.getString(props.get(RedisTableInfo.DATABASE_KEY))); redisTableInfo.setPassword(MathUtil.getString(props.get(RedisTableInfo.PASSWORD_KEY))); + redisTableInfo.setTablename(MathUtil.getString(props.get(RedisTableInfo.TABLENAME_KEY))); if (props.get(RedisTableInfo.TIMEOUT) != null){ redisTableInfo.setTimeout(Integer.parseInt(MathUtil.getString(props.get(RedisTableInfo.TIMEOUT)))); } diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java index eab0ee79b..65beae1e5 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java @@ -31,6 +31,8 @@ public class RedisTableInfo extends TargetTableInfo { public static final String PASSWORD_KEY = "password"; + public static final String TABLENAME_KEY = "tablename"; + public static final String TIMEOUT = "timeout"; public RedisTableInfo(){ @@ -41,6 +43,8 @@ public RedisTableInfo(){ private String database; + private String tableName; + private String password; private int timeout = 1000; @@ -81,7 +85,7 @@ public void setTimeout(int timeout) { public boolean check() { Preconditions.checkNotNull(url, "redis field of URL is required"); Preconditions.checkNotNull(database, "redis field of database is required"); - //Preconditions.checkNotNull(password, "redis field of password is required"); + Preconditions.checkNotNull(password, "redis field of password is required"); return true; } @@ -90,4 +94,11 @@ public String getType() { return super.getType().toLowerCase(); } + public String getTablename() { + return tableName; + } + + public void setTablename(String tablename) { + this.tableName = tablename; + } } From 810c297fbf4510266dabfef62adbc1ae40676ef9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Thu, 15 Nov 2018 10:41:58 +0800 Subject: [PATCH 124/470] add docs --- docs/createView.md | 0 docs/redisSide.md | 0 docs/redisSink.md | 0 3 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 docs/createView.md create mode 100644 docs/redisSide.md create mode 100644 docs/redisSink.md diff --git a/docs/createView.md b/docs/createView.md new file mode 100644 index 000000000..e69de29bb diff --git a/docs/redisSide.md b/docs/redisSide.md new file mode 100644 index 000000000..e69de29bb diff --git a/docs/redisSink.md b/docs/redisSink.md new file mode 100644 index 000000000..e69de29bb From fa4f493b9a24b9ed8db050823a111c0531ac0c11 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Thu, 15 Nov 2018 10:49:03 +0800 Subject: [PATCH 125/470] add docs --- README.md | 10 ++--- docs/createView.md | 103 +++++++++++++++++++++++++++++++++++++++++++++ docs/redisSide.md | 70 ++++++++++++++++++++++++++++++ docs/redisSink.md | 57 +++++++++++++++++++++++++ 4 files changed, 235 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index d53277157..3241a57a5 100644 --- a/README.md +++ b/README.md @@ -1,18 +1,16 @@ # flinkStreamSQL > * 基于开源的flink,对其实时sql进行扩展 > > * 自定义create table 语法(包括源表,输出表,维表) +> > * 自定义create view 语法 > > * 自定义create function 语法 > > * 实现了流与维表的join # 已支持 * 源表:kafka 0.9,1.x版本 - * 维表:mysql,hbase,mongo - * 结果表:mysql,hbase,elasticsearch5.x,mongo + * 维表:mysql,hbase,mongo,redis + * 结果表:mysql,hbase,elasticsearch5.x,mongo,redis # 后续开发计划 - * 增加全局缓存功能 - * 增加临时表功能 - * 增加redis维表,结果表功能 * 增加oracle维表,结果表功能 * 增加SQlServer维表,结果表功能 * 增加kafka结果表功能 @@ -130,11 +128,13 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * [hbase 结果表插件](docs/hbaseSink.md) * [mysql 结果表插件](docs/mysqlSink.md) * [mongo 结果表插件](docs/mongoSink.md) +* [redis 结果表插件](docs/redisSink.md) ### 2.3 维表插件 * [hbase 维表插件](docs/hbaseSide.md) * [mysql 维表插件](docs/mysqlSide.md) * [mongo 维表插件](docs/mongoSide.md) +* [redis 维表插件](docs/redisSide.md) ## 3 样例 diff --git a/docs/createView.md b/docs/createView.md index e69de29bb..6f81fa8cb 100644 --- a/docs/createView.md +++ b/docs/createView.md @@ -0,0 +1,103 @@ +##语法 +``` + CREATE VIEW viewName + [ (columnName[ , columnName]*) ] + AS queryStatement; + 或 + CREATE VIEW viewName [ (columnName[ , columnName]*) ]; + INSERT INTO viewName queryStatement; +``` +##样例 +``` +CREATE TABLE MyTable( + name varchar, + channel varchar, + pv INT, + xctime bigint, + CHARACTER_LENGTH(channel) AS timeLeng + )WITH( + type ='kafka09', + bootstrapServers ='172.16.8.198:9092', + zookeeperQuorum ='172.16.8.198:2181/kafka', + offsetReset ='latest', + topic ='nbTest1', + parallelism ='1' + ); + +CREATE TABLE MyResult( + channel VARCHAR, + pv VARCHAR + )WITH( + type ='mysql', + url ='jdbc:mysql://172.16.8.104:3306/test?charset=utf8', + userName ='dtstack', + password ='abc123', + tableName ='yx', + parallelism ='1' + ); + +CREATE TABLE workerinfo( + cast(logtime as TIMESTAMP)AS rtime, + cast(logtime)AS rtime + )WITH( + type ='hbase', + zookeeperQuorum ='rdos1:2181', + tableName ='workerinfo', + rowKey ='ce,de', + parallelism ='1', + zookeeperParent ='/hbase' + ); + +CREATE TABLE REDIS( + name VARCHAR, + pv VARCHAR +)WITH( + type ='redis', + url ='172.16.10.79:6379', + databsae =0, + password ='' +); + +CREATE TABLE sideTable( + cf:name varchar as name, + cf:info varchar as info, + PRIMARY KEY(name), + PERIOD FOR SYSTEM_TIME + )WITH( + type ='hbase', + zookeeperQuorum ='rdos1:2181', + zookeeperParent ='/hbase', + tableName ='workerinfo', + cache ='ALL', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1' + ); + CREATE VIEW abc1 AS SELECT * FROM MyTable; + CREATE VIEW abc2 AS SELECT d.channel, + d.info + FROM + ( SELECT + a.*,b.info + FROM + MyTable a + JION + sideTable b + ON a.channel=b.name + ) as d; +CREATE VIEW abc3(name varchar, info varchar); +insert into abc3 select + d.channel, + d.info + from + abc2 as d; + +insert +into + MyResult + select + d.channel, + d.info + from + abc3 as d; +``` diff --git a/docs/redisSide.md b/docs/redisSide.md index e69de29bb..588cac9d1 100644 --- a/docs/redisSide.md +++ b/docs/redisSide.md @@ -0,0 +1,70 @@ + +## 1.格式: +``` + CREATE TABLE tableName( + colName cloType, + ... + PERIOD FOR SYSTEM_TIME + )WITH( + type ='redis', + url = 'ip:port', + password = 'redisPwd', + database = 'dbName', + tableName ='sideTableName', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000' + ); +``` +## 2.支持版本 + redis5.0 + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName | 注册到flink的表名称(可选填;不填默认和redis对应的表名称相同)| +| colName | 列名称,维表列名格式 表名:主键名:主键值:列名]| +| colType | 列类型,当前只支持varchart| +| PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| + +## 3.参数 + +|参数名称|含义|是否必填|默认值| +|----|---|---|----| +| type | 表明维表的类型[hbase\|mysql\|redis]|是|| +| url | redis 的地址;格式ip:port[,ip:port]|是|| +| password | redis 的密码 |是|| +| database | reids 的数据库地址|否|| +| tableName | redis 的表名称|是|| +| cache | 维表缓存策略(NONE/LRU/ALL)|否|NONE| +| partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| + +-------------- +> 缓存策略 + * NONE: 不做内存缓存 + * LRU: + * cacheSize: 缓存的条目数量 + * cacheTTLMs:缓存的过期时间(ms) + * ALL: 缓存全量表数据 + +## 4.样例 +``` +create table sideTable( + channel varchar, + xccount varchar, + PRIMARY KEY(channel), + PERIOD FOR SYSTEM_TIME + )WITH( + type='redis', + url='172.16.10.79:6379', + password='abc123', + database='0', + tableName='sidetest', + cache = 'LRU', + cacheTTLMs='10000' + ); + +``` + + diff --git a/docs/redisSink.md b/docs/redisSink.md index e69de29bb..ab1794db0 100644 --- a/docs/redisSink.md +++ b/docs/redisSink.md @@ -0,0 +1,57 @@ +## 1.格式: +``` +CREATE TABLE tableName( + colName colType, + ... + colNameX colType + )WITH( + type ='mysql', + url ='redisUrl', + database ='dbName', + password ='pwd', + tableName ='tableName', + parallelism ='parllNum' + ); + + +``` + +## 2.支持版本 +redis5.0 + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName | 在 sql 中使用的名称;即注册到flink-table-env上的名称 +| colName | 列名称,redis中存储为 表名:主键名:主键值:列名]| +| colType | 列类型,当前只支持varchart| +| PRIMARY KEY(keyInfo) | 结果表主键定义;多个列之间用逗号隔开| + +## 4.参数: + +|参数名称|含义|是否必填|默认值| +|----|---|---|-----| +|type | 表明 输出表类型[mysql\|hbase\|elasticsearch\|redis\]|是|| +| url | redis 的地址;格式ip:port[,ip:port]|是|| +| password | redis 的密码 |是|| +| database | reids 的数据库地址|否|| +| tableName | redis 的表名称|是|| +|parallelism | 并行度设置|否|1| + + +## 5.样例: +``` + CREATE TABLE MyResult( + channel varchar, + pv varchar, + PRIMARY KEY(channel) + )WITH( + type='redis', + url='172.16.10.79:6379', + password='abc123', + database='0', + tableName='sinktoredis', + ); + + ``` \ No newline at end of file From c66a14ca72666118ae069b23bc13d542b851cbe9 Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Thu, 15 Nov 2018 14:23:16 +0800 Subject: [PATCH 126/470] modfy redisSink.md --- docs/redisSink.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/redisSink.md b/docs/redisSink.md index ab1794db0..8b2525f5f 100644 --- a/docs/redisSink.md +++ b/docs/redisSink.md @@ -5,8 +5,8 @@ CREATE TABLE tableName( ... colNameX colType )WITH( - type ='mysql', - url ='redisUrl', + type ='redis', + url = 'ip:port', database ='dbName', password ='pwd', tableName ='tableName', From 879428865b431f7748fac8f9143d4e327448c5c7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Thu, 15 Nov 2018 14:32:36 +0800 Subject: [PATCH 127/470] modify docs and pom --- docs/redisSide.md | 2 +- docs/redisSink.md | 2 +- redis5/redis5-side/pom.xml | 1 + redis5/redis5-sink/pom.xml | 1 + 4 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/redisSide.md b/docs/redisSide.md index 588cac9d1..52d637b70 100644 --- a/docs/redisSide.md +++ b/docs/redisSide.md @@ -25,7 +25,7 @@ |----|---| | tableName | 注册到flink的表名称(可选填;不填默认和redis对应的表名称相同)| | colName | 列名称,维表列名格式 表名:主键名:主键值:列名]| -| colType | 列类型,当前只支持varchart| +| colType | 列类型,当前只支持varchar| | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| ## 3.参数 diff --git a/docs/redisSink.md b/docs/redisSink.md index ab1794db0..4577648f1 100644 --- a/docs/redisSink.md +++ b/docs/redisSink.md @@ -25,7 +25,7 @@ redis5.0 |----|---| | tableName | 在 sql 中使用的名称;即注册到flink-table-env上的名称 | colName | 列名称,redis中存储为 表名:主键名:主键值:列名]| -| colType | 列类型,当前只支持varchart| +| colType | 列类型,当前只支持varchar| | PRIMARY KEY(keyInfo) | 结果表主键定义;多个列之间用逗号隔开| ## 4.参数: diff --git a/redis5/redis5-side/pom.xml b/redis5/redis5-side/pom.xml index c81f1c8c3..c6623b9d2 100644 --- a/redis5/redis5-side/pom.xml +++ b/redis5/redis5-side/pom.xml @@ -16,6 +16,7 @@ com.dtstack.flink sql.core 1.0-SNAPSHOT + provided redis.clients diff --git a/redis5/redis5-sink/pom.xml b/redis5/redis5-sink/pom.xml index ae54fec10..994c3972a 100644 --- a/redis5/redis5-sink/pom.xml +++ b/redis5/redis5-sink/pom.xml @@ -21,6 +21,7 @@ com.dtstack.flink sql.core 1.0-SNAPSHOT + provided redis.clients From 60ec92c93e5dcf61fa7d0fe1058a1aec165ed94c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Thu, 15 Nov 2018 22:55:02 +0800 Subject: [PATCH 128/470] support async --- .../sql/side/redis/RedisAsyncReqRow.java | 53 ++++++++++++------- .../sql/sink/redis/RedisOutputFormat.java | 2 +- 2 files changed, 36 insertions(+), 19 deletions(-) diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index 61a10ad8f..08c53a8a7 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -22,7 +22,10 @@ import com.dtstack.flink.sql.side.*; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.redis.table.RedisSideTableInfo; +import io.lettuce.core.KeyScanCursor; +import io.lettuce.core.KeyValue; import io.lettuce.core.RedisClient; +import io.lettuce.core.RedisFuture; import io.lettuce.core.api.StatefulRedisConnection; import io.lettuce.core.api.async.RedisAsyncCommands; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -37,7 +40,9 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutionException; import java.util.function.Consumer; +import java.util.stream.Collectors; public class RedisAsyncReqRow extends AsyncReqRow { @@ -90,12 +95,12 @@ protected Row fillData(Row input, Object sideInput) { row.setField(entry.getKey(), obj); } - // TODO: 2018/11/13 插入维表数据 for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ if(keyValue == null){ row.setField(entry.getKey(), null); }else{ - row.setField(entry.getKey(), keyValue.get(entry.getValue())); + String key = sideInfo.getSideFieldNameIndex().get(entry.getKey()); + row.setField(entry.getKey(), keyValue.get(key)); } } @@ -135,24 +140,36 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except } } - async.keys(key).whenComplete((value, e)-> { - for (String newKey : value){ - String[] splitKey = newKey.split(":"); - Map keyValue = Maps.newConcurrentMap(); - keyValue.put(splitKey[1], splitKey[2]); - async.get(newKey).thenAccept(new Consumer() { - @Override - public void accept(String s) { - keyValue.put(splitKey[3], s); + Map keyValue = Maps.newHashMap(); + List value = async.keys(key + ":*").get(); + String[] values = value.toArray(new String[value.size()]); + RedisFuture>> future = async.mget(values); + while (future.isDone()){ + try { + List> kvList = future.get(); + if (kvList.size() != 0){ + for (int i=0; i keyData) { String tableName = redisSideTableInfo.getTableName(); StringBuilder preKey = new StringBuilder(); preKey.append(tableName).append(":").append(kv); - return preKey.toString().toLowerCase(); + return preKey.toString(); } @Override diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java index a1d0db721..742b2a377 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java @@ -126,7 +126,7 @@ public void writeRecord(Tuple2 record) throws IOException { for (int i = 0; i < fieldNames.length; i++) { StringBuilder key = new StringBuilder(); key.append(tableName).append(":").append(perKey).append(":").append(fieldNames[i]); - jedis.append(key.toString(), (String) row.getField(i)); + jedis.set(key.toString(), (String) row.getField(i)); } } From 042dfceb5f910371a621291430a68ccaa432f168 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Fri, 16 Nov 2018 11:13:49 +0800 Subject: [PATCH 129/470] fix bug --- .../java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index 08c53a8a7..480b22051 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -144,7 +144,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except List value = async.keys(key + ":*").get(); String[] values = value.toArray(new String[value.size()]); RedisFuture>> future = async.mget(values); - while (future.isDone()){ + if (future.isDone()){ try { List> kvList = future.get(); if (kvList.size() != 0){ From 19adfe9a510d9f6361a8bfbb132ab0a4262b986e Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 16 Nov 2018 14:24:02 +0800 Subject: [PATCH 130/470] Update README.md --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index c632d786c..2fb722c6d 100644 --- a/README.md +++ b/README.md @@ -16,6 +16,7 @@ * 增加SQlServer维表,结果表功能 * 增加kafka结果表功能 * 增加SQL支持CEP + * 维表快照 ## 1 快速起步 ### 1.1 运行模式 From abf98a4db21aa13233ccd416e27e5d7835445f8d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=92=8B=E4=BF=8A=E6=9D=B0?= Date: Fri, 16 Nov 2018 15:19:07 +0800 Subject: [PATCH 131/470] fix redis async bug --- .../sql/side/redis/RedisAsyncReqRow.java | 21 +++++++------------ 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index 480b22051..a541c9300 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -144,14 +144,14 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except List value = async.keys(key + ":*").get(); String[] values = value.toArray(new String[value.size()]); RedisFuture>> future = async.mget(values); - if (future.isDone()){ - try { - List> kvList = future.get(); - if (kvList.size() != 0){ - for (int i=0; i>>() { + @Override + public void accept(List> keyValues) { + if (keyValues.size() != 0){ + for (int i=0; i resultFuture) throws Except putCache(key, CacheMissVal.getMissKeyObj()); } } - } catch (InterruptedException e1) { - e1.printStackTrace(); - } catch (ExecutionException e1) { - e1.printStackTrace(); } - } - + }); } private String buildCacheKey(List keyData) { From 18e23db77410b96d8422cf80eb689682b1ad270a Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 16 Nov 2018 16:53:31 +0800 Subject: [PATCH 132/470] exclude plugin of log4j dependency --- hbase/hbase-side/hbase-all-side/pom.xml | 6 ++++++ hbase/hbase-side/hbase-async-side/pom.xml | 6 ++++++ hbase/pom.xml | 10 ++++++++++ 3 files changed, 22 insertions(+) diff --git a/hbase/hbase-side/hbase-all-side/pom.xml b/hbase/hbase-side/hbase-all-side/pom.xml index 9b461ce94..53e1ca3ac 100644 --- a/hbase/hbase-side/hbase-all-side/pom.xml +++ b/hbase/hbase-side/hbase-all-side/pom.xml @@ -24,6 +24,12 @@ org.apache.hbase hbase-client 1.3.1 + + + log4j + log4j + + diff --git a/hbase/hbase-side/hbase-async-side/pom.xml b/hbase/hbase-side/hbase-async-side/pom.xml index 2a18eeea6..f1b4baf2d 100644 --- a/hbase/hbase-side/hbase-async-side/pom.xml +++ b/hbase/hbase-side/hbase-async-side/pom.xml @@ -25,6 +25,12 @@ org.hbase asynchbase 1.8.2 + + + log4j-over-slf4j + org.slf4j + + diff --git a/hbase/pom.xml b/hbase/pom.xml index 54daa079b..48c8162a0 100644 --- a/hbase/pom.xml +++ b/hbase/pom.xml @@ -34,6 +34,16 @@ org.apache.hbase hbase-client 1.3.1 + + + slf4j-log4j12 + org.slf4j + + + log4j + log4j + + From 8c5507a4016da108a6b7eb6fe71562369799d970 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 16 Nov 2018 17:13:02 +0800 Subject: [PATCH 133/470] exclude plugin of slf4j dependency --- elasticsearch5/elasticsearch5-sink/pom.xml | 4 ++-- hbase/hbase-side/hbase-all-side/pom.xml | 1 + hbase/hbase-side/hbase-async-side/pom.xml | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/elasticsearch5/elasticsearch5-sink/pom.xml b/elasticsearch5/elasticsearch5-sink/pom.xml index 8e2533da0..68f464e48 100644 --- a/elasticsearch5/elasticsearch5-sink/pom.xml +++ b/elasticsearch5/elasticsearch5-sink/pom.xml @@ -47,8 +47,8 @@ - - + org.apache.logging.log4j:log4j-to-slf4j + org.slf4j:slf4j-api diff --git a/hbase/hbase-side/hbase-all-side/pom.xml b/hbase/hbase-side/hbase-all-side/pom.xml index 53e1ca3ac..6e344dd5d 100644 --- a/hbase/hbase-side/hbase-all-side/pom.xml +++ b/hbase/hbase-side/hbase-all-side/pom.xml @@ -51,6 +51,7 @@ org.apache.hadoop:hadoop-common org.apache.hadoop:hadoop-auth org.apache.hadoop:hadoop-mapreduce-client-core + org.slf4j diff --git a/hbase/hbase-side/hbase-async-side/pom.xml b/hbase/hbase-side/hbase-async-side/pom.xml index f1b4baf2d..4da76c68c 100644 --- a/hbase/hbase-side/hbase-async-side/pom.xml +++ b/hbase/hbase-side/hbase-async-side/pom.xml @@ -49,10 +49,10 @@ - org.slf4j:slf4j-log4j12 org.apache.hadoop:hadoop-common org.apache.hadoop:hadoop-auth org.apache.hadoop:hadoop-mapreduce-client-core + org.slf4j From 13261e6aeb8a72721e59009754a87c2e581836ae Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 16 Nov 2018 18:56:46 +0800 Subject: [PATCH 134/470] remove sum --- core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index ae3824f8b..90e45d57e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -374,7 +374,6 @@ private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable Date: Fri, 16 Nov 2018 21:42:20 +0800 Subject: [PATCH 135/470] add per job submit mode --- .../sql/launcher/ClusterClientFactory.java | 16 ++ .../flink/sql/launcher/LauncherMain.java | 2 +- .../perjob/FLinkPerJobResourceUtil.java | 80 ++++++++++ .../perjob/PerJobClusterClientBuilder.java | 145 ++++++++++++++++++ 4 files changed, 242 insertions(+), 1 deletion(-) create mode 100644 launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/FLinkPerJobResourceUtil.java create mode 100644 launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 108ec0989..61702ea76 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -18,6 +18,8 @@ package com.dtstack.flink.sql.launcher; +import com.dtstack.flink.sql.launcher.perjob.FLinkPerJobResourceUtil; +import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang.StringUtils; import org.apache.flink.client.deployment.ClusterRetrieveException; import org.apache.flink.client.deployment.ClusterSpecification; @@ -45,12 +47,14 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; import java.io.File; import java.io.FilenameFilter; +import java.io.IOException; import java.net.InetSocketAddress; import java.util.EnumSet; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -71,7 +75,10 @@ public static ClusterClient createClusterClient(LauncherOptions launcherOptions) return createStandaloneClient(launcherOptions); } else if(mode.equals(ClusterMode.yarn.name())) { return createYarnClient(launcherOptions); + } else if(mode.equals(ClusterMode.yarnPer.name())){ + return createPerJobModeYarnClient(launcherOptions); } + throw new IllegalArgumentException("Unsupported cluster client type: "); } @@ -172,6 +179,15 @@ public boolean accept(File dir, String name) { throw new UnsupportedOperationException("Haven't been developed yet!"); } + public static ClusterClient createPerJobModeYarnClient(LauncherOptions launcherOptions) throws IOException { + + Properties confProperties = PluginUtil.jsonStrToObject(launcherOptions.getConfProp(), Properties.class); + ClusterSpecification clusterSpecification = FLinkPerJobResourceUtil.createClusterSpecification(confProperties); + + //TODO + return null; + } + /** * 处理yarn HA的配置项 */ diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 55f085c98..4553bd2ab 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -55,7 +55,7 @@ public static void main(String[] args) throws Exception { if(mode.equals(ClusterMode.local.name())) { String[] localArgs = argList.toArray(new String[argList.size()]); Main.main(localArgs); - } else { + } else{ ClusterClient clusterClient = ClusterClientFactory.createClusterClient(launcherOptions); String pluginRoot = launcherOptions.getLocalSqlPluginPath(); File jarFile = new File(getLocalCoreJarPath(pluginRoot)); diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/FLinkPerJobResourceUtil.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/FLinkPerJobResourceUtil.java new file mode 100644 index 000000000..3becc41d8 --- /dev/null +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/FLinkPerJobResourceUtil.java @@ -0,0 +1,80 @@ +/** + * 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 com.dtstack.flink.sql.launcher.perjob; + + +import com.dtstack.flink.sql.util.MathUtil; +import org.apache.flink.client.deployment.ClusterSpecification; + +import java.util.Properties; + +/** + * company: www.dtstack.com + * author xuchao + * create: 2018/7/5 + */ +public class FLinkPerJobResourceUtil { + + public final static int MIN_JM_MEMORY = 1024; // the minimum memory should be higher than the min heap cutoff + public final static int MIN_TM_MEMORY = 1024; + + public final static String JOBMANAGER_MEMORY_MB = "jobmanager.memory.mb"; + public final static String TASKMANAGER_MEMORY_MB = "taskmanager.memory.mb"; + public final static String CONTAINER = "container"; + public final static String SLOTS = "slots"; + + public static ClusterSpecification createClusterSpecification(Properties confProperties) { + int jobmanagerMemoryMb = 1024; + int taskmanagerMemoryMb = 1024; + int numberTaskManagers = 1; + int slotsPerTaskManager = 1; + + if (confProperties != null) { + if (confProperties.containsKey(JOBMANAGER_MEMORY_MB)){ + jobmanagerMemoryMb = MathUtil.getIntegerVal(confProperties.get(JOBMANAGER_MEMORY_MB)); + if (jobmanagerMemoryMb < MIN_JM_MEMORY) { + jobmanagerMemoryMb = MIN_JM_MEMORY; + } + } + + if (confProperties.containsKey(TASKMANAGER_MEMORY_MB)){ + taskmanagerMemoryMb = MathUtil.getIntegerVal(confProperties.get(TASKMANAGER_MEMORY_MB)); + if (taskmanagerMemoryMb < MIN_TM_MEMORY) { + taskmanagerMemoryMb = MIN_TM_MEMORY; + } + } + + if (confProperties.containsKey(CONTAINER)){ + numberTaskManagers = MathUtil.getIntegerVal(confProperties.get(CONTAINER)); + } + + if (confProperties.containsKey(SLOTS)){ + slotsPerTaskManager = MathUtil.getIntegerVal(confProperties.get(SLOTS)); + } + } + + return new ClusterSpecification.ClusterSpecificationBuilder() + .setMasterMemoryMB(jobmanagerMemoryMb) + .setTaskManagerMemoryMB(taskmanagerMemoryMb) + .setNumberTaskManagers(numberTaskManagers) + .setSlotsPerTaskManager(slotsPerTaskManager) + .createClusterSpecification(); + } + +} diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java new file mode 100644 index 000000000..6531ff5b0 --- /dev/null +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java @@ -0,0 +1,145 @@ +/** + * 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 com.dtstack.flink.sql.launcher.perjob; + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.hadoop.shaded.com.google.common.base.Strings; +import org.apache.flink.yarn.AbstractYarnClusterDescriptor; +import org.apache.flink.yarn.YarnClusterDescriptor; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +import java.io.File; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +/** + * Reason: + * Date: 2018/11/16 + * Company: www.dtstack.com + * @author xuchao + */ + +public class PerJobClusterClientBuilder { + + public static final String DEFAULT_GATEWAY_CLASS = "org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter"; + + public static final String PROMGATEWAY_CLASS_KEY = "metrics.reporter.promgateway.class"; + + public static final String PROMGATEWAY_HOST_KEY = "metrics.reporter.promgateway.host"; + + public static final String PROMGATEWAY_PORT_KEY = "metrics.reporter.promgateway.port"; + + public static final String PROMGATEWAY_JOBNAME_KEY = "metrics.reporter.promgateway.jobName"; + + public static final String PROMGATEWAY_RANDOMJOBNAMESUFFIX_KEY = "metrics.reporter.promgateway.randomJobNameSuffix"; + + public static final String PROMGATEWAY_DELETEONSHUTDOWN_KEY = "metrics.reporter.promgateway.deleteOnShutdown"; + + private YarnClient yarnClient; + + private YarnConfiguration yarnConf; + + public void init(String yarnConfDir){ + if(Strings.isNullOrEmpty(yarnConfDir)) { + throw new RuntimeException("parameters of yarn is required"); + } + + } + + public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties properties, String flinkJarPath, + String jobmanagerArchiveFsDir, String queue) throws MalformedURLException { + Configuration newConf = new Configuration(); + + if(properties.containsKey(HighAvailabilityOptions.HA_CLUSTER_ID)){ + newConf.setString(HighAvailabilityOptions.HA_CLUSTER_ID, properties.getProperty(HighAvailabilityOptions.HA_CLUSTER_ID.key())); + } + + perJobMetricConfigConfig(newConf, properties); + + AbstractYarnClusterDescriptor clusterDescriptor = getClusterDescriptor(newConf, yarnConf, "."); + + if (StringUtils.isNotBlank(flinkJarPath)) { + + if (!new File(flinkJarPath).exists()) { + throw new RuntimeException("The Flink jar path is not exist"); + } + + } + + if(StringUtils.isNotBlank(jobmanagerArchiveFsDir)){ + newConf.setString(JobManagerOptions.ARCHIVE_DIR, jobmanagerArchiveFsDir); + } + + List classpaths = new ArrayList<>(); + if (flinkJarPath != null) { + File[] jars = new File(flinkJarPath).listFiles(); + + for (File file : jars){ + if (file.toURI().toURL().toString().contains("flink-dist")){ + clusterDescriptor.setLocalJarPath(new Path(file.toURI().toURL().toString())); + } else { + classpaths.add(file.toURI().toURL()); + } + } + + } else { + throw new RuntimeException("The Flink jar path is null"); + } + + clusterDescriptor.setProvidedUserJarFiles(classpaths); + + if(!Strings.isNullOrEmpty(queue)){ + clusterDescriptor.setQueue(queue); + } + return clusterDescriptor; + } + + private void perJobMetricConfigConfig(Configuration configuration, Properties properties){ + if(!properties.containsKey(DEFAULT_GATEWAY_CLASS)){ + return; + } + + configuration.setString(PROMGATEWAY_CLASS_KEY, properties.getProperty(PROMGATEWAY_CLASS_KEY)); + configuration.setString(PROMGATEWAY_HOST_KEY, properties.getProperty(PROMGATEWAY_HOST_KEY)); + configuration.setString(PROMGATEWAY_PORT_KEY, properties.getProperty(PROMGATEWAY_PORT_KEY)); + configuration.setString(PROMGATEWAY_JOBNAME_KEY, properties.getProperty(PROMGATEWAY_JOBNAME_KEY)); + configuration.setString(PROMGATEWAY_RANDOMJOBNAMESUFFIX_KEY, properties.getProperty(PROMGATEWAY_RANDOMJOBNAMESUFFIX_KEY)); + configuration.setString(PROMGATEWAY_DELETEONSHUTDOWN_KEY, properties.getProperty(PROMGATEWAY_DELETEONSHUTDOWN_KEY)); + } + + public AbstractYarnClusterDescriptor getClusterDescriptor( + Configuration configuration, + YarnConfiguration yarnConfiguration, + String configurationDirectory) { + return new YarnClusterDescriptor( + configuration, + yarnConfiguration, + configurationDirectory, + yarnClient, + false); + } +} From e1fc34a5fc3c1acc0ab044bb90f5f79af05071f4 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 17 Nov 2018 14:33:23 +0800 Subject: [PATCH 136/470] add per job submit mode --- .../sql/launcher/ClusterClientFactory.java | 167 ++++-------------- .../flink/sql/launcher/LauncherMain.java | 33 +++- .../sql/launcher/LauncherOptionParser.java | 11 +- .../flink/sql/launcher/LauncherOptions.java | 19 ++ .../flink/sql/launcher/YarnConfLoader.java | 85 +++++++++ .../perjob/PerJobClusterClientBuilder.java | 27 ++- .../sql/launcher/perjob/PerJobSubmitter.java | 66 +++++++ 7 files changed, 253 insertions(+), 155 deletions(-) create mode 100644 launcher/src/main/java/com/dtstack/flink/sql/launcher/YarnConfLoader.java create mode 100644 launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 61702ea76..9dfe2e07e 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -18,15 +18,11 @@ package com.dtstack.flink.sql.launcher; -import com.dtstack.flink.sql.launcher.perjob.FLinkPerJobResourceUtil; -import com.dtstack.flink.sql.util.PluginUtil; +import com.dtstack.flink.sql.ClusterMode; import org.apache.commons.lang.StringUtils; -import org.apache.flink.client.deployment.ClusterRetrieveException; -import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.deployment.StandaloneClusterDescriptor; import org.apache.flink.client.deployment.StandaloneClusterId; import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.client.program.StandaloneClusterClient; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -34,9 +30,6 @@ import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; -import org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperHaServices; import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; @@ -45,21 +38,12 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.client.api.YarnClient; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import java.io.File; -import java.io.FilenameFilter; -import java.io.IOException; + import java.net.InetSocketAddress; import java.util.EnumSet; import java.util.HashSet; -import java.util.Iterator; import java.util.List; -import java.util.Map; -import java.util.Properties; import java.util.Set; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; - -import com.dtstack.flink.sql.ClusterMode; /** * The Factory of ClusterClient @@ -75,8 +59,6 @@ public static ClusterClient createClusterClient(LauncherOptions launcherOptions) return createStandaloneClient(launcherOptions); } else if(mode.equals(ClusterMode.yarn.name())) { return createYarnClient(launcherOptions); - } else if(mode.equals(ClusterMode.yarnPer.name())){ - return createPerJobModeYarnClient(launcherOptions); } throw new IllegalArgumentException("Unsupported cluster client type: "); @@ -101,138 +83,61 @@ public static ClusterClient createYarnClient(LauncherOptions launcherOptions) { String flinkConfDir = launcherOptions.getFlinkconf(); Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); String yarnConfDir = launcherOptions.getYarnconf(); - YarnConfiguration yarnConf = new YarnConfiguration(); if(StringUtils.isNotBlank(yarnConfDir)) { - try { + try { config.setString(ConfigConstants.PATH_HADOOP_CONFIG, yarnConfDir); FileSystem.initialize(config); - File dir = new File(yarnConfDir); - if(dir.exists() && dir.isDirectory()) { - File[] xmlFileList = new File(yarnConfDir).listFiles(new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - if(name.endsWith(".xml")){ - return true; - } - return false; - } - }); - - if(xmlFileList != null) { - for(File xmlFile : xmlFileList) { - yarnConf.addResource(xmlFile.toURI().toURL()); - } + YarnConfiguration yarnConf = YarnConfLoader.getYarnConf(yarnConfDir); + YarnClient yarnClient = YarnClient.createYarnClient(); + yarnClient.init(yarnConf); + yarnClient.start(); + ApplicationId applicationId = null; + + Set set = new HashSet<>(); + set.add("Apache Flink"); + EnumSet enumSet = EnumSet.noneOf(YarnApplicationState.class); + enumSet.add(YarnApplicationState.RUNNING); + List reportList = yarnClient.getApplications(set, enumSet); + + int maxMemory = -1; + int maxCores = -1; + for(ApplicationReport report : reportList) { + if(!report.getName().startsWith("Flink session")){ + continue; } - YarnClient yarnClient = YarnClient.createYarnClient(); - haYarnConf(yarnConf); - yarnClient.init(yarnConf); - yarnClient.start(); - ApplicationId applicationId = null; - - Set set = new HashSet<>(); - set.add("Apache Flink"); - EnumSet enumSet = EnumSet.noneOf(YarnApplicationState.class); - enumSet.add(YarnApplicationState.RUNNING); - List reportList = yarnClient.getApplications(set, enumSet); - - int maxMemory = -1; - int maxCores = -1; - for(ApplicationReport report : reportList) { - if(!report.getName().startsWith("Flink session")){ - continue; - } - - if(!report.getYarnApplicationState().equals(YarnApplicationState.RUNNING)) { - continue; - } - - int thisMemory = report.getApplicationResourceUsageReport().getNeededResources().getMemory(); - int thisCores = report.getApplicationResourceUsageReport().getNeededResources().getVirtualCores(); - if(thisMemory > maxMemory || thisMemory == maxMemory && thisCores > maxCores) { - maxMemory = thisMemory; - maxCores = thisCores; - applicationId = report.getApplicationId(); - } - + if(!report.getYarnApplicationState().equals(YarnApplicationState.RUNNING)) { + continue; } - if(StringUtils.isEmpty(applicationId.toString())) { - throw new RuntimeException("No flink session found on yarn cluster."); + int thisMemory = report.getApplicationResourceUsageReport().getNeededResources().getMemory(); + int thisCores = report.getApplicationResourceUsageReport().getNeededResources().getVirtualCores(); + if(thisMemory > maxMemory || thisMemory == maxMemory && thisCores > maxCores) { + maxMemory = thisMemory; + maxCores = thisCores; + applicationId = report.getApplicationId(); } + } - AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor(config, yarnConf, ".", yarnClient, false); - ClusterClient clusterClient = clusterDescriptor.retrieve(applicationId); - clusterClient.setDetached(true); - return clusterClient; + if(StringUtils.isEmpty(applicationId.toString())) { + throw new RuntimeException("No flink session found on yarn cluster."); } + + AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor(config, yarnConf, ".", yarnClient, false); + ClusterClient clusterClient = clusterDescriptor.retrieve(applicationId); + clusterClient.setDetached(true); + return clusterClient; } catch(Exception e) { throw new RuntimeException(e); } } - - throw new UnsupportedOperationException("Haven't been developed yet!"); } - public static ClusterClient createPerJobModeYarnClient(LauncherOptions launcherOptions) throws IOException { - - Properties confProperties = PluginUtil.jsonStrToObject(launcherOptions.getConfProp(), Properties.class); - ClusterSpecification clusterSpecification = FLinkPerJobResourceUtil.createClusterSpecification(confProperties); - //TODO - return null; - } - - /** - * 处理yarn HA的配置项 - */ - private static org.apache.hadoop.conf.Configuration haYarnConf(org.apache.hadoop.conf.Configuration yarnConf) { - Iterator> iterator = yarnConf.iterator(); - while(iterator.hasNext()) { - Map.Entry entry = iterator.next(); - String key = entry.getKey(); - String value = entry.getValue(); - if(key.startsWith("yarn.resourcemanager.hostname.")) { - String rm = key.substring("yarn.resourcemanager.hostname.".length()); - String addressKey = "yarn.resourcemanager.address." + rm; - if(yarnConf.get(addressKey) == null) { - yarnConf.set(addressKey, value + ":" + YarnConfiguration.DEFAULT_RM_PORT); - } - } - } - return yarnConf; - } - - private static org.apache.hadoop.conf.Configuration getYarnConf(String yarnConfDir) { - org.apache.hadoop.conf.Configuration yarnConf = new YarnConfiguration(); - try { - - File dir = new File(yarnConfDir); - if(dir.exists() && dir.isDirectory()) { - File[] xmlFileList = new File(yarnConfDir).listFiles(new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - if(name.endsWith(".xml")){ - return true; - } - return false; - } - }); - if(xmlFileList != null) { - for(File xmlFile : xmlFileList) { - yarnConf.addResource(xmlFile.toURI().toURL()); - } - } - } - } catch(Exception e) { - throw new RuntimeException(e); - } - return yarnConf; - } } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 4553bd2ab..8e5832294 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -22,11 +22,16 @@ import avro.shaded.com.google.common.collect.Lists; import com.dtstack.flink.sql.Main; +import com.dtstack.flink.sql.launcher.perjob.PerJobSubmitter; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; import java.io.File; import java.util.List; import com.dtstack.flink.sql.ClusterMode; +import org.apache.flink.client.program.PackagedProgramUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.table.shaded.org.apache.commons.lang.StringUtils; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.table.shaded.org.apache.commons.lang.BooleanUtils; @@ -55,17 +60,29 @@ public static void main(String[] args) throws Exception { if(mode.equals(ClusterMode.local.name())) { String[] localArgs = argList.toArray(new String[argList.size()]); Main.main(localArgs); - } else{ + return; + } + + String pluginRoot = launcherOptions.getLocalSqlPluginPath(); + File jarFile = new File(getLocalCoreJarPath(pluginRoot)); + String[] remoteArgs = argList.toArray(new String[argList.size()]); + PackagedProgram program = new PackagedProgram(jarFile, Lists.newArrayList(), remoteArgs); + + if(StringUtils.isNotBlank(launcherOptions.getSavePointPath())){ + program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getSavePointPath(), BooleanUtils.toBoolean(launcherOptions.getAllowNonRestoredState()))); + } + + if(mode.equals(ClusterMode.yarnPer.name())){ + String flinkConfDir = launcherOptions.getFlinkconf(); + Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); + JobGraph jobGraph = PackagedProgramUtils.createJobGraph(program, config, 1); + PerJobSubmitter.submit(launcherOptions, jobGraph); + } else { ClusterClient clusterClient = ClusterClientFactory.createClusterClient(launcherOptions); - String pluginRoot = launcherOptions.getLocalSqlPluginPath(); - File jarFile = new File(getLocalCoreJarPath(pluginRoot)); - String[] remoteArgs = argList.toArray(new String[argList.size()]); - PackagedProgram program = new PackagedProgram(jarFile, Lists.newArrayList(), remoteArgs); - if(StringUtils.isNotBlank(launcherOptions.getSavePointPath())){ - program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getSavePointPath(), BooleanUtils.toBoolean(launcherOptions.getAllowNonRestoredState()))); - } clusterClient.run(program, 1); clusterClient.shutdown(); } + + System.out.println("---submit end----"); } } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java index 5052d22e8..75c5c4f0f 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java @@ -65,6 +65,8 @@ public class LauncherOptionParser { public static final String OPTION_ALLOW_NON_RESTORED_STATE = "allowNonRestoredState"; + public static final String OPTION_FLINK_JAR_PATH = "flinkJarPath"; + private Options options = new Options(); private BasicParser parser = new BasicParser(); @@ -84,6 +86,7 @@ public LauncherOptionParser(String[] args) { options.addOption(OPTION_SAVE_POINT_PATH, true, "Savepoint restore path"); options.addOption(OPTION_ALLOW_NON_RESTORED_STATE, true, "Flag indicating whether non restored state is allowed if the savepoint"); + options.addOption(OPTION_FLINK_JAR_PATH, true, "flink jar path for submit of perjob mode"); try { CommandLine cl = parser.parse(options, args); @@ -137,6 +140,11 @@ public LauncherOptionParser(String[] args) { properties.setAllowNonRestoredState(allow_non); } + String flinkJarPath = cl.getOptionValue(OPTION_FLINK_JAR_PATH); + if(StringUtils.isNotBlank(flinkJarPath)){ + properties.setFlinkJarPath(flinkJarPath); + } + } catch (Exception e) { throw new RuntimeException(e); } @@ -152,7 +160,8 @@ public List getProgramExeArgList() throws Exception { for(Map.Entry one : mapConf.entrySet()){ String key = one.getKey(); if(OPTION_FLINK_CONF_DIR.equalsIgnoreCase(key) - || OPTION_YARN_CONF_DIR.equalsIgnoreCase(key)){ + || OPTION_YARN_CONF_DIR.equalsIgnoreCase(key) + || OPTION_FLINK_JAR_PATH.equalsIgnoreCase(key)){ continue; } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java index f183957f7..5cc762ebd 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java @@ -48,6 +48,11 @@ public class LauncherOptions { private String allowNonRestoredState = "false"; + //just use for per_job mode + private String flinkJarPath; + + private String queue; + public String getMode() { return mode; } @@ -136,5 +141,19 @@ public void setAllowNonRestoredState(String allowNonRestoredState) { this.allowNonRestoredState = allowNonRestoredState; } + public String getFlinkJarPath() { + return flinkJarPath; + } + + public void setFlinkJarPath(String flinkJarPath) { + this.flinkJarPath = flinkJarPath; + } + + public String getQueue() { + return queue; + } + public void setQueue(String queue) { + this.queue = queue; + } } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/YarnConfLoader.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/YarnConfLoader.java new file mode 100644 index 000000000..717da24fa --- /dev/null +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/YarnConfLoader.java @@ -0,0 +1,85 @@ +/* + * 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 com.dtstack.flink.sql.launcher; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +import java.io.File; +import java.util.Iterator; +import java.util.Map; + +/** + * load yarn conf from specify dir + * Date: 2018/11/17 + * Company: www.dtstack.com + * @author xuchao + */ + +public class YarnConfLoader { + + public static YarnConfiguration getYarnConf(String yarnConfDir) { + YarnConfiguration yarnConf = new YarnConfiguration(); + try { + + File dir = new File(yarnConfDir); + if(dir.exists() && dir.isDirectory()) { + + File[] xmlFileList = new File(yarnConfDir).listFiles((dir1, name) -> { + if(name.endsWith(".xml")){ + return true; + } + return false; + }); + + if(xmlFileList != null) { + for(File xmlFile : xmlFileList) { + yarnConf.addResource(xmlFile.toURI().toURL()); + } + } + } + + } catch(Exception e) { + throw new RuntimeException(e); + } + + haYarnConf(yarnConf); + return yarnConf; + } + + /** + * deal yarn HA conf + */ + private static Configuration haYarnConf(Configuration yarnConf) { + Iterator> iterator = yarnConf.iterator(); + while(iterator.hasNext()) { + Map.Entry entry = iterator.next(); + String key = entry.getKey(); + String value = entry.getValue(); + if(key.startsWith("yarn.resourcemanager.hostname.")) { + String rm = key.substring("yarn.resourcemanager.hostname.".length()); + String addressKey = "yarn.resourcemanager.address." + rm; + if(yarnConf.get(addressKey) == null) { + yarnConf.set(addressKey, value + ":" + YarnConfiguration.DEFAULT_RM_PORT); + } + } + } + return yarnConf; + } +} diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java index 6531ff5b0..7d395f0ac 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java @@ -18,10 +18,9 @@ package com.dtstack.flink.sql.launcher.perjob; +import com.dtstack.flink.sql.launcher.YarnConfLoader; import org.apache.commons.lang3.StringUtils; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.HighAvailabilityOptions; -import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.hadoop.shaded.com.google.common.base.Strings; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; @@ -68,18 +67,19 @@ public void init(String yarnConfDir){ throw new RuntimeException("parameters of yarn is required"); } + yarnConf = YarnConfLoader.getYarnConf(yarnConfDir); + yarnClient = YarnClient.createYarnClient(); + yarnClient.init(yarnConf); + yarnClient.start(); + + System.out.println("----init yarn success ----"); } - public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties properties, String flinkJarPath, - String jobmanagerArchiveFsDir, String queue) throws MalformedURLException { + public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, String flinkJarPath, String queue) throws MalformedURLException { Configuration newConf = new Configuration(); + newConf.addAllToProperties(confProp); - if(properties.containsKey(HighAvailabilityOptions.HA_CLUSTER_ID)){ - newConf.setString(HighAvailabilityOptions.HA_CLUSTER_ID, properties.getProperty(HighAvailabilityOptions.HA_CLUSTER_ID.key())); - } - - perJobMetricConfigConfig(newConf, properties); - + //perJobMetricConfigConfig(newConf, properties); AbstractYarnClusterDescriptor clusterDescriptor = getClusterDescriptor(newConf, yarnConf, "."); if (StringUtils.isNotBlank(flinkJarPath)) { @@ -90,10 +90,6 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties pr } - if(StringUtils.isNotBlank(jobmanagerArchiveFsDir)){ - newConf.setString(JobManagerOptions.ARCHIVE_DIR, jobmanagerArchiveFsDir); - } - List classpaths = new ArrayList<>(); if (flinkJarPath != null) { File[] jars = new File(flinkJarPath).listFiles(); @@ -118,6 +114,7 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties pr return clusterDescriptor; } + //FIXME need? private void perJobMetricConfigConfig(Configuration configuration, Properties properties){ if(!properties.containsKey(DEFAULT_GATEWAY_CLASS)){ return; @@ -131,7 +128,7 @@ private void perJobMetricConfigConfig(Configuration configuration, Properties pr configuration.setString(PROMGATEWAY_DELETEONSHUTDOWN_KEY, properties.getProperty(PROMGATEWAY_DELETEONSHUTDOWN_KEY)); } - public AbstractYarnClusterDescriptor getClusterDescriptor( + private AbstractYarnClusterDescriptor getClusterDescriptor( Configuration configuration, YarnConfiguration yarnConfiguration, String configurationDirectory) { diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java new file mode 100644 index 000000000..3cdf3f8d4 --- /dev/null +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java @@ -0,0 +1,66 @@ +/* + * 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 com.dtstack.flink.sql.launcher.perjob; + +import com.dtstack.flink.sql.launcher.LauncherOptions; +import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.yarn.AbstractYarnClusterDescriptor; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Properties; + +/** + * per job mode submitter + * Date: 2018/11/17 + * Company: www.dtstack.com + * @author xuchao + */ + +public class PerJobSubmitter { + + private static final Logger LOG = LoggerFactory.getLogger(PerJobSubmitter.class); + + public static String submit(LauncherOptions launcherOptions, JobGraph jobGraph) throws Exception { + + Properties confProperties = PluginUtil.jsonStrToObject(launcherOptions.getConfProp(), Properties.class); + ClusterSpecification clusterSpecification = FLinkPerJobResourceUtil.createClusterSpecification(confProperties); + + PerJobClusterClientBuilder perJobClusterClientBuilder = new PerJobClusterClientBuilder(); + perJobClusterClientBuilder.init(launcherOptions.getYarnconf()); + + String flinkJarPath = launcherOptions.getFlinkJarPath(); + + AbstractYarnClusterDescriptor yarnClusterDescriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(confProperties, flinkJarPath, launcherOptions.getQueue()); + ClusterClient clusterClient = yarnClusterDescriptor.deployJobCluster(clusterSpecification, jobGraph,true); + + String applicationId = clusterClient.getClusterId().toString(); + String flinkJobId = jobGraph.getJobID().toString(); + + String tips = String.format("deploy per_job with appId: %s, jobId: %s", applicationId, flinkJobId); + System.out.println(tips); + LOG.info(tips); + + return applicationId; + } +} From 071e61bd6b5e011ebe4b0fab00e5c0c9ef52b81c Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Sat, 17 Nov 2018 15:07:19 +0800 Subject: [PATCH 137/470] bugfix --- .../main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 90e45d57e..080d3d48b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -106,7 +106,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl }else if (pollObj instanceof JoinInfo){ preIsSideJoin = true; - jionFun(pollObj, localTableCache, sideTableMap, tableEnv, replaceInfoList); + joinFun(pollObj, localTableCache, sideTableMap, tableEnv, replaceInfoList); } } @@ -545,11 +545,11 @@ public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, }else if (pollObj instanceof JoinInfo){ preIsSideJoin = true; - jionFun(pollObj, localTableCache, sideTableMap, tableEnv, replaceInfoList); + joinFun(pollObj, localTableCache, sideTableMap, tableEnv, replaceInfoList); } } } - private void jionFun(Object pollObj, Map localTableCache, + private void joinFun(Object pollObj, Map localTableCache, Map sideTableMap, StreamTableEnvironment tableEnv, List replaceInfoList) throws Exception{ JoinInfo joinInfo = (JoinInfo) pollObj; From ff913ababe5901ea0d9ecc2e9fc8bce2cf6fe068 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 17 Nov 2018 15:13:41 +0800 Subject: [PATCH 138/470] add per job submit mode --- README.md | 15 +++++++++++++-- .../perjob/PerJobClusterClientBuilder.java | 3 ++- 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 3241a57a5..03ed36002 100644 --- a/README.md +++ b/README.md @@ -52,8 +52,9 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * **model** * 描述:执行模式,也就是flink集群的工作模式 * local: 本地模式 - * standalone: 独立部署模式的flink集群 - * yarn: yarn模式的flink集群 + * standalone: 提交到独立部署模式的flink集群 + * yarn: 提交到yarn模式的flink集群(即提交到已有flink集群) + * yarnPer: yarn per_job模式提交(即创建新flink application) * 必选:否 * 默认值:local @@ -118,6 +119,16 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * 描述:指示保存点是否允许非还原状态的标志 * 必选:否 * 默认值:false + +* **flinkJarPath** + * 描述:per_job 模式提交需要指定本地的flink jar存放路径 + * 必选:否 + * 默认值:false + +* **queue** + * 描述:per_job 模式下指定的yarn queue + * 必选:否 + * 默认值:false ## 2 结构 ### 2.1 源表插件 diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java index 7d395f0ac..820160219 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java @@ -77,7 +77,8 @@ public void init(String yarnConfDir){ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, String flinkJarPath, String queue) throws MalformedURLException { Configuration newConf = new Configuration(); - newConf.addAllToProperties(confProp); + //newConf.addAllToProperties(confProp); + confProp.forEach((key, val) -> newConf.setString(key.toString(), val.toString()) ); //perJobMetricConfigConfig(newConf, properties); AbstractYarnClusterDescriptor clusterDescriptor = getClusterDescriptor(newConf, yarnConf, "."); From 32eb60150e3e5481376698805f412a0d4fb16b98 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 17 Nov 2018 17:01:37 +0800 Subject: [PATCH 139/470] modify readme doc 1:add per_job mode start describe 2:add new metric describe --- README.md | 32 ++++++++++++++++++- docs/prometheus.md | 7 ++++ .../perjob/FLinkPerJobResourceUtil.java | 20 ++++++------ .../perjob/PerJobClusterClientBuilder.java | 30 ----------------- 4 files changed, 48 insertions(+), 41 deletions(-) create mode 100644 docs/prometheus.md diff --git a/README.md b/README.md index 03ed36002..50b4c27f0 100644 --- a/README.md +++ b/README.md @@ -98,6 +98,11 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * sql.max.concurrent.checkpoints: 最大并发生成checkpoint数 * sql.checkpoint.cleanup.mode: 默认是不会将checkpoint存储到外部存储,[true(任务cancel之后会删除外部存储)|false(外部存储需要手动删除)] * flinkCheckpointDataURI: 设置checkpoint的外部存储路径,根据实际的需求设定文件路径,hdfs://, file:// + * jobmanager.memory.mb: per_job模式下指定jobmanager的内存大小(单位MB, 默认值:768) + * taskmanager.memory.mb: per_job模式下指定taskmanager的内存大小(单位MB, 默认值:768) + * taskmanager.num: per_job模式下指定taskmanager的实例数(默认1) + * taskmanager.slots:per_job模式下指定每个taskmanager对应的slot数量(默认1) + * [prometheus 相关参数](docs/prometheus.md) per_job可指定metric写入到外部监控组件,以prometheus pushgateway举例 * **flinkconf** @@ -146,8 +151,33 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * [mysql 维表插件](docs/mysqlSide.md) * [mongo 维表插件](docs/mongoSide.md) * [redis 维表插件](docs/redisSide.md) + +## 3 性能指标(新增) + +### kafka插件 +* 业务延迟: flink_taskmanager_job_task_operator_dtEventDelay(单位s) + 数据本身的时间和进入flink的当前时间的差值. + +* 各个输入源的脏数据:flink_taskmanager_job_task_operator_dtDirtyData + 从kafka获取的数据解析失败的视为脏数据 + +* 各Source的数据输入TPS: flink_taskmanager_job_task_operator_dtNumRecordsInRate + kafka接受的记录数(未解析前)/s + +* 各Source的数据输入RPS: flink_taskmanager_job_task_operator_dtNumRecordsInResolveRate + kafka接受的记录数(解析后)/s + +* 各Source的数据输入BPS: flink_taskmanager_job_task_operator_dtNumBytesInRate + kafka接受的字节数/s + +* Kafka作为输入源的各个分区的延迟数: flink_taskmanager_job_task_operator_topic_partition_dtTopicPartitionLag + 当前kafka10,kafka11有采集该指标 + +* 各个输出源RPS: flink_taskmanager_job_task_operator_dtNumRecordsOutRate + 写入的外部记录数/s + -## 3 样例 +## 4 样例 ``` CREATE TABLE MyTable( diff --git a/docs/prometheus.md b/docs/prometheus.md new file mode 100644 index 000000000..a36498b42 --- /dev/null +++ b/docs/prometheus.md @@ -0,0 +1,7 @@ +## 使用 prometheus pushgateway 需要设置的 confProp 参数 +* metrics.reporter.promgateway.class: org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter +* metrics.reporter.promgateway.host: prometheus pushgateway的地址 +* metrics.reporter.promgateway.port:prometheus pushgateway的端口 +* metrics.reporter.promgateway.jobName: 实例名称 +* metrics.reporter.promgateway.randomJobNameSuffix: 是否在实例名称后面添加随机字符串(默认:true) +* metrics.reporter.promgateway.deleteOnShutdown: 是否在停止的时候删除数据(默认false) \ No newline at end of file diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/FLinkPerJobResourceUtil.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/FLinkPerJobResourceUtil.java index 3becc41d8..468039cb2 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/FLinkPerJobResourceUtil.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/FLinkPerJobResourceUtil.java @@ -31,17 +31,17 @@ */ public class FLinkPerJobResourceUtil { - public final static int MIN_JM_MEMORY = 1024; // the minimum memory should be higher than the min heap cutoff - public final static int MIN_TM_MEMORY = 1024; + public final static int MIN_JM_MEMORY = 768; // the minimum memory should be higher than the min heap cutoff + public final static int MIN_TM_MEMORY = 768; public final static String JOBMANAGER_MEMORY_MB = "jobmanager.memory.mb"; public final static String TASKMANAGER_MEMORY_MB = "taskmanager.memory.mb"; - public final static String CONTAINER = "container"; - public final static String SLOTS = "slots"; + public final static String NUMBER_TASK_MANAGERS = "taskmanager.num"; + public final static String SLOTS_PER_TASKMANAGER = "taskmanager.slots"; public static ClusterSpecification createClusterSpecification(Properties confProperties) { - int jobmanagerMemoryMb = 1024; - int taskmanagerMemoryMb = 1024; + int jobmanagerMemoryMb = 768; + int taskmanagerMemoryMb = 768; int numberTaskManagers = 1; int slotsPerTaskManager = 1; @@ -60,12 +60,12 @@ public static ClusterSpecification createClusterSpecification(Properties confPro } } - if (confProperties.containsKey(CONTAINER)){ - numberTaskManagers = MathUtil.getIntegerVal(confProperties.get(CONTAINER)); + if (confProperties.containsKey(NUMBER_TASK_MANAGERS)){ + numberTaskManagers = MathUtil.getIntegerVal(confProperties.get(NUMBER_TASK_MANAGERS)); } - if (confProperties.containsKey(SLOTS)){ - slotsPerTaskManager = MathUtil.getIntegerVal(confProperties.get(SLOTS)); + if (confProperties.containsKey(SLOTS_PER_TASKMANAGER)){ + slotsPerTaskManager = MathUtil.getIntegerVal(confProperties.get(SLOTS_PER_TASKMANAGER)); } } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java index 820160219..8ddcd541d 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java @@ -44,20 +44,6 @@ public class PerJobClusterClientBuilder { - public static final String DEFAULT_GATEWAY_CLASS = "org.apache.flink.metrics.prometheus.PrometheusPushGatewayReporter"; - - public static final String PROMGATEWAY_CLASS_KEY = "metrics.reporter.promgateway.class"; - - public static final String PROMGATEWAY_HOST_KEY = "metrics.reporter.promgateway.host"; - - public static final String PROMGATEWAY_PORT_KEY = "metrics.reporter.promgateway.port"; - - public static final String PROMGATEWAY_JOBNAME_KEY = "metrics.reporter.promgateway.jobName"; - - public static final String PROMGATEWAY_RANDOMJOBNAMESUFFIX_KEY = "metrics.reporter.promgateway.randomJobNameSuffix"; - - public static final String PROMGATEWAY_DELETEONSHUTDOWN_KEY = "metrics.reporter.promgateway.deleteOnShutdown"; - private YarnClient yarnClient; private YarnConfiguration yarnConf; @@ -77,10 +63,8 @@ public void init(String yarnConfDir){ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, String flinkJarPath, String queue) throws MalformedURLException { Configuration newConf = new Configuration(); - //newConf.addAllToProperties(confProp); confProp.forEach((key, val) -> newConf.setString(key.toString(), val.toString()) ); - //perJobMetricConfigConfig(newConf, properties); AbstractYarnClusterDescriptor clusterDescriptor = getClusterDescriptor(newConf, yarnConf, "."); if (StringUtils.isNotBlank(flinkJarPath)) { @@ -115,20 +99,6 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties co return clusterDescriptor; } - //FIXME need? - private void perJobMetricConfigConfig(Configuration configuration, Properties properties){ - if(!properties.containsKey(DEFAULT_GATEWAY_CLASS)){ - return; - } - - configuration.setString(PROMGATEWAY_CLASS_KEY, properties.getProperty(PROMGATEWAY_CLASS_KEY)); - configuration.setString(PROMGATEWAY_HOST_KEY, properties.getProperty(PROMGATEWAY_HOST_KEY)); - configuration.setString(PROMGATEWAY_PORT_KEY, properties.getProperty(PROMGATEWAY_PORT_KEY)); - configuration.setString(PROMGATEWAY_JOBNAME_KEY, properties.getProperty(PROMGATEWAY_JOBNAME_KEY)); - configuration.setString(PROMGATEWAY_RANDOMJOBNAMESUFFIX_KEY, properties.getProperty(PROMGATEWAY_RANDOMJOBNAMESUFFIX_KEY)); - configuration.setString(PROMGATEWAY_DELETEONSHUTDOWN_KEY, properties.getProperty(PROMGATEWAY_DELETEONSHUTDOWN_KEY)); - } - private AbstractYarnClusterDescriptor getClusterDescriptor( Configuration configuration, YarnConfiguration yarnConfiguration, From e88fcf0a5b27bfd8346280208f175d47ea3697b3 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 17 Nov 2018 17:37:14 +0800 Subject: [PATCH 140/470] exclude slf4j --- hbase/hbase-side/hbase-all-side/pom.xml | 2 +- hbase/hbase-side/hbase-async-side/pom.xml | 2 +- hbase/hbase-sink/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/hbase/hbase-side/hbase-all-side/pom.xml b/hbase/hbase-side/hbase-all-side/pom.xml index 6e344dd5d..7617a76e0 100644 --- a/hbase/hbase-side/hbase-all-side/pom.xml +++ b/hbase/hbase-side/hbase-all-side/pom.xml @@ -51,7 +51,7 @@ org.apache.hadoop:hadoop-common org.apache.hadoop:hadoop-auth org.apache.hadoop:hadoop-mapreduce-client-core - org.slf4j + org.slf4j:* diff --git a/hbase/hbase-side/hbase-async-side/pom.xml b/hbase/hbase-side/hbase-async-side/pom.xml index 4da76c68c..193b720a0 100644 --- a/hbase/hbase-side/hbase-async-side/pom.xml +++ b/hbase/hbase-side/hbase-async-side/pom.xml @@ -52,7 +52,7 @@ org.apache.hadoop:hadoop-common org.apache.hadoop:hadoop-auth org.apache.hadoop:hadoop-mapreduce-client-core - org.slf4j + org.slf4j:* diff --git a/hbase/hbase-sink/pom.xml b/hbase/hbase-sink/pom.xml index 9ec6d161c..26ead3574 100644 --- a/hbase/hbase-sink/pom.xml +++ b/hbase/hbase-sink/pom.xml @@ -27,7 +27,7 @@ - org.slf4j:slf4j-log4j12 + org.slf4j:* org.apache.hadoop:hadoop-common org.apache.hadoop:hadoop-auth org.apache.hadoop:hadoop-mapreduce-client-core From e4060c6f931dd16ed93e22a77a9459d5bbf27cee Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 19 Nov 2018 14:08:21 +0800 Subject: [PATCH 141/470] support json --- launcher/pom.xml | 7 +++ .../flink/sql/launcher/LauncherMain.java | 45 ++++++++++++++++++- 2 files changed, 51 insertions(+), 1 deletion(-) diff --git a/launcher/pom.xml b/launcher/pom.xml index 927f6e2f3..2f8140fd5 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -29,6 +29,13 @@ log4j 1.2.17 + + + com.alibaba + fastjson + 1.2.7 + + diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 8e5832294..10bc09475 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -21,12 +21,18 @@ package com.dtstack.flink.sql.launcher; import avro.shaded.com.google.common.collect.Lists; +import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson.TypeReference; import com.dtstack.flink.sql.Main; import com.dtstack.flink.sql.launcher.perjob.PerJobSubmitter; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; -import java.io.File; + +import java.io.*; +import java.util.LinkedList; import java.util.List; +import java.util.Map; + import com.dtstack.flink.sql.ClusterMode; import org.apache.flink.client.program.PackagedProgramUtils; import org.apache.flink.configuration.Configuration; @@ -53,6 +59,9 @@ private static String getLocalCoreJarPath(String localSqlRootJar){ } public static void main(String[] args) throws Exception { + if (args.length==1 && args[0].endsWith(".json")){ + args = parseJson(args); + } LauncherOptionParser optionParser = new LauncherOptionParser(args); LauncherOptions launcherOptions = optionParser.getLauncherOptions(); String mode = launcherOptions.getMode(); @@ -85,4 +94,38 @@ public static void main(String[] args) throws Exception { System.out.println("---submit end----"); } + + private static String[] parseJson(String[] args) { + BufferedReader reader = null; + String lastStr = ""; + try{ + FileInputStream fileInputStream = new FileInputStream(args[0]); + InputStreamReader inputStreamReader = new InputStreamReader(fileInputStream, "UTF-8"); + reader = new BufferedReader(inputStreamReader); + String tempString = null; + while((tempString = reader.readLine()) != null){ + lastStr += tempString; + } + reader.close(); + }catch(IOException e){ + e.printStackTrace(); + }finally{ + if(reader != null){ + try { + reader.close(); + } catch (IOException e) { + e.printStackTrace(); + } + } + } + Map map = JSON.parseObject(lastStr, new TypeReference>(){} ); + List list = new LinkedList<>(); + + for (Map.Entry entry : map.entrySet()) { + list.add("-" + entry.getKey()); + list.add(entry.getValue().toString()); + } + String[] array = list.toArray(new String[list.size()]); + return array; + } } From 87e048fecaf8aba17f54d06a64c9ffa45b3b2b16 Mon Sep 17 00:00:00 2001 From: zhihui-ge <2972333955@qq.com> Date: Mon, 19 Nov 2018 20:00:41 +0800 Subject: [PATCH 142/470] update kafka source --- docs/kafkaSource.md | 2 +- .../flink/sql/source/kafka/KafkaSource.java | 19 ++++++++++++++++++- .../flink/sql/source/kafka/KafkaSource.java | 19 ++++++++++++++++++- .../flink/sql/source/kafka/KafkaSource.java | 17 +++++++++++++++++ 4 files changed, 54 insertions(+), 3 deletions(-) diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index 2968dbe22..f382ba9ad 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -38,7 +38,7 @@ CREATE TABLE tableName( |bootstrapServers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| |zookeeperQuorum | kafka zk地址信息(多个之间用逗号分隔)|是|| |topic | 需要读取的 topic 名称|是|| -|offsetReset | 读取的topic 的offset初始位置[latest\|earliest]|否|latest| +|offsetReset | 读取的topic 的offset初始位置[latest\|earliest\|指定offset值({"0":12312,"1":12321,"2":12312},{"partition_no":offset_value})]|否|latest| |parallelism | 并行度设置|否|1| ## 5.样例: diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 8486b9883..00988518a 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -23,16 +23,20 @@ import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.types.Row; +import java.util.HashMap; +import java.util.Map; import java.util.Properties; /** @@ -76,7 +80,20 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv //earliest,latest if("earliest".equalsIgnoreCase(kafka09SourceTableInfo.getOffsetReset())){ kafkaSrc.setStartFromEarliest(); - }else{ + }else if(kafka09SourceTableInfo.getOffsetReset().startsWith("{")){ + try { + // {"0":12312,"1":12321,"2":12312} + Properties properties = PluginUtil.jsonStrToObject(kafka09SourceTableInfo.getOffsetReset(), Properties.class); + Map offsetMap = PluginUtil.ObjectToMap(properties); + Map specificStartupOffsets = new HashMap<>(); + for(Map.Entry entry:offsetMap.entrySet()){ + specificStartupOffsets.put(new KafkaTopicPartition(topicName,Integer.valueOf(entry.getKey())),Long.valueOf(entry.getValue().toString())); + } + kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); + } catch (Exception e) { + throw new RuntimeException("not support offsetReset type:" + kafka09SourceTableInfo.getOffsetReset()); + } + }else { kafkaSrc.setStartFromLatest(); } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index ffc42f7ef..c9eaf05bc 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -23,15 +23,19 @@ import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.types.Row; +import java.util.HashMap; +import java.util.Map; import java.util.Properties; /** @@ -75,7 +79,20 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv //earliest,latest if("earliest".equalsIgnoreCase(kafka010SourceTableInfo.getOffsetReset())){ kafkaSrc.setStartFromEarliest(); - }else{ + }else if(kafka010SourceTableInfo.getOffsetReset().startsWith("{")){ + try { + // {"0":12312,"1":12321,"2":12312} + Properties properties = PluginUtil.jsonStrToObject(kafka010SourceTableInfo.getOffsetReset(), Properties.class); + Map offsetMap = PluginUtil.ObjectToMap(properties); + Map specificStartupOffsets = new HashMap<>(); + for(Map.Entry entry:offsetMap.entrySet()){ + specificStartupOffsets.put(new KafkaTopicPartition(topicName,Integer.valueOf(entry.getKey())),Long.valueOf(entry.getValue().toString())); + } + kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); + } catch (Exception e) { + throw new RuntimeException("not support offsetReset type:" + kafka010SourceTableInfo.getOffsetReset()); + } + }else { kafkaSrc.setStartFromLatest(); } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 8bfec8107..5f6146cc9 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -23,15 +23,19 @@ import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.types.Row; +import java.util.HashMap; +import java.util.Map; import java.util.Properties; /** @@ -75,6 +79,19 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv //earliest,latest if("earliest".equalsIgnoreCase(kafka011SourceTableInfo.getOffsetReset())){ kafkaSrc.setStartFromEarliest(); + }else if(kafka011SourceTableInfo.getOffsetReset().startsWith("{")){ + try { + // {"0":12312,"1":12321,"2":12312} + Properties properties = PluginUtil.jsonStrToObject(kafka011SourceTableInfo.getOffsetReset(), Properties.class); + Map offsetMap = PluginUtil.ObjectToMap(properties); + Map specificStartupOffsets = new HashMap<>(); + for(Map.Entry entry:offsetMap.entrySet()){ + specificStartupOffsets.put(new KafkaTopicPartition(topicName,Integer.valueOf(entry.getKey())),Long.valueOf(entry.getValue().toString())); + } + kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); + } catch (Exception e) { + throw new RuntimeException("not support offsetReset type:" + kafka011SourceTableInfo.getOffsetReset()); + } }else{ kafkaSrc.setStartFromLatest(); } From 30ba56d7a3be42cfb1a5516a1756121533ec6096 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 19 Nov 2018 22:13:05 +0800 Subject: [PATCH 143/470] fix jar conflict --- hbase/hbase-side/hbase-all-side/pom.xml | 4 ++++ hbase/pom.xml | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/hbase/hbase-side/hbase-all-side/pom.xml b/hbase/hbase-side/hbase-all-side/pom.xml index 7617a76e0..32850176b 100644 --- a/hbase/hbase-side/hbase-all-side/pom.xml +++ b/hbase/hbase-side/hbase-all-side/pom.xml @@ -29,6 +29,10 @@ log4j log4j + + io.netty + netty + diff --git a/hbase/pom.xml b/hbase/pom.xml index 48c8162a0..835457e29 100644 --- a/hbase/pom.xml +++ b/hbase/pom.xml @@ -43,6 +43,10 @@ log4j log4j + + io.netty + netty + From 23a2652557db213c01153f62370f4d1a84b1b3ed Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 20 Nov 2018 21:29:33 +0800 Subject: [PATCH 144/470] add init log info to hbase sink --- .../dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java index a75f00da7..cc95d1e75 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java @@ -32,6 +32,8 @@ import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.text.SimpleDateFormat; @@ -44,6 +46,8 @@ */ public class HbaseOutputFormat extends RichOutputFormat { + private static final Logger LOG = LoggerFactory.getLogger(HbaseOutputFormat.class); + private String host; private String zkParent; private String[] rowkey; @@ -63,17 +67,21 @@ public class HbaseOutputFormat extends RichOutputFormat { @Override public void configure(Configuration parameters) { + LOG.warn("---configure---"); conf = HBaseConfiguration.create(); conf.set("hbase.zookeeper.quorum", host); if(zkParent != null && !"".equals(zkParent)){ conf.set("zookeeper.znode.parent", zkParent); } + LOG.warn("---configure end ---"); } @Override public void open(int taskNumber, int numTasks) throws IOException { + LOG.warn("---open---"); conn = ConnectionFactory.createConnection(conf); table = conn.getTable(TableName.valueOf(tableName)); + LOG.warn("---open end(get table from hbase) ---"); } @Override From 2338cc250338c5bc384802f59f939b9de0023d87 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 22 Nov 2018 10:21:08 +0800 Subject: [PATCH 145/470] TableInfoParser remove sourceTableInfoMap static --- .../java/com/dtstack/flink/sql/parser/SqlParser.java | 10 +++++----- ...ableInfoParserFactory.java => TableInfoParser.java} | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) rename core/src/main/java/com/dtstack/flink/sql/table/{TableInfoParserFactory.java => TableInfoParser.java} (91%) diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java index 4bd17c65e..7ab616632 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java @@ -22,7 +22,7 @@ import com.dtstack.flink.sql.enums.ETableType; import com.dtstack.flink.sql.table.TableInfo; -import com.dtstack.flink.sql.table.TableInfoParserFactory; +import com.dtstack.flink.sql.table.TableInfoParser; import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.collect.Lists; @@ -75,7 +75,7 @@ public static SqlTree parseSql(String sql) throws Exception { List sqlArr = DtStringUtil.splitIgnoreQuota(sql, SQL_DELIMITER); SqlTree sqlTree = new SqlTree(); - + TableInfoParser tableInfoParser = new TableInfoParser(); for(String childSql : sqlArr){ if(Strings.isNullOrEmpty(childSql)){ continue; @@ -112,7 +112,7 @@ public static SqlTree parseSql(String sql) throws Exception { throw new RuntimeException("can't find table " + tableName); } - TableInfo tableInfo = TableInfoParserFactory.parseWithTableType(ETableType.SOURCE.getType(), + TableInfo tableInfo = tableInfoParser.parseWithTableType(ETableType.SOURCE.getType(), createTableResult, LOCAL_SQL_PLUGIN_ROOT); sqlTree.addTableInfo(tableName, tableInfo); } @@ -125,7 +125,7 @@ public static SqlTree parseSql(String sql) throws Exception { throw new RuntimeException("can't find table " + tableName); } - TableInfo tableInfo = TableInfoParserFactory.parseWithTableType(ETableType.SINK.getType(), + TableInfo tableInfo = tableInfoParser.parseWithTableType(ETableType.SINK.getType(), createTableResult, LOCAL_SQL_PLUGIN_ROOT); sqlTree.addTableInfo(tableName, tableInfo); } @@ -141,7 +141,7 @@ public static SqlTree parseSql(String sql) throws Exception { throw new RuntimeException("can't find table " + tableName); } - TableInfo tableInfo = TableInfoParserFactory.parseWithTableType(ETableType.SOURCE.getType(), + TableInfo tableInfo = tableInfoParser.parseWithTableType(ETableType.SOURCE.getType(), createTableResult, LOCAL_SQL_PLUGIN_ROOT); sqlTree.addTableInfo(tableName, tableInfo); } diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java similarity index 91% rename from core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java rename to core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java index f74e7b620..b3a07d6d5 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParserFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java @@ -42,7 +42,7 @@ * @author xuchao */ -public class TableInfoParserFactory { +public class TableInfoParser { private final static String TYPE_KEY = "type"; @@ -50,14 +50,14 @@ public class TableInfoParserFactory { private final static Pattern SIDE_PATTERN = Pattern.compile(SIDE_TABLE_SIGN); - private static Map sourceTableInfoMap = Maps.newConcurrentMap(); + private Map sourceTableInfoMap = Maps.newConcurrentMap(); - private static Map targetTableInfoMap = Maps.newConcurrentMap(); + private Map targetTableInfoMap = Maps.newConcurrentMap(); - private static Map sideTableInfoMap = Maps.newConcurrentMap(); + private Map sideTableInfoMap = Maps.newConcurrentMap(); //Parsing loaded plugin - public static TableInfo parseWithTableType(int tableType, CreateTableParser.SqlParserResult parserResult, + public TableInfo parseWithTableType(int tableType, CreateTableParser.SqlParserResult parserResult, String localPluginRoot) throws Exception { AbsTableParser absTableParser = null; Map props = parserResult.getPropMap(); From a29a1d25ac621f2b1179fef28ab12e2481695cbe Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 22 Nov 2018 10:57:36 +0800 Subject: [PATCH 146/470] fix hbase thread bug --- .../com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java index 75c65b73b..33dd9b9a9 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java @@ -85,10 +85,7 @@ public HbaseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List(), new DTThreadFactory("hbase-aysnc")); - hBaseClient = new HBaseClient(hbaseSideTableInfo.getHost(), hbaseSideTableInfo.getParent(), executorService); + hBaseClient = new HBaseClient(hbaseSideTableInfo.getHost(), hbaseSideTableInfo.getParent()); try { Deferred deferred = hBaseClient.ensureTableExists(tableName) From b14b7b62e10d8c214d6d6a3cc27e39ca9ba7b8a4 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 22 Nov 2018 16:02:16 +0800 Subject: [PATCH 147/470] support the pool configuration of redis --- .../sql/sink/redis/RedisOutputFormat.java | 110 +++++++++++++----- .../flink/sql/sink/redis/RedisSink.java | 24 +++- .../sql/sink/redis/table/RedisSinkParser.java | 9 +- .../sql/sink/redis/table/RedisTableInfo.java | 79 +++++++++++-- 4 files changed, 182 insertions(+), 40 deletions(-) diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java index 742b2a377..90131d34a 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java @@ -39,6 +39,16 @@ public class RedisOutputFormat extends RichOutputFormat { private String password; + private int redisType; + + private String maxTotal; + + private String maxIdle; + + private String minIdle; + + private String masterName; + protected String[] fieldNames; protected TypeInformation[] fieldTypes; @@ -53,6 +63,8 @@ public class RedisOutputFormat extends RichOutputFormat { private JedisSentinelPool jedisSentinelPool; + private JedisCluster jedisCluster; + private GenericObjectPoolConfig poolConfig; private RedisOutputFormat(){ @@ -67,26 +79,48 @@ public void open(int taskNumber, int numTasks) throws IOException { establishConnection(); } + private GenericObjectPoolConfig setPoolConfig(String maxTotal, String maxIdle, String minIdle){ + GenericObjectPoolConfig config = new GenericObjectPoolConfig(); + if (maxTotal != null){ + config.setMaxTotal(Integer.parseInt(maxTotal)); + } + if (maxIdle != null){ + config.setMaxIdle(Integer.parseInt(maxIdle)); + } + if (minIdle != null){ + config.setMinIdle(Integer.parseInt(minIdle)); + } + return config; + } + private void establishConnection() { - poolConfig = new GenericObjectPoolConfig(); + poolConfig = setPoolConfig(maxTotal, maxIdle, minIdle); String[] nodes = url.split(","); - if (nodes.length > 1){ - //cluster - Set addresses = new HashSet<>(); - Set ipPorts = new HashSet<>(); - for (String ipPort : nodes) { - ipPorts.add(ipPort); - String[] ipPortPair = ipPort.split(":"); - addresses.add(new HostAndPort(ipPortPair[0].trim(), Integer.valueOf(ipPortPair[1].trim()))); - } - jedisSentinelPool = new JedisSentinelPool("Master", ipPorts, poolConfig, timeout, password, Integer.parseInt(database)); - jedis = jedisSentinelPool.getResource(); - } else { - String[] ipPortPair = nodes[0].split(":"); - String ip = ipPortPair[0]; - String port = ipPortPair[1]; - pool = new JedisPool(poolConfig, ip, Integer.parseInt(port), timeout, password, Integer.parseInt(database)); - jedis = pool.getResource(); + String[] firstIpPort = nodes[0].split(":"); + String firstIp = firstIpPort[0]; + String firstPort = firstIpPort[1]; + Set addresses = new HashSet<>(); + Set ipPorts = new HashSet<>(); + for (String ipPort : nodes) { + ipPorts.add(ipPort); + String[] ipPortPair = ipPort.split(":"); + addresses.add(new HostAndPort(ipPortPair[0].trim(), Integer.valueOf(ipPortPair[1].trim()))); + } + + switch (redisType){ + //单机 + case 1: + pool = new JedisPool(poolConfig, firstIp, Integer.parseInt(firstPort), timeout, password, Integer.parseInt(database)); + jedis = pool.getResource(); + break; + //哨兵 + case 2: + jedisSentinelPool = new JedisSentinelPool(masterName, ipPorts, poolConfig, timeout, password, Integer.parseInt(database)); + jedis = jedisSentinelPool.getResource(); + break; + //集群 + case 3: + jedisCluster = new JedisCluster(addresses, timeout, timeout,1, poolConfig); } } @@ -126,7 +160,12 @@ public void writeRecord(Tuple2 record) throws IOException { for (int i = 0; i < fieldNames.length; i++) { StringBuilder key = new StringBuilder(); key.append(tableName).append(":").append(perKey).append(":").append(fieldNames[i]); - jedis.set(key.toString(), (String) row.getField(i)); + if (redisType != 3){ + jedis.set(key.toString(), (String) row.getField(i)); + } else { + jedisCluster.set(key.toString(), (String) row.getField(i)); + } + } } @@ -192,23 +231,40 @@ public RedisOutputFormatBuilder setTimeout(int timeout){ return this; } + public RedisOutputFormatBuilder setRedisType(int redisType){ + redisOutputFormat.redisType = redisType; + return this; + } + + public RedisOutputFormatBuilder setMaxTotal(String maxTotal){ + redisOutputFormat.maxTotal = maxTotal; + return this; + } + + public RedisOutputFormatBuilder setMaxIdle(String maxIdle){ + redisOutputFormat.maxIdle = maxIdle; + return this; + } + + public RedisOutputFormatBuilder setMinIdle(String minIdle){ + redisOutputFormat.minIdle = minIdle; + return this; + } + + public RedisOutputFormatBuilder setMasterName(String masterName){ + redisOutputFormat.masterName = masterName; + return this; + } + public RedisOutputFormat finish(){ if (redisOutputFormat.url == null){ throw new IllegalArgumentException("No URL supplied."); } - if (redisOutputFormat.database == null){ - throw new IllegalArgumentException("No database supplied."); - } - if (redisOutputFormat.tableName == null){ throw new IllegalArgumentException("No tablename supplied."); } - if (redisOutputFormat.password == null){ - throw new IllegalArgumentException("No password supplied."); - } - return redisOutputFormat; } } diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java index c11d96b8e..3e7300929 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java @@ -52,6 +52,16 @@ public class RedisSink implements RetractStreamTableSink, IStreamSinkGener< protected int timeout; + protected int redisType; + + protected String maxTotal; + + protected String maxIdle; + + protected String minIdle; + + protected String masterName; + public RedisSink(){ } @@ -63,7 +73,12 @@ public RedisSink genStreamSink(TargetTableInfo targetTableInfo) { this.database = redisTableInfo.getDatabase(); this.password = redisTableInfo.getPassword(); this.tableName = redisTableInfo.getTablename(); - this.primaryKeys = targetTableInfo.getPrimaryKeys(); + this.primaryKeys = redisTableInfo.getPrimaryKeys(); + this.redisType = redisTableInfo.getRedisType(); + this.maxTotal = redisTableInfo.getMaxTotal(); + this.maxIdle = redisTableInfo.getMaxIdle(); + this.minIdle = redisTableInfo.getMinIdle(); + this.masterName = redisTableInfo.getMasterName(); return this; } @@ -82,7 +97,12 @@ public void emitDataStream(DataStream> dataStream) { .setFieldNames(this.fieldNames) .setFieldTypes(this.fieldTypes) .setPrimaryKeys(this.primaryKeys) - .setTimeout(this.timeout); + .setTimeout(this.timeout) + .setRedisType(this.redisType) + .setMaxTotal(this.maxTotal) + .setMaxIdle(this.maxIdle) + .setMinIdle(this.minIdle) + .setMasterName(this.masterName); RedisOutputFormat redisOutputFormat = builder.finish(); RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(redisOutputFormat); dataStream.addSink(richSinkFunction); diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java index 1bd74eb65..cecf70f95 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java @@ -34,9 +34,12 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Thu, 22 Nov 2018 16:11:32 +0800 Subject: [PATCH 148/470] add metric --- .../flink/sql/sink/redis/RedisOutputFormat.java | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java index 90131d34a..9e16a9b8f 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java @@ -18,11 +18,15 @@ package com.dtstack.flink.sql.sink.redis; +import com.dtstack.flink.sql.metric.MetricConstant; import org.apache.commons.pool2.impl.GenericObjectPoolConfig; import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; import org.apache.flink.types.Row; import redis.clients.jedis.*; @@ -67,6 +71,10 @@ public class RedisOutputFormat extends RichOutputFormat { private GenericObjectPoolConfig poolConfig; + private transient Counter outRecords; + + private transient Meter outRecordsRate; + private RedisOutputFormat(){ } @Override @@ -77,6 +85,7 @@ public void configure(Configuration parameters) { @Override public void open(int taskNumber, int numTasks) throws IOException { establishConnection(); + initMetric(); } private GenericObjectPoolConfig setPoolConfig(String maxTotal, String maxIdle, String minIdle){ @@ -93,6 +102,11 @@ private GenericObjectPoolConfig setPoolConfig(String maxTotal, String maxIdle, S return config; } + private void initMetric(){ + outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); + } + private void establishConnection() { poolConfig = setPoolConfig(maxTotal, maxIdle, minIdle); String[] nodes = url.split(","); @@ -165,8 +179,8 @@ public void writeRecord(Tuple2 record) throws IOException { } else { jedisCluster.set(key.toString(), (String) row.getField(i)); } - } + outRecords.inc(); } @Override From 7e285ea7b8c7b8a51fffc905ddcd8291762da1b6 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 22 Nov 2018 17:17:28 +0800 Subject: [PATCH 149/470] modify HBaseClient init ThreadPoolExecutor pool size --- .../flink/sql/side/hbase/HbaseAsyncReqRow.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java index 33dd9b9a9..fe227bb6a 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java @@ -63,7 +63,12 @@ public class HbaseAsyncReqRow extends AsyncReqRow { private static final Logger LOG = LoggerFactory.getLogger(HbaseAsyncReqRow.class); - private static final int HBASE_WORKER_POOL_SIZE = 10; + //match to the rule of netty3 + private static final int DEFAULT_BOSS_THREADS = 1; + + private static final int DEFAULT_IO_THREADS = Runtime.getRuntime().availableProcessors() * 2; + + private static final int DEFAULT_POOL_SIZE = DEFAULT_IO_THREADS + DEFAULT_BOSS_THREADS; private transient HBaseClient hBaseClient; @@ -85,7 +90,11 @@ public HbaseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List(), new DTThreadFactory("hbase-aysnc")); + + hBaseClient = new HBaseClient(hbaseSideTableInfo.getHost(), hbaseSideTableInfo.getParent(), executorService); try { Deferred deferred = hBaseClient.ensureTableExists(tableName) From 8af2a84fc621ab95f262e6b94c1c72afd4abe679 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 22 Nov 2018 21:50:15 +0800 Subject: [PATCH 150/470] support the pool configuration of redis --- .../flink/sql/side/redis/RedisAllReqRow.java | 162 +++++++++++++----- .../sql/side/redis/RedisAsyncReqRow.java | 63 +++++-- .../sql/side/redis/RedisAsyncSideInfo.java | 1 - .../sql/side/redis/table/RedisSideParser.java | 9 +- .../side/redis/table/RedisSideTableInfo.java | 75 +++++++- .../sql/sink/redis/RedisOutputFormat.java | 21 ++- .../flink/sql/sink/redis/RedisSink.java | 1 + .../sql/sink/redis/table/RedisSinkParser.java | 4 +- .../sql/sink/redis/table/RedisTableInfo.java | 2 +- 9 files changed, 266 insertions(+), 72 deletions(-) diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java index 5b9ebbe10..e0e0a7d17 100644 --- a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java @@ -31,6 +31,8 @@ import org.slf4j.LoggerFactory; import redis.clients.jedis.*; +import java.io.Closeable; +import java.io.IOException; import java.sql.SQLException; import java.sql.Timestamp; import java.util.*; @@ -45,8 +47,6 @@ public class RedisAllReqRow extends AllReqRow{ private static final int CONN_RETRY_NUM = 3; - private static final int TIMEOUT = 10000; - private JedisPool pool; private JedisSentinelPool jedisSentinelPool; @@ -147,13 +147,13 @@ private String buildKey(Map inputParams) { } private void loadData(Map> tmpCache) throws SQLException { - Jedis jedis = null; + JedisCommands jedis = null; try { for(int i=0; i> tmpCache) throws SQLExcep } } - String perKey = tableInfo.getTableName() + "*"; - Set keys = jedis.keys(perKey); - List newPerKeys = new LinkedList<>(); - for (String key : keys){ - String[] splitKey = key.split(":"); - String newKey = splitKey[0] + ":" + splitKey[1] + ":" + splitKey[2]; - newPerKeys.add(newKey); - } - List list = newPerKeys.stream().distinct().collect(Collectors.toList()); - for(String key : list){ - Map kv = Maps.newHashMap(); - String[] primaryKv = key.split(":"); - kv.put(primaryKv[1], primaryKv[2]); - - String pattern = key + "*"; - Set realKeys = jedis.keys(pattern); - for (String realKey : realKeys){ - kv.put(realKey.split(":")[3], jedis.get(realKey)); + if (tableInfo.getRedisType() != 3){ + String perKey = tableInfo.getTableName() + "*"; + Set keys = ((Jedis) jedis).keys(perKey); + List newPerKeys = new LinkedList<>(); + for (String key : keys){ + String[] splitKey = key.split(":"); + String newKey = splitKey[0] + ":" + splitKey[1] + ":" + splitKey[2]; + newPerKeys.add(newKey); + } + List list = newPerKeys.stream().distinct().collect(Collectors.toList()); + for(String key : list){ + Map kv = Maps.newHashMap(); + String[] primaryKv = key.split(":"); + kv.put(primaryKv[1], primaryKv[2]); + String pattern = key + "*"; + Set realKeys = ((Jedis) jedis).keys(pattern); + for (String realKey : realKeys){ + kv.put(realKey.split(":")[3], jedis.get(realKey)); + } + tmpCache.put(key, kv); + } + } else { + String perKey = tableInfo.getTableName() + "*"; + Set keys = keys((JedisCluster) jedis, perKey); + List newPerKeys = new LinkedList<>(); + for (String key : keys){ + String[] splitKey = key.split(":"); + String newKey = splitKey[0] + ":" + splitKey[1] + ":" + splitKey[2]; + newPerKeys.add(newKey); + } + List list = newPerKeys.stream().distinct().collect(Collectors.toList()); + for(String key : list){ + Map kv = Maps.newHashMap(); + String[] primaryKv = key.split(":"); + kv.put(primaryKv[1], primaryKv[2]); + String pattern = key + "*"; + Set realKeys = keys((JedisCluster) jedis, pattern); + for (String realKey : realKeys){ + kv.put(realKey.split(":")[3], jedis.get(realKey)); + } + tmpCache.put(key, kv); } - tmpCache.put(key, kv); } @@ -197,7 +219,11 @@ private void loadData(Map> tmpCache) throws SQLExcep LOG.error("", e); } finally { if (jedis != null){ - jedis.close(); + try { + ((Closeable) jedis).close(); + } catch (IOException e) { + e.printStackTrace(); + } } if (jedisSentinelPool != null) { jedisSentinelPool.close(); @@ -208,26 +234,78 @@ private void loadData(Map> tmpCache) throws SQLExcep } } - private Jedis getJedis(String url, String password, String database){ - JedisPoolConfig poolConfig = new JedisPoolConfig(); + private JedisCommands getJedis(RedisSideTableInfo tableInfo) { + String url = tableInfo.getUrl(); + String password = tableInfo.getPassword(); + String database = tableInfo.getDatabase(); + int timeout = tableInfo.getTimeout(); + if (timeout == 0){ + timeout = 1000; + } + String[] nodes = url.split(","); - if (nodes.length > 1){ - //cluster - Set addresses = new HashSet<>(); - Set ipPorts = new HashSet<>(); - for (String ipPort : nodes) { - ipPorts.add(ipPort); - String[] ipPortPair = ipPort.split(":"); - addresses.add(new HostAndPort(ipPortPair[0].trim(), Integer.valueOf(ipPortPair[1].trim()))); + String[] firstIpPort = nodes[0].split(":"); + String firstIp = firstIpPort[0]; + String firstPort = firstIpPort[1]; + Set addresses = new HashSet<>(); + Set ipPorts = new HashSet<>(); + for (String ipPort : nodes) { + ipPorts.add(ipPort); + String[] ipPortPair = ipPort.split(":"); + addresses.add(new HostAndPort(ipPortPair[0].trim(), Integer.valueOf(ipPortPair[1].trim()))); + } + if (timeout == 0){ + timeout = 1000; + } + JedisCommands jedis = null; + GenericObjectPoolConfig poolConfig = setPoolConfig(tableInfo.getMaxTotal(), tableInfo.getMaxIdle(), tableInfo.getMinIdle()); + switch (tableInfo.getRedisType()){ + //单机 + case 1: + pool = new JedisPool(poolConfig, firstIp, Integer.parseInt(firstPort), timeout, password, Integer.parseInt(database)); + jedis = pool.getResource(); + break; + //哨兵 + case 2: + jedisSentinelPool = new JedisSentinelPool(tableInfo.getMasterName(), ipPorts, poolConfig, timeout, password, Integer.parseInt(database)); + jedis = jedisSentinelPool.getResource(); + break; + //集群 + case 3: + jedis = new JedisCluster(addresses, timeout, timeout,1, poolConfig); + } + + return jedis; + } + + private Set keys(JedisCluster jedisCluster, String pattern){ + Set keys = new TreeSet<>(); + Map clusterNodes = jedisCluster.getClusterNodes(); + for(String k : clusterNodes.keySet()){ + JedisPool jp = clusterNodes.get(k); + Jedis connection = jp.getResource(); + try { + keys.addAll(connection.keys(pattern)); + } catch (Exception e){ + LOG.error("Getting keys error: {}", e); + } finally { + connection.close(); } - jedisSentinelPool = new JedisSentinelPool("Master", ipPorts, poolConfig, TIMEOUT, password, Integer.parseInt(database)); - return jedisSentinelPool.getResource(); - } else { - String[] ipPortPair = nodes[0].split(":"); - String ip = ipPortPair[0]; - String port = ipPortPair[1]; - pool = new JedisPool(poolConfig, ip, Integer.parseInt(port), TIMEOUT, password, Integer.parseInt(database)); - return pool.getResource(); } + return keys; + } + + private GenericObjectPoolConfig setPoolConfig(String maxTotal, String maxIdle, String minIdle){ + GenericObjectPoolConfig config = new GenericObjectPoolConfig(); + if (maxTotal != null){ + config.setMaxTotal(Integer.parseInt(maxTotal)); + } + if (maxIdle != null){ + config.setMaxIdle(Integer.parseInt(maxIdle)); + } + if (minIdle != null){ + config.setMinIdle(Integer.parseInt(minIdle)); + } + return config; } } diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index a541c9300..82532c527 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -22,12 +22,14 @@ import com.dtstack.flink.sql.side.*; import com.dtstack.flink.sql.side.cache.CacheObj; import com.dtstack.flink.sql.side.redis.table.RedisSideTableInfo; -import io.lettuce.core.KeyScanCursor; import io.lettuce.core.KeyValue; import io.lettuce.core.RedisClient; import io.lettuce.core.RedisFuture; import io.lettuce.core.api.StatefulRedisConnection; -import io.lettuce.core.api.async.RedisAsyncCommands; +import io.lettuce.core.api.async.RedisKeyAsyncCommands; +import io.lettuce.core.api.async.RedisStringAsyncCommands; +import io.lettuce.core.cluster.RedisClusterClient; +import io.lettuce.core.cluster.api.StatefulRedisClusterConnection; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; @@ -40,9 +42,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.concurrent.ExecutionException; import java.util.function.Consumer; -import java.util.stream.Collectors; public class RedisAsyncReqRow extends AsyncReqRow { @@ -52,7 +52,11 @@ public class RedisAsyncReqRow extends AsyncReqRow { private StatefulRedisConnection connection; - private RedisAsyncCommands async; + private RedisClusterClient clusterClient; + + private StatefulRedisClusterConnection clusterConnection; + + private RedisKeyAsyncCommands async; private RedisSideTableInfo redisSideTableInfo; @@ -65,19 +69,42 @@ public RedisAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List 1){ - uri.append("redis-sentinel://").append(password).append("@") - .append(url).append("/").append(database).append("#").append(url.split(",")[0]); - } else { - uri.append("redis://").append(password).append("@").append(url).append("/").append(database); + if (database == null){ + database = "0"; + } + switch (tableInfo.getRedisType()){ + case 1: + StringBuilder redisUri = new StringBuilder(); + redisUri.append("redis://").append(password).append(url).append("/").append(database); + redisClient = RedisClient.create(redisUri.toString()); + connection = redisClient.connect(); + async = connection.async(); + break; + case 2: + StringBuilder sentinelUri = new StringBuilder(); + sentinelUri.append("redis-sentinel://").append(password) + .append(url).append("/").append(database).append("#").append(redisSideTableInfo.getMasterName()); + redisClient = RedisClient.create(sentinelUri.toString()); + connection = redisClient.connect(); + async = connection.async(); + break; + case 3: + StringBuilder clusterUri = new StringBuilder(); + clusterUri.append("redis://").append(password).append(url); + clusterClient = RedisClusterClient.create(clusterUri.toString()); + clusterConnection = clusterClient.connect(); + async = clusterConnection.async(); } - redisClient = RedisClient.create(uri.toString()); - connection = redisClient.connect(); - async = connection.async(); } @Override @@ -143,7 +170,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except Map keyValue = Maps.newHashMap(); List value = async.keys(key + ":*").get(); String[] values = value.toArray(new String[value.size()]); - RedisFuture>> future = async.mget(values); + RedisFuture>> future = ((RedisStringAsyncCommands) async).mget(values); future.thenAccept(new Consumer>>() { @Override public void accept(List> keyValues) { @@ -185,6 +212,12 @@ public void close() throws Exception { if (redisClient != null){ redisClient.shutdown(); } + if (clusterConnection != null){ + clusterConnection.close(); + } + if (clusterClient != null){ + clusterClient.shutdown(); + } } } diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java index a5cf0fcaf..a019b74cf 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java @@ -22,7 +22,6 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.redis.table.RedisSideTableInfo; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; diff --git a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java index d9a56d892..fc0d3cb6f 100644 --- a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java +++ b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java @@ -47,7 +47,14 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map { private JedisPool pool; - private Jedis jedis; + private JedisCommands jedis; private JedisSentinelPool jedisSentinelPool; - private JedisCluster jedisCluster; - private GenericObjectPoolConfig poolConfig; private transient Counter outRecords; @@ -120,6 +119,9 @@ private void establishConnection() { String[] ipPortPair = ipPort.split(":"); addresses.add(new HostAndPort(ipPortPair[0].trim(), Integer.valueOf(ipPortPair[1].trim()))); } + if (timeout == 0){ + timeout = 10000; + } switch (redisType){ //单机 @@ -134,7 +136,7 @@ private void establishConnection() { break; //集群 case 3: - jedisCluster = new JedisCluster(addresses, timeout, timeout,1, poolConfig); + jedis = new JedisCluster(addresses, timeout, timeout,1, poolConfig); } } @@ -174,11 +176,7 @@ public void writeRecord(Tuple2 record) throws IOException { for (int i = 0; i < fieldNames.length; i++) { StringBuilder key = new StringBuilder(); key.append(tableName).append(":").append(perKey).append(":").append(fieldNames[i]); - if (redisType != 3){ - jedis.set(key.toString(), (String) row.getField(i)); - } else { - jedisCluster.set(key.toString(), (String) row.getField(i)); - } + jedis.set(key.toString(), (String) row.getField(i)); } outRecords.inc(); } @@ -191,6 +189,11 @@ public void close() throws IOException { if (pool != null) { pool.close(); } + if (jedis != null){ + if (jedis instanceof Closeable){ + ((Closeable) jedis).close(); + } + } } diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java index 3e7300929..d2e28c01f 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisSink.java @@ -79,6 +79,7 @@ public RedisSink genStreamSink(TargetTableInfo targetTableInfo) { this.maxIdle = redisTableInfo.getMaxIdle(); this.minIdle = redisTableInfo.getMinIdle(); this.masterName = redisTableInfo.getMasterName(); + this.timeout = redisTableInfo.getTimeout(); return this; } diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java index cecf70f95..7b1633257 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java @@ -34,7 +34,9 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Fri, 23 Nov 2018 16:47:30 +0800 Subject: [PATCH 151/470] fix mysql output bug(type of float) --- .../dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java index f09b2a1eb..4a231b043 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java @@ -226,9 +226,9 @@ private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLExce pstmt.setLong(index + 1, (long) row.getField(index)); break; case java.sql.Types.REAL: + case java.sql.Types.FLOAT: pstmt.setFloat(index + 1, (float) row.getField(index)); break; - case java.sql.Types.FLOAT: case java.sql.Types.DOUBLE: pstmt.setDouble(index + 1, (double) row.getField(index)); break; From 5250800fb0ac3e81db5a9283bc51561d33745367 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 23 Nov 2018 17:01:45 +0800 Subject: [PATCH 152/470] comment --- .../sink/mysql/RetractJDBCOutputFormat.java | 20 ------------------- 1 file changed, 20 deletions(-) diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java index f09b2a1eb..74e100417 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java @@ -16,26 +16,6 @@ * limitations under the License. */ - - -/* - * 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 com.dtstack.flink.sql.sink.mysql; import com.dtstack.flink.sql.metric.MetricConstant; From 79395d230a84fc270b58e7ae9a2205707a7d0a1d Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Fri, 23 Nov 2018 17:17:10 +0800 Subject: [PATCH 153/470] add rdb module and add sqlserver side all request --- pom.xml | 1 + rdb/pom.xml | 40 ++++ rdb/sqlserver/pom.xml | 40 ++++ rdb/sqlserver/sqlserver-side/pom.xml | 25 ++ .../sqlserver-side/sqlserver-all-side/pom.xml | 87 +++++++ .../side/sqlserver/SqlserverAllReqRow.java | 221 ++++++++++++++++++ .../side/sqlserver/SqlserverAllSideInfo.java | 89 +++++++ .../sqlserver-side-core/pom.xml | 18 ++ .../sqlserver/table/SqlserverSideParser.java | 41 ++++ .../table/SqlserverSideTableInfo.java | 69 ++++++ 10 files changed, 631 insertions(+) create mode 100644 rdb/pom.xml create mode 100644 rdb/sqlserver/pom.xml create mode 100644 rdb/sqlserver/sqlserver-side/pom.xml create mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml create mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java create mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java create mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml create mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java create mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideTableInfo.java diff --git a/pom.xml b/pom.xml index 07694589c..f6911ad58 100644 --- a/pom.xml +++ b/pom.xml @@ -16,6 +16,7 @@ mongo redis5 launcher + rdb pom diff --git a/rdb/pom.xml b/rdb/pom.xml new file mode 100644 index 000000000..6b1a7fd8a --- /dev/null +++ b/rdb/pom.xml @@ -0,0 +1,40 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.rdb + 1.0-SNAPSHOT + + sqlserver + + pom + + + 3.8.1 + + + + + junit + junit + ${junit.version} + test + + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + + + \ No newline at end of file diff --git a/rdb/sqlserver/pom.xml b/rdb/sqlserver/pom.xml new file mode 100644 index 000000000..beb9bb8aa --- /dev/null +++ b/rdb/sqlserver/pom.xml @@ -0,0 +1,40 @@ + + + + sql.rdb + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sqlserver + 1.0-SNAPSHOT + pom + + + 1.3.1 + + + + sqlserver-side + + + + + net.sourceforge.jtds + jtds + ${jtds.version} + + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + + \ No newline at end of file diff --git a/rdb/sqlserver/sqlserver-side/pom.xml b/rdb/sqlserver/sqlserver-side/pom.xml new file mode 100644 index 000000000..dc9229931 --- /dev/null +++ b/rdb/sqlserver/sqlserver-side/pom.xml @@ -0,0 +1,25 @@ + + + + sql.sqlserver + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.sqlserver + 1.0-SNAPSHOT + + sqlserver-side + pom + + + sqlserver-side-core + sqlserver-all-side + + + + \ No newline at end of file diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml b/rdb/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml new file mode 100644 index 000000000..45f36422e --- /dev/null +++ b/rdb/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml @@ -0,0 +1,87 @@ + + + + sql.side.sqlserver + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.all.sqlserver + sqlserver-all-side + 1.0-SNAPSHOT + jar + + + + com.dtstack.flink + sql.side.sqlserver.core + 1.0-SNAPSHOT + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java b/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java new file mode 100644 index 000000000..5ef274377 --- /dev/null +++ b/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java @@ -0,0 +1,221 @@ +package com.dtstack.flink.sql.side.sqlserver; + +import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.sqlserver.table.SqlserverSideTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.calcite.sql.JoinType; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.*; +import java.util.Calendar; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; + +/** + * side operator with cache for all(period reload) + */ +public class SqlserverAllReqRow extends AllReqRow { + + + private static final Logger LOG = LoggerFactory.getLogger(SqlserverAllReqRow.class); + + private static final String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; + + private static final int CONN_RETRY_NUM = 3; + + private static final int FETCH_SIZE = 1000; + + private AtomicReference>>> cacheRef = new AtomicReference<>(); + + public SqlserverAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new SqlserverAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + @Override + protected Row fillData(Row input, Object sideInput) { + Map cacheInfo = (Map) sideInput; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. + if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { + obj = ((Timestamp) obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for (Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()) { + if (cacheInfo == null) { + row.setField(entry.getKey(), null); + } else { + row.setField(entry.getKey(), cacheInfo.get(entry.getValue())); + } + } + + return row; + } + + @Override + protected void initCache() throws SQLException { + Map>> newCache = Maps.newConcurrentMap(); + cacheRef.set(newCache); + loadData(newCache); + } + + @Override + protected void reloadCache() { + //reload cacheRef and replace to old cacheRef + Map>> newCache = Maps.newConcurrentMap(); + try { + loadData(newCache); + } catch (SQLException e) { + LOG.error("", e); + } + + cacheRef.set(newCache); + LOG.info("----- rdb all cacheRef reload end:{}", Calendar.getInstance()); + } + + @Override + public void flatMap(Row value, Collector out) throws Exception { + List inputParams = Lists.newArrayList(); + for (Integer conValIndex : sideInfo.getEqualValIndex()) { + Object equalObj = value.getField(conValIndex); + if (equalObj == null) { + out.collect(null); + } + + inputParams.add(equalObj); + } + + String key = buildKey(inputParams); + List> cacheList = cacheRef.get().get(key); + if (CollectionUtils.isEmpty(cacheList)) { + if (sideInfo.getJoinType() == JoinType.LEFT) { + Row row = fillData(value, null); + out.collect(row); + } else { + return; + } + + return; + } + + for (Map one : cacheList) { + out.collect(fillData(value, one)); + } + + } + + /** + * load data by diff db + * + * @param tmpCache + */ + private void loadData(Map>> tmpCache) throws SQLException { + //这个地方抽取为RdbSideTableInfo + SqlserverSideTableInfo tableInfo = (SqlserverSideTableInfo) sideInfo.getSideTableInfo(); + Connection connection = null; + + + try { + for (int i = 0; i < CONN_RETRY_NUM; i++) { + + try { + connection = getConn(tableInfo.getUrl(), tableInfo.getUserName(), tableInfo.getPassword()); + break; + } catch (Exception e) { + if (i == CONN_RETRY_NUM - 1) { + throw new RuntimeException("", e); + } + + try { + String connInfo = "url:" + tableInfo.getUrl() + ";userName:" + tableInfo.getUserName() + ",pwd:" + tableInfo.getPassword(); + LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + connInfo); + Thread.sleep(5 * 1000); + } catch (InterruptedException e1) { + e1.printStackTrace(); + } + } + + } + + //load data from table + String sql = sideInfo.getSqlCondition(); + Statement statement = connection.createStatement(); + statement.setFetchSize(FETCH_SIZE); + ResultSet resultSet = statement.executeQuery(sql); + String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); + while (resultSet.next()) { + Map oneRow = Maps.newHashMap(); + for (String fieldName : sideFieldNames) { + oneRow.put(fieldName.trim(), resultSet.getObject(fieldName.trim())); + } + + String cacheKey = buildKey(oneRow, sideInfo.getEqualFieldList()); + List> list = tmpCache.computeIfAbsent(cacheKey, key -> Lists.newArrayList()); + list.add(oneRow); + } + } catch (Exception e) { + LOG.error("", e); + } finally { + if (connection != null) { + connection.close(); + } + } + } + + + private String buildKey(List equalValList) { + StringBuilder sb = new StringBuilder(""); + for (Object equalVal : equalValList) { + sb.append(equalVal).append("_"); + } + return sb.toString(); + } + + private String buildKey(Map val, List equalFieldList) { + StringBuilder sb = new StringBuilder(""); + for (String equalField : equalFieldList) { + sb.append(val.get(equalField)).append("_"); + } + + return sb.toString(); + } + + /** + * 这个方法创建在RdbAllReqRow.java中 + * + * @param dbURL + * @param userName + * @param password + * @return + */ + private Connection getConn(String dbURL, String userName, String password) { + try { + Class.forName(SQLSERVER_DRIVER); + //add param useCursorFetch=true + Map addParams = Maps.newHashMap(); + //addParams.put("useCursorFetch", "true"); + String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); + return DriverManager.getConnection(targetDbUrl, userName, password); + } catch (Exception e) { + LOG.error("", e); + throw new RuntimeException("", e); + } + } + + + +} diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java b/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java new file mode 100644 index 000000000..f112ae251 --- /dev/null +++ b/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java @@ -0,0 +1,89 @@ +package com.dtstack.flink.sql.side.sqlserver; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +//import com.dtstack.flink.sql.side.sqlserver.table.SqlserverSideTableInfo; +import com.dtstack.flink.sql.side.sqlserver.table.SqlserverSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.List; + +public class SqlserverAllSideInfo extends SideInfo { + + + public SqlserverAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + SqlserverSideTableInfo sqlserverSideTableInfo = (SqlserverSideTableInfo) sideTableInfo; + + sqlCondition = "select ${selectField} from ${tableName} "; + sqlCondition = sqlCondition.replace("${tableName}", sqlserverSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); + System.out.println("---------side_exe_sql-----\n" + sqlCondition); + + } + + @Override + public void parseSelectFields(JoinInfo joinInfo){ + String sideTableName = joinInfo.getSideTableName(); + String nonSideTableName = joinInfo.getNonSideTable(); + List fields = Lists.newArrayList(); + + int sideIndex = 0; + for( int i=0; i sqlNodeList = Lists.newArrayList(); + if(conditionNode.getKind() == SqlKind.AND){ + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); + }else{ + sqlNodeList.add(conditionNode); + } + + for(SqlNode sqlNode : sqlNodeList){ + dealOneEqualCon(sqlNode, sideTableName); + } + + if(CollectionUtils.isEmpty(equalFieldList)){ + throw new RuntimeException("no join condition found after table " + joinInfo.getLeftTableName()); + } + + for(String equalField : equalFieldList){ + if(fields.contains(equalField)){ + continue; + } + + fields.add(equalField); + } + + sideSelectFields = String.join(",", fields); + } +} diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml b/rdb/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml new file mode 100644 index 000000000..6cb64eb7e --- /dev/null +++ b/rdb/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml @@ -0,0 +1,18 @@ + + + + sql.side.sqlserver + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.sqlserver.core + 1.0-SNAPSHOT + jar + + + \ No newline at end of file diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java b/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java new file mode 100644 index 000000000..4fbe35c17 --- /dev/null +++ b/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java @@ -0,0 +1,41 @@ +package com.dtstack.flink.sql.side.sqlserver.table; + +import com.dtstack.flink.sql.table.AbsSideTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class SqlserverSideParser extends AbsSideTableParser { + private final static String SIDE_SIGN_KEY = "sideSignKey"; + + private final static Pattern SIDE_TABLE_SIGN = Pattern.compile("(?i)^PERIOD\\s+FOR\\s+SYSTEM_TIME$"); + + static { + keyPatternMap.put(SIDE_SIGN_KEY, SIDE_TABLE_SIGN); + keyHandlerMap.put(SIDE_SIGN_KEY, SqlserverSideParser::dealSideSign); + } + + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + SqlserverSideTableInfo sqlserverSideTableInfo = new SqlserverSideTableInfo(); + sqlserverSideTableInfo.setName(tableName); + + parseFieldsInfo(fieldsInfo, sqlserverSideTableInfo); + parseCacheProp(sqlserverSideTableInfo, props); + + sqlserverSideTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(SqlserverSideTableInfo.PARALLELISM_KEY.toLowerCase()))); + sqlserverSideTableInfo.setUrl(MathUtil.getString(props.get(SqlserverSideTableInfo.URL_KEY.toLowerCase()))); + sqlserverSideTableInfo.setTableName(MathUtil.getString(props.get(SqlserverSideTableInfo.TABLE_NAME_KEY.toLowerCase()))); + sqlserverSideTableInfo.setUserName(MathUtil.getString(props.get(SqlserverSideTableInfo.USER_NAME_KEY.toLowerCase()))); + sqlserverSideTableInfo.setPassword(MathUtil.getString(props.get(SqlserverSideTableInfo.PASSWORD_KEY.toLowerCase()))); + + return sqlserverSideTableInfo; + } + + private static void dealSideSign(Matcher matcher, TableInfo tableInfo) { + } +} diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideTableInfo.java b/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideTableInfo.java new file mode 100644 index 000000000..f8f41d9d4 --- /dev/null +++ b/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideTableInfo.java @@ -0,0 +1,69 @@ +package com.dtstack.flink.sql.side.sqlserver.table; + +import com.dtstack.flink.sql.side.SideTableInfo; + + +public class SqlserverSideTableInfo extends SideTableInfo { + + private static final long serialVersionUID = -1L; + + private static final String CURR_TYPE = "sqlserver"; + + public static final String URL_KEY = "url"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String USER_NAME_KEY = "userName"; + + public static final String PASSWORD_KEY = "password"; + + public SqlserverSideTableInfo() { + setType(CURR_TYPE); + } + + private String url; + + private String tableName; + + private String userName; + + private String password; + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } + + @Override + public boolean check() { + return false; + } + +} From b08b4377a609968d73806173a788b2173ef9af9d Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 23 Nov 2018 18:03:57 +0800 Subject: [PATCH 154/470] fix mysql output bug(type of float) --- .../main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java index 0fe9426c5..c2a4e50fd 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java @@ -113,7 +113,9 @@ protected void buildSqlTypes(List fieldTypeArray){ tmpFieldsType[i] = Types.CHAR; }else if(fieldType.equals(Byte.class.getName())){ tmpFieldsType[i] = Types.BINARY; - }else if(fieldType.equals(Float.class.getName()) || fieldType.equals(Double.class.getName())){ + }else if(fieldType.equals(Float.class.getName())){ + tmpFieldsType[i] = Types.FLOAT; + }else if(fieldType.equals(Double.class.getName())){ tmpFieldsType[i] = Types.DOUBLE; }else if (fieldType.equals(Timestamp.class.getName())){ tmpFieldsType[i] = Types.TIMESTAMP; From 7e2212f8f8c7f7e756ca31ae284b03c18d5f1047 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 23 Nov 2018 21:31:32 +0800 Subject: [PATCH 155/470] fix mysql output bug(type of BigDecimal) --- .../main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java index c2a4e50fd..c3526ecd1 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java @@ -32,6 +32,7 @@ import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; +import java.math.BigDecimal; import java.sql.Timestamp; import java.sql.Types; import java.util.List; @@ -117,8 +118,10 @@ protected void buildSqlTypes(List fieldTypeArray){ tmpFieldsType[i] = Types.FLOAT; }else if(fieldType.equals(Double.class.getName())){ tmpFieldsType[i] = Types.DOUBLE; - }else if (fieldType.equals(Timestamp.class.getName())){ + }else if(fieldType.equals(Timestamp.class.getName())){ tmpFieldsType[i] = Types.TIMESTAMP; + }else if(fieldType.equals(BigDecimal.class.getName())){ + tmpFieldsType[i] = Types.DECIMAL; }else{ throw new RuntimeException("no support field type for sql. the input type:" + fieldType); } From 95e91f953c4dbac7ca1c9584b06c082bf70b7db3 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 26 Nov 2018 11:02:00 +0800 Subject: [PATCH 156/470] move side sign parser to absclass --- .../flink/sql/table/AbsSideTableParser.java | 15 ++++++++++++++ .../sql/side/hbase/table/HbaseSideParser.java | 10 ---------- .../sql/side/mongo/table/MongoSideParser.java | 12 ----------- .../sql/side/mysql/table/MysqlSideParser.java | 12 ----------- .../sqlserver/table/SqlserverSideParser.java | 13 ------------ .../sql/side/redis/table/RedisSideParser.java | 20 +++++-------------- 6 files changed, 20 insertions(+), 62 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java index e46a72494..f8ede801b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java @@ -25,6 +25,8 @@ import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; /** * Reason: @@ -35,6 +37,19 @@ public abstract class AbsSideTableParser extends AbsTableParser { + private final static String SIDE_SIGN_KEY = "sideSignKey"; + + private final static Pattern SIDE_TABLE_SIGN = Pattern.compile("(?i)^PERIOD\\s+FOR\\s+SYSTEM_TIME$"); + + static { + keyPatternMap.put(SIDE_SIGN_KEY, SIDE_TABLE_SIGN); + keyHandlerMap.put(SIDE_SIGN_KEY, AbsSideTableParser::dealSideSign); + } + + private static void dealSideSign(Matcher matcher, TableInfo tableInfo){ + //FIXME SIDE_TABLE_SIGN current just used as a sign for side table; and do nothing + } + //Analytical create table attributes ==> Get information cache protected void parseCacheProp(SideTableInfo sideTableInfo, Map props){ if(props.containsKey(SideTableInfo.CACHE_KEY.toLowerCase())){ diff --git a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java index 299db961d..3031de100 100644 --- a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java +++ b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/table/HbaseSideParser.java @@ -44,10 +44,6 @@ public class HbaseSideParser extends AbsSideTableParser { private final static Pattern FIELD_PATTERN = Pattern.compile("(?i)(.*)\\s+AS\\s+(\\w+)$"); - private final static String SIDE_SIGN_KEY = "sideSignKey"; - - private final static Pattern SIDE_TABLE_SIGN = Pattern.compile("(?i)^PERIOD\\s+FOR\\s+SYSTEM_TIME$"); - public static final String HBASE_ZOOKEEPER_QUORUM = "zookeeperQuorum"; public static final String ZOOKEEPER_PARENT = "zookeeperParent"; @@ -60,9 +56,6 @@ public class HbaseSideParser extends AbsSideTableParser { static { - keyPatternMap.put(SIDE_SIGN_KEY, SIDE_TABLE_SIGN); - keyHandlerMap.put(SIDE_SIGN_KEY, HbaseSideParser::dealSideSign); - keyPatternMap.put(FIELD_KEY, FIELD_PATTERN); keyHandlerMap.put(FIELD_KEY, HbaseSideParser::dealField); } @@ -82,9 +75,6 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { MongoSideTableInfo mongoSideTableInfo = new MongoSideTableInfo(); @@ -73,7 +64,4 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { MysqlSideTableInfo mysqlTableInfo = new MysqlSideTableInfo(); @@ -61,7 +52,4 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + RedisSideTableInfo redisSideTableInfo = new RedisSideTableInfo(); redisSideTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, redisSideTableInfo); @@ -47,17 +37,17 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Mon, 26 Nov 2018 16:07:29 +0800 Subject: [PATCH 157/470] youhua --- .../dtstack/flink/sql/util/DtStringUtil.java | 18 ++++++++++++++++++ .../flink/sql/source/kafka/KafkaSource.java | 4 ++-- .../flink/sql/source/kafka/KafkaSource.java | 4 ++-- .../flink/sql/source/kafka/KafkaSource.java | 4 ++-- 4 files changed, 24 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index db74480d6..959d4b13c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -21,8 +21,10 @@ package com.dtstack.flink.sql.util; import com.dtstack.flink.sql.enums.ColumnType; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import java.math.BigDecimal; import java.util.ArrayList; @@ -42,6 +44,9 @@ public class DtStringUtil { private static final Pattern NO_VERSION_PATTERN = Pattern.compile("([a-zA-Z]+).*"); + private static ObjectMapper objectMapper = new ObjectMapper(); + + /** * Split the specified string delimiter --- ignored quotes delimiter * @param str @@ -207,4 +212,17 @@ public static String addJdbcParam(String dbUrl, Map addParams, b return preStr + "?" + sb.toString(); } + + public static boolean isJosn(String str){ + boolean flag = false; + if(StringUtils.isNotBlank(str)){ + try { + objectMapper.readValue(str,Map.class); + flag = true; + } catch (Throwable e) { + flag=false; + } + } + return flag; + } } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 00988518a..cbc697c82 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.functions.RuntimeContext; @@ -80,9 +81,8 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv //earliest,latest if("earliest".equalsIgnoreCase(kafka09SourceTableInfo.getOffsetReset())){ kafkaSrc.setStartFromEarliest(); - }else if(kafka09SourceTableInfo.getOffsetReset().startsWith("{")){ + }else if(DtStringUtil.isJosn(kafka09SourceTableInfo.getOffsetReset())){// {"0":12312,"1":12321,"2":12312} try { - // {"0":12312,"1":12321,"2":12312} Properties properties = PluginUtil.jsonStrToObject(kafka09SourceTableInfo.getOffsetReset(), Properties.class); Map offsetMap = PluginUtil.ObjectToMap(properties); Map specificStartupOffsets = new HashMap<>(); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index c9eaf05bc..2953d0f86 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -79,9 +80,8 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv //earliest,latest if("earliest".equalsIgnoreCase(kafka010SourceTableInfo.getOffsetReset())){ kafkaSrc.setStartFromEarliest(); - }else if(kafka010SourceTableInfo.getOffsetReset().startsWith("{")){ + }else if(DtStringUtil.isJosn(kafka010SourceTableInfo.getOffsetReset())){// {"0":12312,"1":12321,"2":12312} try { - // {"0":12312,"1":12321,"2":12312} Properties properties = PluginUtil.jsonStrToObject(kafka010SourceTableInfo.getOffsetReset(), Properties.class); Map offsetMap = PluginUtil.ObjectToMap(properties); Map specificStartupOffsets = new HashMap<>(); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 5f6146cc9..d10151920 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.source.IStreamSourceGener; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -79,9 +80,8 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv //earliest,latest if("earliest".equalsIgnoreCase(kafka011SourceTableInfo.getOffsetReset())){ kafkaSrc.setStartFromEarliest(); - }else if(kafka011SourceTableInfo.getOffsetReset().startsWith("{")){ + }else if(DtStringUtil.isJosn(kafka011SourceTableInfo.getOffsetReset())){// {"0":12312,"1":12321,"2":12312} try { - // {"0":12312,"1":12321,"2":12312} Properties properties = PluginUtil.jsonStrToObject(kafka011SourceTableInfo.getOffsetReset(), Properties.class); Map offsetMap = PluginUtil.ObjectToMap(properties); Map specificStartupOffsets = new HashMap<>(); From f49e370a421ebec0af358052f041fb92454e5e30 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Mon, 26 Nov 2018 11:12:29 +0800 Subject: [PATCH 158/470] create rdb module and extract common code --- pom.xml | 76 +++--- rdb/pom.xml | 3 +- rdb/rdb-side/pom.xml | 17 ++ rdb/rdb-sink/pom.xml | 17 ++ .../side/sqlserver/SqlserverAllReqRow.java | 221 ------------------ .../side/sqlserver/SqlserverAllSideInfo.java | 89 ------- .../table/SqlserverSideTableInfo.java | 69 ------ {rdb/sqlserver => sqlserver}/pom.xml | 12 +- .../sqlserver-side/pom.xml | 4 +- .../sqlserver-side/sqlserver-all-side/pom.xml | 4 +- .../sqlserver-side-core/pom.xml | 1 - 11 files changed, 83 insertions(+), 430 deletions(-) create mode 100644 rdb/rdb-side/pom.xml create mode 100644 rdb/rdb-sink/pom.xml delete mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java delete mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java delete mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideTableInfo.java rename {rdb/sqlserver => sqlserver}/pom.xml (88%) rename {rdb/sqlserver => sqlserver}/sqlserver-side/pom.xml (93%) rename {rdb/sqlserver => sqlserver}/sqlserver-side/sqlserver-all-side/pom.xml (98%) rename {rdb/sqlserver => sqlserver}/sqlserver-side/sqlserver-side-core/pom.xml (92%) diff --git a/pom.xml b/pom.xml index f6911ad58..808b55976 100644 --- a/pom.xml +++ b/pom.xml @@ -1,43 +1,45 @@ - 4.0.0 + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + 4.0.0 - com.dtstack.flink - flink.sql - 1.0-SNAPSHOT - - core - kafka09 - kafka10 - kafka11 - mysql - hbase - elasticsearch5 - mongo - redis5 - launcher - rdb - - pom + com.dtstack.flink + flink.sql + 1.0-SNAPSHOT + + core + kafka09 + kafka10 + kafka11 + mysql + hbase + elasticsearch5 + mongo + redis5 + launcher + rdb + sqlserver + rdb + + pom - flink.sql - http://maven.apache.org + flink.sql + http://maven.apache.org - - UTF-8 - 1.5.4 - + + UTF-8 + 1.5.4 + - - - - org.apache.maven.plugins - maven-compiler-plugin - - 1.8 - 1.8 - - - - + + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.8 + 1.8 + + + + diff --git a/rdb/pom.xml b/rdb/pom.xml index 6b1a7fd8a..705ad728c 100644 --- a/rdb/pom.xml +++ b/rdb/pom.xml @@ -12,7 +12,8 @@ sql.rdb 1.0-SNAPSHOT - sqlserver + rdb-side + rdb-sink pom diff --git a/rdb/rdb-side/pom.xml b/rdb/rdb-side/pom.xml new file mode 100644 index 000000000..0103ca4cf --- /dev/null +++ b/rdb/rdb-side/pom.xml @@ -0,0 +1,17 @@ + + + + sql.rdb + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.rdb + 1.0-SNAPSHOT + rdb-side + jar + + \ No newline at end of file diff --git a/rdb/rdb-sink/pom.xml b/rdb/rdb-sink/pom.xml new file mode 100644 index 000000000..6b4868c95 --- /dev/null +++ b/rdb/rdb-sink/pom.xml @@ -0,0 +1,17 @@ + + + + sql.rdb + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.sink.rdb + 1.0-SNAPSHOT + rdb-sink + jar + + \ No newline at end of file diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java b/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java deleted file mode 100644 index 5ef274377..000000000 --- a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java +++ /dev/null @@ -1,221 +0,0 @@ -package com.dtstack.flink.sql.side.sqlserver; - -import com.dtstack.flink.sql.side.*; -import com.dtstack.flink.sql.side.sqlserver.table.SqlserverSideTableInfo; -import com.dtstack.flink.sql.util.DtStringUtil; -import org.apache.calcite.sql.JoinType; -import org.apache.commons.collections.CollectionUtils; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; -import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; -import org.apache.flink.types.Row; -import org.apache.flink.util.Collector; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.sql.*; -import java.util.Calendar; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicReference; - -/** - * side operator with cache for all(period reload) - */ -public class SqlserverAllReqRow extends AllReqRow { - - - private static final Logger LOG = LoggerFactory.getLogger(SqlserverAllReqRow.class); - - private static final String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; - - private static final int CONN_RETRY_NUM = 3; - - private static final int FETCH_SIZE = 1000; - - private AtomicReference>>> cacheRef = new AtomicReference<>(); - - public SqlserverAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(new SqlserverAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); - } - - @Override - protected Row fillData(Row input, Object sideInput) { - Map cacheInfo = (Map) sideInput; - Row row = new Row(sideInfo.getOutFieldInfoList().size()); - for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { - Object obj = input.getField(entry.getValue()); - boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); - - //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. - if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { - obj = ((Timestamp) obj).getTime(); - } - row.setField(entry.getKey(), obj); - } - - for (Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()) { - if (cacheInfo == null) { - row.setField(entry.getKey(), null); - } else { - row.setField(entry.getKey(), cacheInfo.get(entry.getValue())); - } - } - - return row; - } - - @Override - protected void initCache() throws SQLException { - Map>> newCache = Maps.newConcurrentMap(); - cacheRef.set(newCache); - loadData(newCache); - } - - @Override - protected void reloadCache() { - //reload cacheRef and replace to old cacheRef - Map>> newCache = Maps.newConcurrentMap(); - try { - loadData(newCache); - } catch (SQLException e) { - LOG.error("", e); - } - - cacheRef.set(newCache); - LOG.info("----- rdb all cacheRef reload end:{}", Calendar.getInstance()); - } - - @Override - public void flatMap(Row value, Collector out) throws Exception { - List inputParams = Lists.newArrayList(); - for (Integer conValIndex : sideInfo.getEqualValIndex()) { - Object equalObj = value.getField(conValIndex); - if (equalObj == null) { - out.collect(null); - } - - inputParams.add(equalObj); - } - - String key = buildKey(inputParams); - List> cacheList = cacheRef.get().get(key); - if (CollectionUtils.isEmpty(cacheList)) { - if (sideInfo.getJoinType() == JoinType.LEFT) { - Row row = fillData(value, null); - out.collect(row); - } else { - return; - } - - return; - } - - for (Map one : cacheList) { - out.collect(fillData(value, one)); - } - - } - - /** - * load data by diff db - * - * @param tmpCache - */ - private void loadData(Map>> tmpCache) throws SQLException { - //这个地方抽取为RdbSideTableInfo - SqlserverSideTableInfo tableInfo = (SqlserverSideTableInfo) sideInfo.getSideTableInfo(); - Connection connection = null; - - - try { - for (int i = 0; i < CONN_RETRY_NUM; i++) { - - try { - connection = getConn(tableInfo.getUrl(), tableInfo.getUserName(), tableInfo.getPassword()); - break; - } catch (Exception e) { - if (i == CONN_RETRY_NUM - 1) { - throw new RuntimeException("", e); - } - - try { - String connInfo = "url:" + tableInfo.getUrl() + ";userName:" + tableInfo.getUserName() + ",pwd:" + tableInfo.getPassword(); - LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + connInfo); - Thread.sleep(5 * 1000); - } catch (InterruptedException e1) { - e1.printStackTrace(); - } - } - - } - - //load data from table - String sql = sideInfo.getSqlCondition(); - Statement statement = connection.createStatement(); - statement.setFetchSize(FETCH_SIZE); - ResultSet resultSet = statement.executeQuery(sql); - String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); - while (resultSet.next()) { - Map oneRow = Maps.newHashMap(); - for (String fieldName : sideFieldNames) { - oneRow.put(fieldName.trim(), resultSet.getObject(fieldName.trim())); - } - - String cacheKey = buildKey(oneRow, sideInfo.getEqualFieldList()); - List> list = tmpCache.computeIfAbsent(cacheKey, key -> Lists.newArrayList()); - list.add(oneRow); - } - } catch (Exception e) { - LOG.error("", e); - } finally { - if (connection != null) { - connection.close(); - } - } - } - - - private String buildKey(List equalValList) { - StringBuilder sb = new StringBuilder(""); - for (Object equalVal : equalValList) { - sb.append(equalVal).append("_"); - } - return sb.toString(); - } - - private String buildKey(Map val, List equalFieldList) { - StringBuilder sb = new StringBuilder(""); - for (String equalField : equalFieldList) { - sb.append(val.get(equalField)).append("_"); - } - - return sb.toString(); - } - - /** - * 这个方法创建在RdbAllReqRow.java中 - * - * @param dbURL - * @param userName - * @param password - * @return - */ - private Connection getConn(String dbURL, String userName, String password) { - try { - Class.forName(SQLSERVER_DRIVER); - //add param useCursorFetch=true - Map addParams = Maps.newHashMap(); - //addParams.put("useCursorFetch", "true"); - String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); - return DriverManager.getConnection(targetDbUrl, userName, password); - } catch (Exception e) { - LOG.error("", e); - throw new RuntimeException("", e); - } - } - - - -} diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java b/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java deleted file mode 100644 index f112ae251..000000000 --- a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java +++ /dev/null @@ -1,89 +0,0 @@ -package com.dtstack.flink.sql.side.sqlserver; - -import com.dtstack.flink.sql.side.FieldInfo; -import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; -import com.dtstack.flink.sql.side.SideTableInfo; -//import com.dtstack.flink.sql.side.sqlserver.table.SqlserverSideTableInfo; -import com.dtstack.flink.sql.side.sqlserver.table.SqlserverSideTableInfo; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlNode; -import org.apache.commons.collections.CollectionUtils; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; - -import java.util.List; - -public class SqlserverAllSideInfo extends SideInfo { - - - public SqlserverAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); - } - - @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { - SqlserverSideTableInfo sqlserverSideTableInfo = (SqlserverSideTableInfo) sideTableInfo; - - sqlCondition = "select ${selectField} from ${tableName} "; - sqlCondition = sqlCondition.replace("${tableName}", sqlserverSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); - System.out.println("---------side_exe_sql-----\n" + sqlCondition); - - } - - @Override - public void parseSelectFields(JoinInfo joinInfo){ - String sideTableName = joinInfo.getSideTableName(); - String nonSideTableName = joinInfo.getNonSideTable(); - List fields = Lists.newArrayList(); - - int sideIndex = 0; - for( int i=0; i sqlNodeList = Lists.newArrayList(); - if(conditionNode.getKind() == SqlKind.AND){ - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); - }else{ - sqlNodeList.add(conditionNode); - } - - for(SqlNode sqlNode : sqlNodeList){ - dealOneEqualCon(sqlNode, sideTableName); - } - - if(CollectionUtils.isEmpty(equalFieldList)){ - throw new RuntimeException("no join condition found after table " + joinInfo.getLeftTableName()); - } - - for(String equalField : equalFieldList){ - if(fields.contains(equalField)){ - continue; - } - - fields.add(equalField); - } - - sideSelectFields = String.join(",", fields); - } -} diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideTableInfo.java b/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideTableInfo.java deleted file mode 100644 index f8f41d9d4..000000000 --- a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideTableInfo.java +++ /dev/null @@ -1,69 +0,0 @@ -package com.dtstack.flink.sql.side.sqlserver.table; - -import com.dtstack.flink.sql.side.SideTableInfo; - - -public class SqlserverSideTableInfo extends SideTableInfo { - - private static final long serialVersionUID = -1L; - - private static final String CURR_TYPE = "sqlserver"; - - public static final String URL_KEY = "url"; - - public static final String TABLE_NAME_KEY = "tableName"; - - public static final String USER_NAME_KEY = "userName"; - - public static final String PASSWORD_KEY = "password"; - - public SqlserverSideTableInfo() { - setType(CURR_TYPE); - } - - private String url; - - private String tableName; - - private String userName; - - private String password; - - public String getUrl() { - return url; - } - - public void setUrl(String url) { - this.url = url; - } - - public String getTableName() { - return tableName; - } - - public void setTableName(String tableName) { - this.tableName = tableName; - } - - public String getUserName() { - return userName; - } - - public void setUserName(String userName) { - this.userName = userName; - } - - public String getPassword() { - return password; - } - - public void setPassword(String password) { - this.password = password; - } - - @Override - public boolean check() { - return false; - } - -} diff --git a/rdb/sqlserver/pom.xml b/sqlserver/pom.xml similarity index 88% rename from rdb/sqlserver/pom.xml rename to sqlserver/pom.xml index beb9bb8aa..05f880a3f 100644 --- a/rdb/sqlserver/pom.xml +++ b/sqlserver/pom.xml @@ -3,25 +3,23 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> - sql.rdb + flink.sql com.dtstack.flink 1.0-SNAPSHOT - ../pom.xml 4.0.0 sql.sqlserver 1.0-SNAPSHOT + + sqlserver-side + pom 1.3.1 - - sqlserver-side - - net.sourceforge.jtds @@ -31,7 +29,7 @@ com.dtstack.flink - sql.core + sql.side.rdb 1.0-SNAPSHOT provided diff --git a/rdb/sqlserver/sqlserver-side/pom.xml b/sqlserver/sqlserver-side/pom.xml similarity index 93% rename from rdb/sqlserver/sqlserver-side/pom.xml rename to sqlserver/sqlserver-side/pom.xml index dc9229931..fc9179f62 100644 --- a/rdb/sqlserver/sqlserver-side/pom.xml +++ b/sqlserver/sqlserver-side/pom.xml @@ -6,7 +6,6 @@ sql.sqlserver com.dtstack.flink 1.0-SNAPSHOT - ../pom.xml 4.0.0 @@ -19,7 +18,6 @@ sqlserver-side-core sqlserver-all-side + sqlserver-side-core - - \ No newline at end of file diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml b/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml similarity index 98% rename from rdb/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml rename to sqlserver/sqlserver-side/sqlserver-all-side/pom.xml index 45f36422e..a11abe0f4 100644 --- a/rdb/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml +++ b/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml @@ -6,13 +6,12 @@ sql.side.sqlserver com.dtstack.flink 1.0-SNAPSHOT - ../pom.xml 4.0.0 sql.side.all.sqlserver - sqlserver-all-side 1.0-SNAPSHOT + sqlserver-all-side jar @@ -84,4 +83,5 @@ + \ No newline at end of file diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml b/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml similarity index 92% rename from rdb/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml rename to sqlserver/sqlserver-side/sqlserver-side-core/pom.xml index 6cb64eb7e..36940fe66 100644 --- a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml +++ b/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml @@ -6,7 +6,6 @@ sql.side.sqlserver com.dtstack.flink 1.0-SNAPSHOT - ../pom.xml 4.0.0 From 1fd73f0855a48ad61be4b46d236455efbcb5675e Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Mon, 26 Nov 2018 21:42:33 +0800 Subject: [PATCH 159/470] sqlserver side depend on rdb --- pom.xml | 1 - rdb/pom.xml | 3 +- rdb/rdb-side/pom.xml | 23 +++++++++++++++ .../sqlserver/table/SqlserverSideParser.java | 28 ------------------- sqlserver/pom.xml | 19 +++++++++---- sqlserver/sqlserver-side/pom.xml | 20 ++++++++++++- .../sqlserver-side/sqlserver-all-side/pom.xml | 12 +++++--- .../sqlserver-side-core/pom.xml | 1 + 8 files changed, 67 insertions(+), 40 deletions(-) delete mode 100644 rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java diff --git a/pom.xml b/pom.xml index 808b55976..bde73d6df 100644 --- a/pom.xml +++ b/pom.xml @@ -18,7 +18,6 @@ launcher rdb sqlserver - rdb pom diff --git a/rdb/pom.xml b/rdb/pom.xml index 705ad728c..0ef3aa252 100644 --- a/rdb/pom.xml +++ b/rdb/pom.xml @@ -19,6 +19,7 @@ 3.8.1 + 1.0-SNAPSHOT @@ -32,7 +33,7 @@ com.dtstack.flink sql.core - 1.0-SNAPSHOT + ${sql.core.version} provided diff --git a/rdb/rdb-side/pom.xml b/rdb/rdb-side/pom.xml index 0103ca4cf..2c1680fb0 100644 --- a/rdb/rdb-side/pom.xml +++ b/rdb/rdb-side/pom.xml @@ -14,4 +14,27 @@ rdb-side jar + + 3.5.2 + + + + + + + + io.vertx + vertx-jdbc-client + ${vertx.version} + + + + io.vertx + vertx-core + ${vertx.version} + + + + + \ No newline at end of file diff --git a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java b/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java deleted file mode 100644 index 7ad2d6f0c..000000000 --- a/rdb/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java +++ /dev/null @@ -1,28 +0,0 @@ -package com.dtstack.flink.sql.side.sqlserver.table; - -import com.dtstack.flink.sql.table.AbsSideTableParser; -import com.dtstack.flink.sql.table.TableInfo; -import com.dtstack.flink.sql.util.MathUtil; - -import java.util.Map; - -public class SqlserverSideParser extends AbsSideTableParser { - - - @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - SqlserverSideTableInfo sqlserverSideTableInfo = new SqlserverSideTableInfo(); - sqlserverSideTableInfo.setName(tableName); - - parseFieldsInfo(fieldsInfo, sqlserverSideTableInfo); - parseCacheProp(sqlserverSideTableInfo, props); - - sqlserverSideTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(SqlserverSideTableInfo.PARALLELISM_KEY.toLowerCase()))); - sqlserverSideTableInfo.setUrl(MathUtil.getString(props.get(SqlserverSideTableInfo.URL_KEY.toLowerCase()))); - sqlserverSideTableInfo.setTableName(MathUtil.getString(props.get(SqlserverSideTableInfo.TABLE_NAME_KEY.toLowerCase()))); - sqlserverSideTableInfo.setUserName(MathUtil.getString(props.get(SqlserverSideTableInfo.USER_NAME_KEY.toLowerCase()))); - sqlserverSideTableInfo.setPassword(MathUtil.getString(props.get(SqlserverSideTableInfo.PASSWORD_KEY.toLowerCase()))); - - return sqlserverSideTableInfo; - } -} diff --git a/sqlserver/pom.xml b/sqlserver/pom.xml index 05f880a3f..463fd55c8 100644 --- a/sqlserver/pom.xml +++ b/sqlserver/pom.xml @@ -18,21 +18,30 @@ 1.3.1 + 1.0-SNAPSHOT + 1.0-SNAPSHOT - net.sourceforge.jtds - jtds - ${jtds.version} + com.dtstack.flink + sql.side.rdb + ${sql.side.rdb.version} + provided com.dtstack.flink - sql.side.rdb - 1.0-SNAPSHOT + sql.core + ${sql.core.version} provided + + + net.sourceforge.jtds + jtds + ${jtds.version} + \ No newline at end of file diff --git a/sqlserver/sqlserver-side/pom.xml b/sqlserver/sqlserver-side/pom.xml index fc9179f62..dac53bed0 100644 --- a/sqlserver/sqlserver-side/pom.xml +++ b/sqlserver/sqlserver-side/pom.xml @@ -15,9 +15,27 @@ sqlserver-side pom + + 1.0-SNAPSHOT + + + sqlserver-side-core sqlserver-all-side - sqlserver-side-core + sqlserver-async-side + + + + com.dtstack.flink + sql.side.rdb + ${rdb.side.version} + + + + + + + \ No newline at end of file diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml b/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml index a11abe0f4..b188aaedf 100644 --- a/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml +++ b/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml @@ -14,11 +14,15 @@ sqlserver-all-side jar + + 1.0-SNAPSHOT + + com.dtstack.flink sql.side.sqlserver.core - 1.0-SNAPSHOT + ${sql.side.sqlserver.core.version} @@ -68,14 +72,14 @@ - + - + diff --git a/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml b/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml index 36940fe66..aed72e25f 100644 --- a/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml +++ b/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml @@ -14,4 +14,5 @@ jar + \ No newline at end of file From 2a0ae3330be1b3b6052f507f2e66f5c957f9c9cb Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Mon, 26 Nov 2018 21:50:30 +0800 Subject: [PATCH 160/470] sqlserver side depend on rdb module --- .../flink/sql/side/rdb/all/RdbAllReqRow.java | 219 ++++++++++++++++++ .../sql/side/rdb/all/RdbAllSideInfo.java | 115 +++++++++ .../sql/side/rdb/async/RdbAsyncReqRow.java | 195 ++++++++++++++++ .../sql/side/rdb/async/RdbAsyncSideInfo.java | 135 +++++++++++ .../sql/side/rdb/table/RdbSideParser.java | 53 +++++ .../sql/side/rdb/table/RdbSideTableInfo.java | 72 ++++++ .../side/sqlserver/SqlserverAllReqRow.java | 47 ++++ .../side/sqlserver/SqlserverAllSideInfo.java | 16 ++ .../sqlserver-async-side/pom.xml | 91 ++++++++ .../side/sqlserver/SqlserverAsyncReqRow.java | 71 ++++++ .../sqlserver/SqlserverAsyncSideInfo.java | 35 +++ .../sqlserver/table/SqlserverSideParser.java | 17 ++ 12 files changed, 1066 insertions(+) create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java create mode 100644 sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java create mode 100644 sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java create mode 100644 sqlserver/sqlserver-side/sqlserver-async-side/pom.xml create mode 100644 sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java create mode 100644 sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncSideInfo.java create mode 100644 sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java new file mode 100644 index 000000000..118ee048f --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java @@ -0,0 +1,219 @@ +/* + * 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 com.dtstack.flink.sql.side.rdb.all; + +import com.dtstack.flink.sql.side.AllReqRow; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import org.apache.calcite.sql.JoinType; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.*; +import java.util.Calendar; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; + +/** + * side operator with cache for all(period reload) + * Date: 2018/11/26 + * Company: www.dtstack.com + * + * @author maqi + */ + +public abstract class RdbAllReqRow extends AllReqRow { + + private static final long serialVersionUID = 2098635140857937718L; + + private static final Logger LOG = LoggerFactory.getLogger(RdbAllReqRow.class); + + //private static final String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; + + private static final int CONN_RETRY_NUM = 3; + + private static final int FETCH_SIZE = 1000; + + private AtomicReference>>> cacheRef = new AtomicReference<>(); + + public RdbAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new RdbAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + @Override + protected Row fillData(Row input, Object sideInput) { + Map cacheInfo = (Map) sideInput; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. + if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { + obj = ((Timestamp) obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for (Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()) { + if (cacheInfo == null) { + row.setField(entry.getKey(), null); + } else { + row.setField(entry.getKey(), cacheInfo.get(entry.getValue())); + } + } + + return row; + } + + @Override + protected void initCache() throws SQLException { + Map>> newCache = Maps.newConcurrentMap(); + cacheRef.set(newCache); + loadData(newCache); + } + + @Override + protected void reloadCache() { + //reload cacheRef and replace to old cacheRef + Map>> newCache = Maps.newConcurrentMap(); + try { + loadData(newCache); + } catch (SQLException e) { + LOG.error("", e); + } + + cacheRef.set(newCache); + LOG.info("----- rdb all cacheRef reload end:{}", Calendar.getInstance()); + } + + + @Override + public void flatMap(Row value, Collector out) throws Exception { + List inputParams = Lists.newArrayList(); + for (Integer conValIndex : sideInfo.getEqualValIndex()) { + Object equalObj = value.getField(conValIndex); + if (equalObj == null) { + out.collect(null); + } + + inputParams.add(equalObj); + } + + String key = buildKey(inputParams); + List> cacheList = cacheRef.get().get(key); + if (CollectionUtils.isEmpty(cacheList)) { + if (sideInfo.getJoinType() == JoinType.LEFT) { + Row row = fillData(value, null); + out.collect(row); + } else { + return; + } + + return; + } + + for (Map one : cacheList) { + out.collect(fillData(value, one)); + } + + } + + private String buildKey(List equalValList) { + StringBuilder sb = new StringBuilder(""); + for (Object equalVal : equalValList) { + sb.append(equalVal).append("_"); + } + + return sb.toString(); + } + + private String buildKey(Map val, List equalFieldList) { + StringBuilder sb = new StringBuilder(""); + for (String equalField : equalFieldList) { + sb.append(val.get(equalField)).append("_"); + } + + return sb.toString(); + } + + public abstract Connection getConn(String dbURL, String userName, String password); + + + private void loadData(Map>> tmpCache) throws SQLException { + RdbSideTableInfo tableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); + Connection connection = null; + + try { + for (int i = 0; i < CONN_RETRY_NUM; i++) { + + try { + connection = getConn(tableInfo.getUrl(), tableInfo.getUserName(), tableInfo.getPassword()); + break; + } catch (Exception e) { + if (i == CONN_RETRY_NUM - 1) { + throw new RuntimeException("", e); + } + + try { + String connInfo = "url:" + tableInfo.getUrl() + ";userName:" + tableInfo.getUserName() + ",pwd:" + tableInfo.getPassword(); + LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + connInfo); + Thread.sleep(5 * 1000); + } catch (InterruptedException e1) { + e1.printStackTrace(); + } + } + + } + + //load data from table + String sql = sideInfo.getSqlCondition(); + Statement statement = connection.createStatement(); + statement.setFetchSize(FETCH_SIZE); + ResultSet resultSet = statement.executeQuery(sql); + String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); + while (resultSet.next()) { + Map oneRow = Maps.newHashMap(); + for (String fieldName : sideFieldNames) { + oneRow.put(fieldName.trim(), resultSet.getObject(fieldName.trim())); + } + + String cacheKey = buildKey(oneRow, sideInfo.getEqualFieldList()); + List> list = tmpCache.computeIfAbsent(cacheKey, key -> Lists.newArrayList()); + list.add(oneRow); + } + } catch (Exception e) { + LOG.error("", e); + } finally { + if (connection != null) { + connection.close(); + } + } + } +} diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java new file mode 100644 index 000000000..501686f95 --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java @@ -0,0 +1,115 @@ +/* + * 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 com.dtstack.flink.sql.side.rdb.all; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.List; + +/** + * Reason: + * Date: 2018/11/26 + * Company: www.dtstack.com + * + * @author maqi + */ + +public class RdbAllSideInfo extends SideInfo { + + private static final long serialVersionUID = -5858335638589472159L; + + + public RdbAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideTableInfo; + + sqlCondition = "select ${selectField} from ${tableName} "; + sqlCondition = sqlCondition.replace("${tableName}", rdbSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); + System.out.println("---------side_exe_sql-----\n" + sqlCondition); + } + + @Override + public void parseSelectFields(JoinInfo joinInfo) { + String sideTableName = joinInfo.getSideTableName(); + String nonSideTableName = joinInfo.getNonSideTable(); + List fields = Lists.newArrayList(); + + int sideIndex = 0; + for (int i = 0; i < outFieldInfoList.size(); i++) { + FieldInfo fieldInfo = outFieldInfoList.get(i); + if (fieldInfo.getTable().equalsIgnoreCase(sideTableName)) { + fields.add(fieldInfo.getFieldName()); + sideFieldIndex.put(i, sideIndex); + sideFieldNameIndex.put(i, fieldInfo.getFieldName()); + sideIndex++; + } else if (fieldInfo.getTable().equalsIgnoreCase(nonSideTableName)) { + int nonSideIndex = rowTypeInfo.getFieldIndex(fieldInfo.getFieldName()); + inFieldIndex.put(i, nonSideIndex); + } else { + throw new RuntimeException("unknown table " + fieldInfo.getTable()); + } + } + + if (fields.size() == 0) { + throw new RuntimeException("select non field from table " + sideTableName); + } + + //add join on condition field to select fields + SqlNode conditionNode = joinInfo.getCondition(); + + List sqlNodeList = Lists.newArrayList(); + if (conditionNode.getKind() == SqlKind.AND) { + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall) conditionNode).getOperands())); + } else { + sqlNodeList.add(conditionNode); + } + + for (SqlNode sqlNode : sqlNodeList) { + dealOneEqualCon(sqlNode, sideTableName); + } + + if (CollectionUtils.isEmpty(equalFieldList)) { + throw new RuntimeException("no join condition found after table " + joinInfo.getLeftTableName()); + } + + for (String equalField : equalFieldList) { + if (fields.contains(equalField)) { + continue; + } + + fields.add(equalField); + } + + sideSelectFields = String.join(",", fields); + } +} diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java new file mode 100644 index 000000000..15f1ee923 --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -0,0 +1,195 @@ +/* + * 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 com.dtstack.flink.sql.side.rdb.async; + +import com.dtstack.flink.sql.enums.ECacheContentType; +import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.cache.CacheObj; +import io.vertx.core.json.JsonArray; +import io.vertx.ext.sql.SQLClient; +import io.vertx.ext.sql.SQLConnection; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Timestamp; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Date: 2018/11/26 + * Company: www.dtstack.com + * + * @author maqi + */ + +public class RdbAsyncReqRow extends AsyncReqRow { + + private static final long serialVersionUID = 2098635244857937720L; + + private static final Logger LOG = LoggerFactory.getLogger(RdbAsyncReqRow.class); + + private transient SQLClient rdbSQLClient; + + public RdbAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new RdbAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + +// @Override +// public void open(Configuration parameters) throws Exception{ +// super.open(parameters); +// } + + @Override + public void asyncInvoke(Row input, ResultFuture resultFuture) throws Exception { + + JsonArray inputParams = new JsonArray(); + for (Integer conValIndex : sideInfo.getEqualValIndex()) { + Object equalObj = input.getField(conValIndex); + if (equalObj == null) { + resultFuture.complete(null); + } + + inputParams.add(equalObj); + } + + String key = buildCacheKey(inputParams); + if (openCache()) { + CacheObj val = getFromCache(key); + if (val != null) { + + if (ECacheContentType.MissVal == val.getType()) { + dealMissKey(input, resultFuture); + return; + } else if (ECacheContentType.MultiLine == val.getType()) { + + for (Object jsonArray : (List) val.getContent()) { + Row row = fillData(input, jsonArray); + resultFuture.complete(Collections.singleton(row)); + } + + } else { + throw new RuntimeException("not support cache obj type " + val.getType()); + } + return; + } + } + + rdbSQLClient.getConnection(conn -> { + if (conn.failed()) { + //Treatment failures + resultFuture.completeExceptionally(conn.cause()); + return; + } + + final SQLConnection connection = conn.result(); + String sqlCondition = sideInfo.getSqlCondition(); + connection.queryWithParams(sqlCondition, inputParams, rs -> { + if (rs.failed()) { + LOG.error("Cannot retrieve the data from the database"); + LOG.error("", rs.cause()); + resultFuture.complete(null); + return; + } + + List cacheContent = Lists.newArrayList(); + + int resultSize = rs.result().getResults().size(); + if (resultSize > 0) { + for (JsonArray line : rs.result().getResults()) { + Row row = fillData(input, line); + if (openCache()) { + cacheContent.add(line); + } + resultFuture.complete(Collections.singleton(row)); + } + + if (openCache()) { + putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); + } + } else { + dealMissKey(input, resultFuture); + if (openCache()) { + putCache(key, CacheMissVal.getMissKeyObj()); + } + } + + // and close the connection + connection.close(done -> { + if (done.failed()) { + throw new RuntimeException(done.cause()); + } + }); + }); + }); + } + + @Override + public Row fillData(Row input, Object line) { + JsonArray jsonArray = (JsonArray) line; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { + obj = ((Timestamp) obj).getTime(); + } + + row.setField(entry.getKey(), obj); + } + + for (Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()) { + if (jsonArray == null) { + row.setField(entry.getKey(), null); + } else { + row.setField(entry.getKey(), jsonArray.getValue(entry.getValue())); + } + } + + return row; + } + + @Override + public void close() throws Exception { + super.close(); + rdbSQLClient.close(); + } + + public String buildCacheKey(JsonArray jsonArray) { + StringBuilder sb = new StringBuilder(); + for (Object ele : jsonArray.getList()) { + sb.append(ele.toString()) + .append("_"); + } + + return sb.toString(); + } + + public void setRdbSQLClient(SQLClient rdbSQLClient) { + this.rdbSQLClient = rdbSQLClient; + } +} diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java new file mode 100644 index 000000000..b7ff94ea9 --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java @@ -0,0 +1,135 @@ +/* + * 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 com.dtstack.flink.sql.side.rdb.async; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.List; + +/** + * Reason: + * Date: 2018/11/26 + * Company: www.dtstack.com + * + * @author maqi + */ + +public class RdbAsyncSideInfo extends SideInfo { + + public RdbAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideTableInfo; + + String sideTableName = joinInfo.getSideTableName(); + + SqlNode conditionNode = joinInfo.getCondition(); + + List sqlNodeList = Lists.newArrayList(); + if (conditionNode.getKind() == SqlKind.AND) { + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall) conditionNode).getOperands())); + } else { + sqlNodeList.add(conditionNode); + } + + for (SqlNode sqlNode : sqlNodeList) { + dealOneEqualCon(sqlNode, sideTableName); + } + + sqlCondition = "select ${selectField} from ${tableName} where "; + for (int i = 0; i < equalFieldList.size(); i++) { + String equalField = equalFieldList.get(i); + + sqlCondition += equalField + "=? "; + if (i != equalFieldList.size() - 1) { + sqlCondition += " and "; + } + } + + sqlCondition = sqlCondition.replace("${tableName}", rdbSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); + System.out.println("---------side_exe_sql-----\n" + sqlCondition); + } + + + @Override + public void dealOneEqualCon(SqlNode sqlNode, String sideTableName) { + if (sqlNode.getKind() != SqlKind.EQUALS) { + throw new RuntimeException("not equal operator."); + } + + SqlIdentifier left = (SqlIdentifier) ((SqlBasicCall) sqlNode).getOperands()[0]; + SqlIdentifier right = (SqlIdentifier) ((SqlBasicCall) sqlNode).getOperands()[1]; + + String leftTableName = left.getComponent(0).getSimple(); + String leftField = left.getComponent(1).getSimple(); + + String rightTableName = right.getComponent(0).getSimple(); + String rightField = right.getComponent(1).getSimple(); + + if (leftTableName.equalsIgnoreCase(sideTableName)) { + equalFieldList.add(leftField); + int equalFieldIndex = -1; + for (int i = 0; i < rowTypeInfo.getFieldNames().length; i++) { + String fieldName = rowTypeInfo.getFieldNames()[i]; + if (fieldName.equalsIgnoreCase(rightField)) { + equalFieldIndex = i; + } + } + if (equalFieldIndex == -1) { + throw new RuntimeException("can't deal equal field: " + sqlNode); + } + + equalValIndex.add(equalFieldIndex); + + } else if (rightTableName.equalsIgnoreCase(sideTableName)) { + + equalFieldList.add(rightField); + int equalFieldIndex = -1; + for (int i = 0; i < rowTypeInfo.getFieldNames().length; i++) { + String fieldName = rowTypeInfo.getFieldNames()[i]; + if (fieldName.equalsIgnoreCase(leftField)) { + equalFieldIndex = i; + } + } + if (equalFieldIndex == -1) { + throw new RuntimeException("can't deal equal field: " + sqlNode.toString()); + } + + equalValIndex.add(equalFieldIndex); + + } else { + throw new RuntimeException("resolve equalFieldList error:" + sqlNode.toString()); + } + + } + +} diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java new file mode 100644 index 000000000..82d013a98 --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java @@ -0,0 +1,53 @@ +/* + * 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 com.dtstack.flink.sql.side.rdb.table; + +import com.dtstack.flink.sql.table.AbsSideTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/26 + * Company: www.dtstack.com + * + * @author maqi + */ + +public class RdbSideParser extends AbsSideTableParser { + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + RdbSideTableInfo rdbTableInfo = new RdbSideTableInfo(); + rdbTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, rdbTableInfo); + + parseCacheProp(rdbTableInfo, props); + rdbTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(RdbSideTableInfo.PARALLELISM_KEY.toLowerCase()))); + rdbTableInfo.setUrl(MathUtil.getString(props.get(RdbSideTableInfo.URL_KEY.toLowerCase()))); + rdbTableInfo.setTableName(MathUtil.getString(props.get(RdbSideTableInfo.TABLE_NAME_KEY.toLowerCase()))); + rdbTableInfo.setUserName(MathUtil.getString(props.get(RdbSideTableInfo.USER_NAME_KEY.toLowerCase()))); + rdbTableInfo.setPassword(MathUtil.getString(props.get(RdbSideTableInfo.PASSWORD_KEY.toLowerCase()))); + + return rdbTableInfo; + } +} diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java new file mode 100644 index 000000000..2117639c4 --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java @@ -0,0 +1,72 @@ +package com.dtstack.flink.sql.side.rdb.table; + +import com.dtstack.flink.sql.side.SideTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * Reason: + * Date: 2018/11/26 + * Company: www.dtstack.com + * + * @author maqi + */ +public class RdbSideTableInfo extends SideTableInfo { + private static final long serialVersionUID = -1L; + + public static final String URL_KEY = "url"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String USER_NAME_KEY = "userName"; + + public static final String PASSWORD_KEY = "password"; + + @Override + public boolean check() { + Preconditions.checkNotNull(url, "rdb of URL is required"); + Preconditions.checkNotNull(tableName, "rdb of tableName is required"); + Preconditions.checkNotNull(userName, "rdb of userName is required"); + Preconditions.checkNotNull(password, "rdb of password is required"); + return true; + } + + private String url; + + private String tableName; + + private String userName; + + private String password; + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } +} diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java new file mode 100644 index 000000000..ef2786f57 --- /dev/null +++ b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java @@ -0,0 +1,47 @@ +package com.dtstack.flink.sql.side.sqlserver; + + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.util.List; +import java.util.Map; + +/** + * side operator with cache for all(period reload) + */ +public class SqlserverAllReqRow extends RdbAllReqRow { + + private static final Logger LOG = LoggerFactory.getLogger(SqlserverAllReqRow.class); + + private static final String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; + + public SqlserverAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public Connection getConn(String dbURL, String userName, String password) { + try { + Class.forName(SQLSERVER_DRIVER); + //add param useCursorFetch=true + Map addParams = Maps.newHashMap(); + //addParams.put("useCursorFetch", "true"); + String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); + return DriverManager.getConnection(targetDbUrl, userName, password); + } catch (Exception e) { + LOG.error("", e); + throw new RuntimeException("", e); + } + } + +} diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java new file mode 100644 index 000000000..f671dc0e6 --- /dev/null +++ b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java @@ -0,0 +1,16 @@ +package com.dtstack.flink.sql.side.sqlserver; + + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import java.util.List; + +public class SqlserverAllSideInfo extends RdbAllSideInfo { + + public SqlserverAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } +} diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml b/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml new file mode 100644 index 000000000..08413dc43 --- /dev/null +++ b/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml @@ -0,0 +1,91 @@ + + + + sql.side.sqlserver + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.async.sqlserver + 1.0-SNAPSHOT + sqlserver-async-side + jar + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.side.sqlserver.core + ${sql.side.sqlserver.core.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java new file mode 100644 index 000000000..1774d4388 --- /dev/null +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java @@ -0,0 +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 com.dtstack.flink.sql.side.sqlserver; + +import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.rdb.async.RdbAsyncReqRow; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import io.vertx.core.Vertx; +import io.vertx.core.VertxOptions; +import io.vertx.core.json.JsonObject; +import io.vertx.ext.sql.SQLClient; +import io.vertx.ext.jdbc.JDBCClient; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + + +public class SqlserverAsyncReqRow extends RdbAsyncReqRow { + + private static final Logger LOG = LoggerFactory.getLogger(SqlserverAsyncReqRow.class); + + private final static String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; + + private final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 10; + + private final static int DEFAULT_VERTX_WORKER_POOL_SIZE = 20; + + private final static int DEFAULT_MAX_DB_CONN_POOL_SIZE = 20; + + public SqlserverAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + JsonObject sqlserverClientConfig = new JsonObject(); + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); + sqlserverClientConfig.put("url", rdbSideTableInfo.getUrl()) + .put("driver_class", SQLSERVER_DRIVER) + .put("max_pool_size", DEFAULT_MAX_DB_CONN_POOL_SIZE) + .put("user", rdbSideTableInfo.getUserName()) + .put("password", rdbSideTableInfo.getPassword()); + + VertxOptions vo = new VertxOptions(); + vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); + vo.setWorkerPoolSize(DEFAULT_VERTX_WORKER_POOL_SIZE); + Vertx vertx = Vertx.vertx(vo); + setRdbSQLClient(JDBCClient.createNonShared(vertx, sqlserverClientConfig)); + } +} diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncSideInfo.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncSideInfo.java new file mode 100644 index 000000000..347ca9bf4 --- /dev/null +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncSideInfo.java @@ -0,0 +1,35 @@ +/* + * 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 com.dtstack.flink.sql.side.sqlserver; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import java.util.List; + + + +public class SqlserverAsyncSideInfo extends RdbAsyncSideInfo { + + public SqlserverAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } +} diff --git a/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java b/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java new file mode 100644 index 000000000..edd4820be --- /dev/null +++ b/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java @@ -0,0 +1,17 @@ +package com.dtstack.flink.sql.side.sqlserver.table; + +import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; +import com.dtstack.flink.sql.table.TableInfo; +import java.util.Map; + + +public class SqlserverSideParser extends RdbSideParser { + private static final String CURR_TYPE = "sqlserver"; + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + TableInfo sqlServerTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + sqlServerTableInfo.setType(CURR_TYPE); + return sqlServerTableInfo; + } +} From 3656f1c3108982673e6a4348783f1243164f4ad7 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Mon, 26 Nov 2018 21:59:15 +0800 Subject: [PATCH 161/470] pom.xml --- rdb/rdb-side/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/rdb/rdb-side/pom.xml b/rdb/rdb-side/pom.xml index 2c1680fb0..c7350877f 100644 --- a/rdb/rdb-side/pom.xml +++ b/rdb/rdb-side/pom.xml @@ -6,6 +6,7 @@ sql.rdb com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 From 874f19892e81f351ad1f9885c4a3d9dea3b0aa67 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Tue, 27 Nov 2018 20:37:12 +0800 Subject: [PATCH 162/470] mysql module depend on rdb --- mysql/mysql-side/mysql-all-side/pom.xml | 10 +- .../flink/sql/side/mysql/MysqlAllReqRow.java | 186 +-------- .../sql/side/mysql/MysqlAllSideInfo.java | 78 +--- mysql/mysql-side/mysql-async-side/pom.xml | 25 +- .../sql/side/mysql/MysqlAsyncReqRow.java | 166 +------- .../sql/side/mysql/MysqlAsyncSideInfo.java | 99 +---- mysql/mysql-side/mysql-side-core/pom.xml | 10 +- .../sql/side/mysql/table/MysqlSideParser.java | 24 +- .../side/mysql/table/MysqlSideTableInfo.java | 99 ----- mysql/mysql-side/pom.xml | 17 +- mysql/mysql-sink/pom.xml | 14 +- .../flink/sql/sink/mysql/MysqlSink.java | 53 +-- .../sink/mysql/RetractJDBCOutputFormat.java | 362 ------------------ .../sql/sink/mysql/table/MysqlSinkParser.java | 23 +- mysql/pom.xml | 13 +- rdb/pom.xml | 1 + .../sql/side/rdb/async/RdbAsyncReqRow.java | 9 +- .../sql/side/rdb/table/RdbSideTableInfo.java | 17 + rdb/rdb-sink/pom.xml | 1 + .../dtstack/flink/sql/sink/rdb}/DBSink.java | 68 ++-- .../dtstack/flink/sql/sink/rdb/RdbSink.java | 71 ++++ .../sql/sink/rdb/RetractJDBCOutputFormat.java | 362 ++++++++++++++++++ .../sql/sink/rdb/table/RdbSinkParser.java | 51 +++ .../sql/sink/rdb/table/RdbTableInfo.java | 35 +- sqlserver/pom.xml | 17 +- sqlserver/sqlserver-side/pom.xml | 4 +- .../sqlserver-side/sqlserver-all-side/pom.xml | 1 + .../side/sqlserver/SqlserverAllReqRow.java | 17 + .../side/sqlserver/SqlserverAllSideInfo.java | 17 + .../sqlserver-async-side/pom.xml | 1 + .../side/sqlserver/SqlserverAsyncReqRow.java | 1 - .../sqlserver-side-core/pom.xml | 3 +- .../sqlserver/table/SqlserverSideParser.java | 17 + sqlserver/sqlserver-sink/pom.xml | 92 +++++ .../sql/sink/sqlserver/SqlserverSink.java | 47 +++ .../sqlserver/table/SqlserverSinkParser.java | 41 ++ 36 files changed, 904 insertions(+), 1148 deletions(-) delete mode 100644 mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java delete mode 100644 mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java rename {mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql => rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb}/DBSink.java (79%) create mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java create mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java create mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java rename mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlTableInfo.java => rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java (81%) create mode 100644 sqlserver/sqlserver-sink/pom.xml create mode 100644 sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java create mode 100644 sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/table/SqlserverSinkParser.java diff --git a/mysql/mysql-side/mysql-all-side/pom.xml b/mysql/mysql-side/mysql-all-side/pom.xml index 4d6feb236..92ce13a85 100644 --- a/mysql/mysql-side/mysql-all-side/pom.xml +++ b/mysql/mysql-side/mysql-all-side/pom.xml @@ -15,11 +15,15 @@ jar + + 1.0-SNAPSHOT + + com.dtstack.flink sql.side.mysql.core - 1.0-SNAPSHOT + ${sql.side.mysql.core.version} @@ -71,12 +75,12 @@ - + + tofile="${basedir}/../../../plugins/mysqlallside/${project.name}.jar"/> diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java index a8839e951..fa4895ded 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java @@ -18,42 +18,30 @@ package com.dtstack.flink.sql.side.mysql; -import com.dtstack.flink.sql.side.AllReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; import com.dtstack.flink.sql.util.DtStringUtil; -import org.apache.calcite.sql.JoinType; -import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; -import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; -import org.apache.flink.types.Row; -import org.apache.flink.util.Collector; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.sql.Connection; import java.sql.DriverManager; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import java.sql.Timestamp; -import java.util.Calendar; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicReference; /** * side operator with cache for all(period reload) * Date: 2018/9/19 * Company: www.dtstack.com + * * @author xuchao */ -public class MysqlAllReqRow extends AllReqRow{ +public class MysqlAllReqRow extends RdbAllReqRow { private static final long serialVersionUID = 2098635140857937717L; @@ -61,176 +49,22 @@ public class MysqlAllReqRow extends AllReqRow{ private static final String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; - private static final int CONN_RETRY_NUM = 3; - - private static final int FETCH_SIZE = 1000; - - private AtomicReference>>> cacheRef = new AtomicReference<>(); - - public MysqlAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo){ - super(new MysqlAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); - } - - @Override - protected Row fillData(Row input, Object sideInput) { - Map cacheInfo = (Map) sideInput; - Row row = new Row(sideInfo.getOutFieldInfoList().size()); - for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ - Object obj = input.getField(entry.getValue()); - boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); - - //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. - if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ - obj = ((Timestamp)obj).getTime(); - } - row.setField(entry.getKey(), obj); - } - - for(Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()){ - if(cacheInfo == null){ - row.setField(entry.getKey(), null); - }else{ - row.setField(entry.getKey(), cacheInfo.get(entry.getValue())); - } - } - - return row; - } - - @Override - protected void initCache() throws SQLException { - Map>> newCache = Maps.newConcurrentMap(); - cacheRef.set(newCache); - loadData(newCache); + public MysqlAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override - protected void reloadCache() { - //reload cacheRef and replace to old cacheRef - Map>> newCache = Maps.newConcurrentMap(); + public Connection getConn(String dbURL, String userName, String password) { try { - loadData(newCache); - } catch (SQLException e) { - LOG.error("", e); - } - - cacheRef.set(newCache); - LOG.info("----- mysql all cacheRef reload end:{}", Calendar.getInstance()); - } - - - @Override - public void flatMap(Row value, Collector out) throws Exception { - List inputParams = Lists.newArrayList(); - for(Integer conValIndex : sideInfo.getEqualValIndex()){ - Object equalObj = value.getField(conValIndex); - if(equalObj == null){ - out.collect(null); - } - - inputParams.add(equalObj); - } - - String key = buildKey(inputParams); - List> cacheList = cacheRef.get().get(key); - if(CollectionUtils.isEmpty(cacheList)){ - if(sideInfo.getJoinType() == JoinType.LEFT){ - Row row = fillData(value, null); - out.collect(row); - }else{ - return; - } - - return; - } - - for(Map one : cacheList){ - out.collect(fillData(value, one)); - } - - } - - private String buildKey(List equalValList){ - StringBuilder sb = new StringBuilder(""); - for(Object equalVal : equalValList){ - sb.append(equalVal).append("_"); - } - - return sb.toString(); - } - - private String buildKey(Map val, List equalFieldList){ - StringBuilder sb = new StringBuilder(""); - for(String equalField : equalFieldList){ - sb.append(val.get(equalField)).append("_"); - } - - return sb.toString(); - } - - private Connection getConn(String dbURL, String userName, String password){ - try{ Class.forName(MYSQL_DRIVER); //add param useCursorFetch=true - Map addParams = Maps.newHashMap(); + Map addParams = Maps.newHashMap(); addParams.put("useCursorFetch", "true"); - String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams,true); + String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); return DriverManager.getConnection(targetDbUrl, userName, password); - }catch (Exception e){ + } catch (Exception e) { LOG.error("", e); throw new RuntimeException("", e); } } - - - private void loadData(Map>> tmpCache) throws SQLException { - MysqlSideTableInfo tableInfo = (MysqlSideTableInfo) sideInfo.getSideTableInfo(); - Connection connection = null; - - try{ - for(int i=0; i oneRow = Maps.newHashMap(); - for(String fieldName : sideFieldNames){ - oneRow.put(fieldName.trim(), resultSet.getObject(fieldName.trim())); - } - - String cacheKey = buildKey(oneRow, sideInfo.getEqualFieldList()); - List> list = tmpCache.computeIfAbsent(cacheKey, key -> Lists.newArrayList()); - list.add(oneRow); - } - }catch (Exception e){ - LOG.error("", e); - }finally { - if(connection != null){ - connection.close(); - } - } - } } diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java index 2364aaac8..81193c0f9 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllSideInfo.java @@ -20,15 +20,9 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlNode; -import org.apache.commons.collections.CollectionUtils; +import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import java.util.List; @@ -36,78 +30,12 @@ * Reason: * Date: 2018/9/19 * Company: www.dtstack.com + * * @author xuchao */ -public class MysqlAllSideInfo extends SideInfo{ - - private static final long serialVersionUID = -5858335638589472158L; - +public class MysqlAllSideInfo extends RdbAllSideInfo { public MysqlAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } - - @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { - MysqlSideTableInfo mysqlSideTableInfo = (MysqlSideTableInfo) sideTableInfo; - - sqlCondition = "select ${selectField} from ${tableName} "; - sqlCondition = sqlCondition.replace("${tableName}", mysqlSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); - System.out.println("---------side_exe_sql-----\n" + sqlCondition); - } - - @Override - public void parseSelectFields(JoinInfo joinInfo){ - String sideTableName = joinInfo.getSideTableName(); - String nonSideTableName = joinInfo.getNonSideTable(); - List fields = Lists.newArrayList(); - - int sideIndex = 0; - for( int i=0; i sqlNodeList = Lists.newArrayList(); - if(conditionNode.getKind() == SqlKind.AND){ - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); - }else{ - sqlNodeList.add(conditionNode); - } - - for(SqlNode sqlNode : sqlNodeList){ - dealOneEqualCon(sqlNode, sideTableName); - } - - if(CollectionUtils.isEmpty(equalFieldList)){ - throw new RuntimeException("no join condition found after table " + joinInfo.getLeftTableName()); - } - - for(String equalField : equalFieldList){ - if(fields.contains(equalField)){ - continue; - } - - fields.add(equalField); - } - - sideSelectFields = String.join(",", fields); - } } diff --git a/mysql/mysql-side/mysql-async-side/pom.xml b/mysql/mysql-side/mysql-async-side/pom.xml index a7cf1a70e..a5588ea97 100644 --- a/mysql/mysql-side/mysql-async-side/pom.xml +++ b/mysql/mysql-side/mysql-async-side/pom.xml @@ -16,27 +16,16 @@ jar - - - - - io.vertx - vertx-jdbc-client - 3.5.2 - - - - io.vertx - vertx-core - 3.5.2 - + + 1.0-SNAPSHOT + + com.dtstack.flink sql.side.mysql.core - 1.0-SNAPSHOT + ${sql.side.mysql.core.version} - @@ -87,12 +76,12 @@ - + + tofile="${basedir}/../../../plugins/mysqlasyncside/${project.name}.jar"/> diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java index f72df5dea..86fe8c8fa 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -16,55 +16,37 @@ * limitations under the License. */ - package com.dtstack.flink.sql.side.mysql; -import com.dtstack.flink.sql.enums.ECacheContentType; -import com.dtstack.flink.sql.side.AsyncReqRow; -import com.dtstack.flink.sql.side.CacheMissVal; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.cache.CacheObj; -import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; +import com.dtstack.flink.sql.side.rdb.async.RdbAsyncReqRow; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import io.vertx.core.Vertx; import io.vertx.core.VertxOptions; -import io.vertx.core.json.JsonArray; import io.vertx.core.json.JsonObject; import io.vertx.ext.jdbc.JDBCClient; -import io.vertx.ext.sql.SQLClient; -import io.vertx.ext.sql.SQLConnection; -import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.functions.async.ResultFuture; -import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; -import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.sql.Timestamp; -import java.util.Collections; import java.util.List; -import java.util.Map; /** * Mysql dim table * Date: 2018/7/27 * Company: www.dtstack.com + * * @author xuchao */ -public class MysqlAsyncReqRow extends AsyncReqRow { - - private static final long serialVersionUID = 2098635244857937717L; +public class MysqlAsyncReqRow extends RdbAsyncReqRow { private static final Logger LOG = LoggerFactory.getLogger(MysqlAsyncReqRow.class); - private transient SQLClient mySQLClient; - private final static String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; private final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 10; @@ -75,152 +57,26 @@ public class MysqlAsyncReqRow extends AsyncReqRow { public MysqlAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(new MysqlAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } @Override public void open(Configuration parameters) throws Exception { super.open(parameters); - JsonObject mySQLClientConfig = new JsonObject(); - MysqlSideTableInfo mysqlSideTableInfo = (MysqlSideTableInfo) sideInfo.getSideTableInfo(); - mySQLClientConfig.put("url", mysqlSideTableInfo.getUrl()) + JsonObject mysqlClientConfig = new JsonObject(); + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); + mysqlClientConfig.put("url", rdbSideTableInfo.getUrl()) .put("driver_class", MYSQL_DRIVER) .put("max_pool_size", DEFAULT_MAX_DB_CONN_POOL_SIZE) - .put("user", mysqlSideTableInfo.getUserName()) - .put("password", mysqlSideTableInfo.getPassword()); + .put("user", rdbSideTableInfo.getUserName()) + .put("password", rdbSideTableInfo.getPassword()); VertxOptions vo = new VertxOptions(); vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); vo.setWorkerPoolSize(DEFAULT_VERTX_WORKER_POOL_SIZE); Vertx vertx = Vertx.vertx(vo); - mySQLClient = JDBCClient.createNonShared(vertx, mySQLClientConfig); - } - - @Override - public void asyncInvoke(Row input, ResultFuture resultFuture) throws Exception { - - JsonArray inputParams = new JsonArray(); - for(Integer conValIndex : sideInfo.getEqualValIndex()){ - Object equalObj = input.getField(conValIndex); - if(equalObj == null){ - resultFuture.complete(null); - } - - inputParams.add(equalObj); - } - - String key = buildCacheKey(inputParams); - if(openCache()){ - CacheObj val = getFromCache(key); - if(val != null){ - - if(ECacheContentType.MissVal == val.getType()){ - dealMissKey(input, resultFuture); - return; - }else if(ECacheContentType.MultiLine == val.getType()){ - - for(Object jsonArray : (List)val.getContent()){ - Row row = fillData(input, jsonArray); - resultFuture.complete(Collections.singleton(row)); - } - - }else{ - throw new RuntimeException("not support cache obj type " + val.getType()); - } - return; - } - } - - mySQLClient.getConnection(conn -> { - if (conn.failed()) { - //Treatment failures - resultFuture.completeExceptionally(conn.cause()); - return; - } - - final SQLConnection connection = conn.result(); - String sqlCondition = sideInfo.getSqlCondition(); - connection.queryWithParams(sqlCondition, inputParams, rs -> { - if (rs.failed()) { - LOG.error("Cannot retrieve the data from the database"); - LOG.error("", rs.cause()); - resultFuture.complete(null); - return; - } - - List cacheContent = Lists.newArrayList(); - - int resultSize = rs.result().getResults().size(); - if(resultSize > 0){ - for (JsonArray line : rs.result().getResults()) { - Row row = fillData(input, line); - if(openCache()){ - cacheContent.add(line); - } - resultFuture.complete(Collections.singleton(row)); - } - - if(openCache()){ - putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); - } - }else{ - dealMissKey(input, resultFuture); - if(openCache()){ - putCache(key, CacheMissVal.getMissKeyObj()); - } - } - - // and close the connection - connection.close(done -> { - if (done.failed()) { - throw new RuntimeException(done.cause()); - } - }); - }); - }); - } - - @Override - public Row fillData(Row input, Object line){ - JsonArray jsonArray = (JsonArray) line; - Row row = new Row(sideInfo.getOutFieldInfoList().size()); - for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ - Object obj = input.getField(entry.getValue()); - boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); - - if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ - obj = ((Timestamp)obj).getTime(); - } - - row.setField(entry.getKey(), obj); - } - - for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ - if(jsonArray == null){ - row.setField(entry.getKey(), null); - }else{ - row.setField(entry.getKey(), jsonArray.getValue(entry.getValue())); - } - } - - return row; - } - - @Override - public void close() throws Exception { - super.close(); - mySQLClient.close(); - } - - public String buildCacheKey(JsonArray jsonArray){ - StringBuilder sb = new StringBuilder(); - for(Object ele : jsonArray.getList()){ - sb.append(ele.toString()) - .append("_"); - } - - return sb.toString(); + setRdbSQLClient(JDBCClient.createNonShared(vertx, mysqlClientConfig)); } } diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncSideInfo.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncSideInfo.java index 18f1220d9..f72671ce0 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncSideInfo.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncSideInfo.java @@ -20,15 +20,9 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; -import com.dtstack.flink.sql.side.mysql.table.MysqlSideTableInfo; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlNode; +import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import java.util.List; @@ -40,98 +34,9 @@ * @author xuchao */ -public class MysqlAsyncSideInfo extends SideInfo { - - private static final long serialVersionUID = -5931494270201575201L; +public class MysqlAsyncSideInfo extends RdbAsyncSideInfo { public MysqlAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } - - @Override - public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { - MysqlSideTableInfo mysqlSideTableInfo = (MysqlSideTableInfo) sideTableInfo; - - String sideTableName = joinInfo.getSideTableName(); - - SqlNode conditionNode = joinInfo.getCondition(); - - List sqlNodeList = Lists.newArrayList(); - if(conditionNode.getKind() == SqlKind.AND){ - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); - }else{ - sqlNodeList.add(conditionNode); - } - - for(SqlNode sqlNode : sqlNodeList){ - dealOneEqualCon(sqlNode, sideTableName); - } - - sqlCondition = "select ${selectField} from ${tableName} where "; - for(int i=0; i4.0.0 sql.side.mysql.core - - - com.dtstack.flink - sql.core - 1.0-SNAPSHOT - provided - - + 1.0-SNAPSHOT jar + mysql-side-core \ No newline at end of file diff --git a/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java b/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java index 94a50e8ab..40f68e7e4 100644 --- a/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java +++ b/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideParser.java @@ -16,40 +16,30 @@ * limitations under the License. */ - package com.dtstack.flink.sql.side.mysql.table; -import com.dtstack.flink.sql.table.AbsSideTableParser; +import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; import com.dtstack.flink.sql.table.TableInfo; -import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; -import java.util.regex.Matcher; -import java.util.regex.Pattern; /** * Reason: * Date: 2018/7/25 * Company: www.dtstack.com + * * @author xuchao */ -public class MysqlSideParser extends AbsSideTableParser { +public class MysqlSideParser extends RdbSideParser { + + private static final String CURR_TYPE = "mysql"; @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - MysqlSideTableInfo mysqlTableInfo = new MysqlSideTableInfo(); - mysqlTableInfo.setName(tableName); - parseFieldsInfo(fieldsInfo, mysqlTableInfo); - - parseCacheProp(mysqlTableInfo, props); - mysqlTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(MysqlSideTableInfo.PARALLELISM_KEY.toLowerCase()))); - mysqlTableInfo.setUrl(MathUtil.getString(props.get(MysqlSideTableInfo.URL_KEY.toLowerCase()))); - mysqlTableInfo.setTableName(MathUtil.getString(props.get(MysqlSideTableInfo.TABLE_NAME_KEY.toLowerCase()))); - mysqlTableInfo.setUserName(MathUtil.getString(props.get(MysqlSideTableInfo.USER_NAME_KEY.toLowerCase()))); - mysqlTableInfo.setPassword(MathUtil.getString(props.get(MysqlSideTableInfo.PASSWORD_KEY.toLowerCase()))); - + TableInfo mysqlTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + mysqlTableInfo.setType(CURR_TYPE); return mysqlTableInfo; } } diff --git a/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java b/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java deleted file mode 100644 index 72ac0dc55..000000000 --- a/mysql/mysql-side/mysql-side-core/src/main/java/com/dtstack/flink/sql/side/mysql/table/MysqlSideTableInfo.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * 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 com.dtstack.flink.sql.side.mysql.table; - -import com.dtstack.flink.sql.side.SideTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; - -/** - * Reason: - * Date: 2018/7/25 - * Company: www.dtstack.com - * @author xuchao - */ - -public class MysqlSideTableInfo extends SideTableInfo { - - private static final long serialVersionUID = -1L; - - private static final String CURR_TYPE = "mysql"; - - public static final String URL_KEY = "url"; - - public static final String TABLE_NAME_KEY = "tableName"; - - public static final String USER_NAME_KEY = "userName"; - - public static final String PASSWORD_KEY = "password"; - - public MysqlSideTableInfo(){ - setType(CURR_TYPE); - } - - @Override - public boolean check() { - Preconditions.checkNotNull(url, "mysql of URL is required"); - Preconditions.checkNotNull(tableName, "mysql of tableName is required"); - Preconditions.checkNotNull(userName, "mysql of userName is required"); - Preconditions.checkNotNull(password, "mysql of password is required"); - return true; - } - - private String url; - - private String tableName; - - private String userName; - - private String password; - - public String getUrl() { - return url; - } - - public void setUrl(String url) { - this.url = url; - } - - public String getTableName() { - return tableName; - } - - public void setTableName(String tableName) { - this.tableName = tableName; - } - - public String getUserName() { - return userName; - } - - public void setUserName(String userName) { - this.userName = userName; - } - - public String getPassword() { - return password; - } - - public void setPassword(String password) { - this.password = password; - } -} diff --git a/mysql/mysql-side/pom.xml b/mysql/mysql-side/pom.xml index 78957c1e0..4f104b07f 100644 --- a/mysql/mysql-side/pom.xml +++ b/mysql/mysql-side/pom.xml @@ -11,13 +11,28 @@ 4.0.0 sql.side.mysql + 1.0-SNAPSHOT mysql-side + pom + mysql-side-core mysql-async-side mysql-all-side - pom + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.side.rdb + ${rdb.side.version} + + + \ No newline at end of file diff --git a/mysql/mysql-sink/pom.xml b/mysql/mysql-sink/pom.xml index 1aeeccba6..01b964da8 100644 --- a/mysql/mysql-sink/pom.xml +++ b/mysql/mysql-sink/pom.xml @@ -14,8 +14,16 @@ mysql-sink http://maven.apache.org - + + 1.0-SNAPSHOT + + + + com.dtstack.flink + sql.sink.rdb + ${sql.sink.rdb.version} + @@ -66,12 +74,12 @@ - + + tofile="${basedir}/../../plugins/mysqlsink/${project.name}.jar"/> diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java index ba6eeb7f2..fe190f10f 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java @@ -16,40 +16,39 @@ * limitations under the License. */ - package com.dtstack.flink.sql.sink.mysql; -import com.dtstack.flink.sql.sink.IStreamSinkGener; -import com.dtstack.flink.sql.sink.mysql.table.MysqlTableInfo; -import com.dtstack.flink.sql.table.TargetTableInfo; +import com.dtstack.flink.sql.sink.rdb.RdbSink; -import java.util.Arrays; import java.util.List; /** * Date: 2017/2/27 * Company: www.dtstack.com + * * @author xuchao */ -public class MysqlSink extends DBSink implements IStreamSinkGener { +public class MysqlSink extends RdbSink { + + private static final String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; - public MysqlSink(){ + public MysqlSink() { } @Override - public void buildSql(String tableName, List fields){ + public void buildSql(String tableName, List fields) { buildInsertSql(tableName, fields); } - private void buildInsertSql(String tableName, List fields){ + private void buildInsertSql(String tableName, List fields) { String sqlTmp = "replace into " + tableName + " (${fields}) values (${placeholder})"; String fieldsStr = ""; String placeholder = ""; - for(String fieldName : fields){ + for (String fieldName : fields) { fieldsStr += ",`" + fieldName + "`"; placeholder += ",?"; } @@ -61,38 +60,10 @@ private void buildInsertSql(String tableName, List fields){ this.sql = sqlTmp; } - @Override - public MysqlSink genStreamSink(TargetTableInfo targetTableInfo) { - - MysqlTableInfo mysqlTableInfo = (MysqlTableInfo) targetTableInfo; - - String tmpDbURL = mysqlTableInfo.getUrl(); - String tmpUserName = mysqlTableInfo.getUserName(); - String tmpPassword = mysqlTableInfo.getPassword(); - String tmpTableName = mysqlTableInfo.getTableName(); - Integer tmpSqlBatchSize = mysqlTableInfo.getBatchSize(); - if(tmpSqlBatchSize != null){ - setBatchInterval(tmpSqlBatchSize); - } - - Integer tmpSinkParallelism = mysqlTableInfo.getParallelism(); - if(tmpSinkParallelism != null){ - setParallelism(tmpSinkParallelism); - } - - List fields = Arrays.asList(mysqlTableInfo.getFields()); - List fieldTypeArray = Arrays.asList(mysqlTableInfo.getFieldClasses()); - - this.driverName = "com.mysql.jdbc.Driver"; - this.dbURL = tmpDbURL; - this.userName = tmpUserName; - this.password = tmpPassword; - this.tableName = tmpTableName; - this.primaryKeys = mysqlTableInfo.getPrimaryKeys(); - buildSql(tableName, fields); - buildSqlTypes(fieldTypeArray); - return this; + @Override + public String getDriverName() { + return MYSQL_DRIVER; } } diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java deleted file mode 100644 index 8ad543ae8..000000000 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/RetractJDBCOutputFormat.java +++ /dev/null @@ -1,362 +0,0 @@ -/* - * 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 com.dtstack.flink.sql.sink.mysql; - -import com.dtstack.flink.sql.metric.MetricConstant; -import org.apache.flink.api.common.io.RichOutputFormat; -import org.apache.flink.api.java.tuple.Tuple; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.MeterView; -import org.apache.flink.types.Row; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.PreparedStatement; -import java.sql.SQLException; - -/** - * OutputFormat to write tuples into a database. - * The OutputFormat has to be configured using the supplied OutputFormatBuilder. - * - * @see Tuple - * @see DriverManager - */ -public class RetractJDBCOutputFormat extends RichOutputFormat { - private static final long serialVersionUID = 1L; - - private static final Logger LOG = LoggerFactory.getLogger(RetractJDBCOutputFormat.class); - - private String username; - private String password; - private String drivername; - private String dbURL; - private String insertQuery; - private String tableName; - private int batchInterval = 5000; - - private Connection dbConn; - private PreparedStatement upload; - - private int batchCount = 0; - - public int[] typesArray; - - private transient Counter outRecords; - - private transient Meter outRecordsRate; - - public RetractJDBCOutputFormat() { - } - - @Override - public void configure(Configuration parameters) { - } - - /** - * Connects to the target database and initializes the prepared statement. - * - * @param taskNumber The number of the parallel instance. - * @throws IOException Thrown, if the output could not be opened due to an - * I/O problem. - */ - @Override - public void open(int taskNumber, int numTasks) throws IOException { - try { - establishConnection(); - upload = dbConn.prepareStatement(insertQuery); - initMetric(); - if (dbConn.getMetaData().getTables(null, null, tableName, null).next()){ - upload = dbConn.prepareStatement(insertQuery); - } else { - throw new SQLException("Table " + tableName +" doesn't exist"); - } - - } catch (SQLException sqe) { - throw new IllegalArgumentException("open() failed.", sqe); - } catch (ClassNotFoundException cnfe) { - throw new IllegalArgumentException("JDBC driver class not found.", cnfe); - } - } - - private void initMetric(){ - outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); - outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); - } - - private void establishConnection() throws SQLException, ClassNotFoundException { - Class.forName(drivername); - if (username == null) { - dbConn = DriverManager.getConnection(dbURL); - } else { - dbConn = DriverManager.getConnection(dbURL, username, password); - } - } - - /** - * Adds a record to the prepared statement. - *

- * When this method is called, the output format is guaranteed to be opened. - *

- * - * WARNING: this may fail when no column types specified (because a best effort approach is attempted in order to - * insert a null value but it's not guaranteed that the JDBC driver handles PreparedStatement.setObject(pos, null)) - * - * @param tuple2 The records to add to the output. - * @see PreparedStatement - * @throws IOException Thrown, if the records could not be added due to an I/O problem. - */ - @Override - public void writeRecord(Tuple2 tuple2) throws IOException { - - Tuple2 tupleTrans = tuple2; - Boolean retract = tupleTrans.getField(0); - Row row = tupleTrans.getField(1); - - - if (typesArray != null && typesArray.length > 0 && typesArray.length != row.getArity()) { - LOG.warn("Column SQL types array doesn't match arity of passed Row! Check the passed array..."); - } - try { - if(retract){ - insertWrite(row); - outRecords.inc(); - }else{ - //do nothing - } - } catch (SQLException | IllegalArgumentException e) { - throw new IllegalArgumentException("writeRecord() failed", e); - } - } - - - private void insertWrite(Row row) throws SQLException { - - updatePreparedStmt(row, upload); - upload.addBatch(); - batchCount++; - if (batchCount >= batchInterval) { - upload.executeBatch(); - batchCount = 0; - } - } - - - private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLException { - if (typesArray == null ) { - // no types provided - for (int index = 0; index < row.getArity(); index++) { - LOG.warn("Unknown column type for column %s. Best effort approach to set its value: %s.", index + 1, row.getField(index)); - pstmt.setObject(index + 1, row.getField(index)); - } - } else { - // types provided - for (int index = 0; index < row.getArity(); index++) { - - if (row.getField(index) == null) { - pstmt.setNull(index + 1, typesArray[index]); - } else { - // casting values as suggested by http://docs.oracle.com/javase/1.5.0/docs/guide/jdbc/getstart/mapping.html - switch (typesArray[index]) { - case java.sql.Types.NULL: - pstmt.setNull(index + 1, typesArray[index]); - break; - case java.sql.Types.BOOLEAN: - case java.sql.Types.BIT: - pstmt.setBoolean(index + 1, (boolean) row.getField(index)); - break; - case java.sql.Types.CHAR: - case java.sql.Types.NCHAR: - case java.sql.Types.VARCHAR: - case java.sql.Types.LONGVARCHAR: - case java.sql.Types.LONGNVARCHAR: - pstmt.setString(index + 1, (String) row.getField(index)); - break; - case java.sql.Types.TINYINT: - pstmt.setByte(index + 1, (byte) row.getField(index)); - break; - case java.sql.Types.SMALLINT: - pstmt.setShort(index + 1, (short) row.getField(index)); - break; - case java.sql.Types.INTEGER: - pstmt.setInt(index + 1, (int) row.getField(index)); - break; - case java.sql.Types.BIGINT: - pstmt.setLong(index + 1, (long) row.getField(index)); - break; - case java.sql.Types.REAL: - case java.sql.Types.FLOAT: - pstmt.setFloat(index + 1, (float) row.getField(index)); - break; - case java.sql.Types.DOUBLE: - pstmt.setDouble(index + 1, (double) row.getField(index)); - break; - case java.sql.Types.DECIMAL: - case java.sql.Types.NUMERIC: - pstmt.setBigDecimal(index + 1, (java.math.BigDecimal) row.getField(index)); - break; - case java.sql.Types.DATE: - pstmt.setDate(index + 1, (java.sql.Date) row.getField(index)); - break; - case java.sql.Types.TIME: - pstmt.setTime(index + 1, (java.sql.Time) row.getField(index)); - break; - case java.sql.Types.TIMESTAMP: - pstmt.setTimestamp(index + 1, (java.sql.Timestamp) row.getField(index)); - break; - case java.sql.Types.BINARY: - case java.sql.Types.VARBINARY: - case java.sql.Types.LONGVARBINARY: - pstmt.setBytes(index + 1, (byte[]) row.getField(index)); - break; - default: - pstmt.setObject(index + 1, row.getField(index)); - LOG.warn("Unmanaged sql type (%s) for column %s. Best effort approach to set its value: %s.", - typesArray[index], index + 1, row.getField(index)); - // case java.sql.Types.SQLXML - // case java.sql.Types.ARRAY: - // case java.sql.Types.JAVA_OBJECT: - // case java.sql.Types.BLOB: - // case java.sql.Types.CLOB: - // case java.sql.Types.NCLOB: - // case java.sql.Types.DATALINK: - // case java.sql.Types.DISTINCT: - // case java.sql.Types.OTHER: - // case java.sql.Types.REF: - // case java.sql.Types.ROWID: - // case java.sql.Types.STRUC - } - } - } - } - } - - /** - * Executes prepared statement and closes all resources of this instance. - * - * @throws IOException Thrown, if the input could not be closed properly. - */ - @Override - public void close() throws IOException { - try { - if (upload != null) { - upload.executeBatch(); - upload.close(); - } - } catch (SQLException se) { - LOG.info("Inputformat couldn't be closed - " + se.getMessage()); - } finally { - upload = null; - batchCount = 0; - } - - try { - if (dbConn != null) { - dbConn.close(); - } - } catch (SQLException se) { - LOG.info("Inputformat couldn't be closed - " + se.getMessage()); - } finally { - dbConn = null; - } - } - - public static JDBCOutputFormatBuilder buildJDBCOutputFormat() { - return new JDBCOutputFormatBuilder(); - } - - public static class JDBCOutputFormatBuilder { - private final RetractJDBCOutputFormat format; - - protected JDBCOutputFormatBuilder() { - this.format = new RetractJDBCOutputFormat(); - } - - public JDBCOutputFormatBuilder setUsername(String username) { - format.username = username; - return this; - } - - public JDBCOutputFormatBuilder setPassword(String password) { - format.password = password; - return this; - } - - public JDBCOutputFormatBuilder setDrivername(String drivername) { - format.drivername = drivername; - return this; - } - - public JDBCOutputFormatBuilder setDBUrl(String dbURL) { - format.dbURL = dbURL; - return this; - } - - public JDBCOutputFormatBuilder setInsertQuery(String query) { - format.insertQuery = query; - return this; - } - - - public JDBCOutputFormatBuilder setBatchInterval(int batchInterval) { - format.batchInterval = batchInterval; - return this; - } - - public JDBCOutputFormatBuilder setSqlTypes(int[] typesArray) { - format.typesArray = typesArray; - return this; - } - - public JDBCOutputFormatBuilder setTableName(String tableName) { - format.tableName = tableName; - return this; - } - - /** - * Finalizes the configuration and checks validity. - * - * @return Configured RetractJDBCOutputFormat - */ - public RetractJDBCOutputFormat finish() { - if (format.username == null) { - LOG.info("Username was not supplied separately."); - } - if (format.password == null) { - LOG.info("Password was not supplied separately."); - } - if (format.dbURL == null) { - throw new IllegalArgumentException("No dababase URL supplied."); - } - if (format.insertQuery == null) { - throw new IllegalArgumentException("No insertQuery suplied"); - } - if (format.drivername == null) { - throw new IllegalArgumentException("No driver supplied"); - } - return format; - } - } - -} diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java index f1522226b..2247eb8cc 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlSinkParser.java @@ -16,13 +16,11 @@ * limitations under the License. */ - package com.dtstack.flink.sql.sink.mysql.table; -import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; import com.dtstack.flink.sql.table.TableInfo; -import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; @@ -30,26 +28,17 @@ * Reason: * Date: 2018/7/4 * Company: www.dtstack.com + * * @author xuchao */ -public class MysqlSinkParser extends AbsTableParser { +public class MysqlSinkParser extends RdbSinkParser { + private static final String CURR_TYPE = "mysql"; @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - MysqlTableInfo mysqlTableInfo = new MysqlTableInfo(); - mysqlTableInfo.setName(tableName); - parseFieldsInfo(fieldsInfo, mysqlTableInfo); - - mysqlTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(MysqlTableInfo.PARALLELISM_KEY.toLowerCase()))); - mysqlTableInfo.setUrl(MathUtil.getString(props.get(MysqlTableInfo.URL_KEY.toLowerCase()))); - mysqlTableInfo.setTableName(MathUtil.getString(props.get(MysqlTableInfo.TABLE_NAME_KEY.toLowerCase()))); - mysqlTableInfo.setUserName(MathUtil.getString(props.get(MysqlTableInfo.USER_NAME_KEY.toLowerCase()))); - mysqlTableInfo.setPassword(MathUtil.getString(props.get(MysqlTableInfo.PASSWORD_KEY.toLowerCase()))); - mysqlTableInfo.setBatchSize(MathUtil.getIntegerVal(props.get(MysqlTableInfo.BATCH_SIZE_KEY.toLowerCase()))); - mysqlTableInfo.setBufferSize(MathUtil.getString(props.get(MysqlTableInfo.BUFFER_SIZE_KEY.toLowerCase()))); - mysqlTableInfo.setFlushIntervalMs(MathUtil.getString(props.get(MysqlTableInfo.FLUSH_INTERVALMS_KEY.toLowerCase()))); - + TableInfo mysqlTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + mysqlTableInfo.setType(CURR_TYPE); return mysqlTableInfo; } } diff --git a/mysql/pom.xml b/mysql/pom.xml index 747646f74..36221bcff 100644 --- a/mysql/pom.xml +++ b/mysql/pom.xml @@ -6,8 +6,10 @@ flink.sql com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 + sql.mysql pom @@ -16,6 +18,11 @@ mysql-side + + 5.1.46 + 1.0-SNAPSHOT + + junit @@ -23,16 +30,18 @@ 3.8.1 test + com.dtstack.flink sql.core - 1.0-SNAPSHOT + ${sql.core.version} provided + mysql mysql-connector-java - 5.1.46 + ${mysql.connector.version} diff --git a/rdb/pom.xml b/rdb/pom.xml index 0ef3aa252..5cd5d4187 100644 --- a/rdb/pom.xml +++ b/rdb/pom.xml @@ -6,6 +6,7 @@ flink.sql com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index 15f1ee923..66945c513 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -58,10 +58,6 @@ public RdbAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List resultFuture) throws Exception { @@ -176,7 +172,10 @@ public Row fillData(Row input, Object line) { @Override public void close() throws Exception { super.close(); - rdbSQLClient.close(); + if (rdbSQLClient != null) { + rdbSQLClient.close(); + } + } public String buildCacheKey(JsonArray jsonArray) { diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java index 2117639c4..c08f9a3a9 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java @@ -1,3 +1,20 @@ +/** + * 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 com.dtstack.flink.sql.side.rdb.table; import com.dtstack.flink.sql.side.SideTableInfo; diff --git a/rdb/rdb-sink/pom.xml b/rdb/rdb-sink/pom.xml index 6b4868c95..176615769 100644 --- a/rdb/rdb-sink/pom.xml +++ b/rdb/rdb-sink/pom.xml @@ -6,6 +6,7 @@ sql.rdb com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java similarity index 79% rename from mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java rename to rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java index c3526ecd1..35ecf926c 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/DBSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -6,19 +6,16 @@ * 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 - * + *

+ * 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 com.dtstack.flink.sql.sink.mysql; +package com.dtstack.flink.sql.sink.rdb; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; @@ -38,11 +35,12 @@ import java.util.List; /** - * Date: 2017/2/27 + * Reason: + * Date: 2018/11/27 * Company: www.dtstack.com - * @author xuchao + * + * @author maqi */ - public abstract class DBSink implements RetractStreamTableSink { protected String driverName; @@ -69,10 +67,10 @@ public abstract class DBSink implements RetractStreamTableSink { private int parallelism = -1; - public RichSinkFunction createJdbcSinkFunc(){ + public RichSinkFunction createJdbcSinkFunc() { - if(driverName == null || dbURL == null || userName == null - || password == null || sqlTypes == null || tableName == null){ + if (driverName == null || dbURL == null || userName == null + || password == null || sqlTypes == null || tableName == null) { throw new RuntimeException("any of params in(driverName, dbURL, userName, password, type, tableName) " + " must not be null. please check it!!!"); } @@ -95,34 +93,35 @@ public RichSinkFunction createJdbcSinkFunc(){ /** * By now specified class type conversion. * FIXME Follow-up has added a new type of time needs to be modified + * * @param fieldTypeArray */ - protected void buildSqlTypes(List fieldTypeArray){ + protected void buildSqlTypes(List fieldTypeArray) { int[] tmpFieldsType = new int[fieldTypeArray.size()]; - for(int i=0; i fieldTypeArray){ /** * Set the default frequency submit updated every submission + * * @param batchInterval */ public void setBatchInterval(int batchInterval) { @@ -143,7 +143,7 @@ public void emitDataStream(DataStream> dataStream) { RichSinkFunction richSinkFunction = createJdbcSinkFunc(); DataStreamSink streamSink = dataStream.addSink(richSinkFunction); streamSink.name(tableName); - if(parallelism > 0){ + if (parallelism > 0) { streamSink.setParallelism(parallelism); } } @@ -176,11 +176,15 @@ public TypeInformation[] getFieldTypes() { } - public void setParallelism(int parallelism){ + public void setParallelism(int parallelism) { this.parallelism = parallelism; } - public void buildSql(String tableName, List fields){ - throw new RuntimeException("you need to overwrite this method in your own class."); - } + /** + * you need to implements this method in your own class. + * + * @param tableName + * @param fields + */ + public abstract void buildSql(String tableName, List fields); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java new file mode 100644 index 000000000..ae1630a38 --- /dev/null +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java @@ -0,0 +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 com.dtstack.flink.sql.sink.rdb; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.rdb.table.RdbTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; + +import java.util.Arrays; +import java.util.List; + +/** + * Reason: + * Date: 2018/11/27 + * Company: www.dtstack.com + * + * @author maqi + */ +public abstract class RdbSink extends DBSink implements IStreamSinkGener { + + @Override + public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { + RdbTableInfo rdbTableInfo = (RdbTableInfo) targetTableInfo; + + String tmpDbURL = rdbTableInfo.getUrl(); + String tmpUserName = rdbTableInfo.getUserName(); + String tmpPassword = rdbTableInfo.getPassword(); + String tmpTableName = rdbTableInfo.getTableName(); + + Integer tmpSqlBatchSize = rdbTableInfo.getBatchSize(); + if (tmpSqlBatchSize != null) { + setBatchInterval(tmpSqlBatchSize); + } + + Integer tmpSinkParallelism = rdbTableInfo.getParallelism(); + if (tmpSinkParallelism != null) { + setParallelism(tmpSinkParallelism); + } + + List fields = Arrays.asList(rdbTableInfo.getFields()); + List fieldTypeArray = Arrays.asList(rdbTableInfo.getFieldClasses()); + + this.driverName = getDriverName(); + this.dbURL = tmpDbURL; + this.userName = tmpUserName; + this.password = tmpPassword; + this.tableName = tmpTableName; + this.primaryKeys = rdbTableInfo.getPrimaryKeys(); + buildSql(tableName, fields); + buildSqlTypes(fieldTypeArray); + return this; + } + + public abstract String getDriverName(); + +} diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java new file mode 100644 index 000000000..a47b6472c --- /dev/null +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java @@ -0,0 +1,362 @@ +/* + * 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 com.dtstack.flink.sql.sink.rdb; + +import com.dtstack.flink.sql.metric.MetricConstant; +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.SQLException; + +/** + * OutputFormat to write tuples into a database. + * The OutputFormat has to be configured using the supplied OutputFormatBuilder. + * + * @see Tuple + * @see DriverManager + */ +public class RetractJDBCOutputFormat extends RichOutputFormat { + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(RetractJDBCOutputFormat.class); + + private String username; + private String password; + private String drivername; + private String dbURL; + private String insertQuery; + private String tableName; + private int batchInterval = 5000; + + private Connection dbConn; + private PreparedStatement upload; + + private int batchCount = 0; + + public int[] typesArray; + + private transient Counter outRecords; + + private transient Meter outRecordsRate; + + public RetractJDBCOutputFormat() { + } + + @Override + public void configure(Configuration parameters) { + } + + /** + * Connects to the target database and initializes the prepared statement. + * + * @param taskNumber The number of the parallel instance. + * @throws IOException Thrown, if the output could not be opened due to an + * I/O problem. + */ + @Override + public void open(int taskNumber, int numTasks) throws IOException { + try { + establishConnection(); + upload = dbConn.prepareStatement(insertQuery); + initMetric(); + if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { + upload = dbConn.prepareStatement(insertQuery); + } else { + throw new SQLException("Table " + tableName + " doesn't exist"); + } + + } catch (SQLException sqe) { + throw new IllegalArgumentException("open() failed.", sqe); + } catch (ClassNotFoundException cnfe) { + throw new IllegalArgumentException("JDBC driver class not found.", cnfe); + } + } + + private void initMetric() { + outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); + } + + private void establishConnection() throws SQLException, ClassNotFoundException { + Class.forName(drivername); + if (username == null) { + dbConn = DriverManager.getConnection(dbURL); + } else { + dbConn = DriverManager.getConnection(dbURL, username, password); + } + } + + /** + * Adds a record to the prepared statement. + *

+ * When this method is called, the output format is guaranteed to be opened. + *

+ *

+ * WARNING: this may fail when no column types specified (because a best effort approach is attempted in order to + * insert a null value but it's not guaranteed that the JDBC driver handles PreparedStatement.setObject(pos, null)) + * + * @param tuple2 The records to add to the output. + * @throws IOException Thrown, if the records could not be added due to an I/O problem. + * @see PreparedStatement + */ + @Override + public void writeRecord(Tuple2 tuple2) throws IOException { + + Tuple2 tupleTrans = tuple2; + Boolean retract = tupleTrans.getField(0); + Row row = tupleTrans.getField(1); + + + if (typesArray != null && typesArray.length > 0 && typesArray.length != row.getArity()) { + LOG.warn("Column SQL types array doesn't match arity of passed Row! Check the passed array..."); + } + try { + if (retract) { + insertWrite(row); + outRecords.inc(); + } else { + //do nothing + } + } catch (SQLException | IllegalArgumentException e) { + throw new IllegalArgumentException("writeRecord() failed", e); + } + } + + + private void insertWrite(Row row) throws SQLException { + + updatePreparedStmt(row, upload); + upload.addBatch(); + batchCount++; + if (batchCount >= batchInterval) { + upload.executeBatch(); + batchCount = 0; + } + } + + + private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLException { + if (typesArray == null) { + // no types provided + for (int index = 0; index < row.getArity(); index++) { + LOG.warn("Unknown column type for column %s. Best effort approach to set its value: %s.", index + 1, row.getField(index)); + pstmt.setObject(index + 1, row.getField(index)); + } + } else { + // types provided + for (int index = 0; index < row.getArity(); index++) { + + if (row.getField(index) == null) { + pstmt.setNull(index + 1, typesArray[index]); + } else { + // casting values as suggested by http://docs.oracle.com/javase/1.5.0/docs/guide/jdbc/getstart/mapping.html + switch (typesArray[index]) { + case java.sql.Types.NULL: + pstmt.setNull(index + 1, typesArray[index]); + break; + case java.sql.Types.BOOLEAN: + case java.sql.Types.BIT: + pstmt.setBoolean(index + 1, (boolean) row.getField(index)); + break; + case java.sql.Types.CHAR: + case java.sql.Types.NCHAR: + case java.sql.Types.VARCHAR: + case java.sql.Types.LONGVARCHAR: + case java.sql.Types.LONGNVARCHAR: + pstmt.setString(index + 1, (String) row.getField(index)); + break; + case java.sql.Types.TINYINT: + pstmt.setByte(index + 1, (byte) row.getField(index)); + break; + case java.sql.Types.SMALLINT: + pstmt.setShort(index + 1, (short) row.getField(index)); + break; + case java.sql.Types.INTEGER: + pstmt.setInt(index + 1, (int) row.getField(index)); + break; + case java.sql.Types.BIGINT: + pstmt.setLong(index + 1, (long) row.getField(index)); + break; + case java.sql.Types.REAL: + case java.sql.Types.FLOAT: + pstmt.setFloat(index + 1, (float) row.getField(index)); + break; + case java.sql.Types.DOUBLE: + pstmt.setDouble(index + 1, (double) row.getField(index)); + break; + case java.sql.Types.DECIMAL: + case java.sql.Types.NUMERIC: + pstmt.setBigDecimal(index + 1, (java.math.BigDecimal) row.getField(index)); + break; + case java.sql.Types.DATE: + pstmt.setDate(index + 1, (java.sql.Date) row.getField(index)); + break; + case java.sql.Types.TIME: + pstmt.setTime(index + 1, (java.sql.Time) row.getField(index)); + break; + case java.sql.Types.TIMESTAMP: + pstmt.setTimestamp(index + 1, (java.sql.Timestamp) row.getField(index)); + break; + case java.sql.Types.BINARY: + case java.sql.Types.VARBINARY: + case java.sql.Types.LONGVARBINARY: + pstmt.setBytes(index + 1, (byte[]) row.getField(index)); + break; + default: + pstmt.setObject(index + 1, row.getField(index)); + LOG.warn("Unmanaged sql type (%s) for column %s. Best effort approach to set its value: %s.", + typesArray[index], index + 1, row.getField(index)); + // case java.sql.Types.SQLXML + // case java.sql.Types.ARRAY: + // case java.sql.Types.JAVA_OBJECT: + // case java.sql.Types.BLOB: + // case java.sql.Types.CLOB: + // case java.sql.Types.NCLOB: + // case java.sql.Types.DATALINK: + // case java.sql.Types.DISTINCT: + // case java.sql.Types.OTHER: + // case java.sql.Types.REF: + // case java.sql.Types.ROWID: + // case java.sql.Types.STRUC + } + } + } + } + } + + /** + * Executes prepared statement and closes all resources of this instance. + * + * @throws IOException Thrown, if the input could not be closed properly. + */ + @Override + public void close() throws IOException { + try { + if (upload != null) { + upload.executeBatch(); + upload.close(); + } + } catch (SQLException se) { + LOG.info("Inputformat couldn't be closed - " + se.getMessage()); + } finally { + upload = null; + batchCount = 0; + } + + try { + if (dbConn != null) { + dbConn.close(); + } + } catch (SQLException se) { + LOG.info("Inputformat couldn't be closed - " + se.getMessage()); + } finally { + dbConn = null; + } + } + + public static JDBCOutputFormatBuilder buildJDBCOutputFormat() { + return new JDBCOutputFormatBuilder(); + } + + public static class JDBCOutputFormatBuilder { + private final RetractJDBCOutputFormat format; + + protected JDBCOutputFormatBuilder() { + this.format = new RetractJDBCOutputFormat(); + } + + public JDBCOutputFormatBuilder setUsername(String username) { + format.username = username; + return this; + } + + public JDBCOutputFormatBuilder setPassword(String password) { + format.password = password; + return this; + } + + public JDBCOutputFormatBuilder setDrivername(String drivername) { + format.drivername = drivername; + return this; + } + + public JDBCOutputFormatBuilder setDBUrl(String dbURL) { + format.dbURL = dbURL; + return this; + } + + public JDBCOutputFormatBuilder setInsertQuery(String query) { + format.insertQuery = query; + return this; + } + + + public JDBCOutputFormatBuilder setBatchInterval(int batchInterval) { + format.batchInterval = batchInterval; + return this; + } + + public JDBCOutputFormatBuilder setSqlTypes(int[] typesArray) { + format.typesArray = typesArray; + return this; + } + + public JDBCOutputFormatBuilder setTableName(String tableName) { + format.tableName = tableName; + return this; + } + + /** + * Finalizes the configuration and checks validity. + * + * @return Configured RetractJDBCOutputFormat + */ + public RetractJDBCOutputFormat finish() { + if (format.username == null) { + LOG.info("Username was not supplied separately."); + } + if (format.password == null) { + LOG.info("Password was not supplied separately."); + } + if (format.dbURL == null) { + throw new IllegalArgumentException("No dababase URL supplied."); + } + if (format.insertQuery == null) { + throw new IllegalArgumentException("No insertQuery suplied"); + } + if (format.drivername == null) { + throw new IllegalArgumentException("No driver supplied"); + } + return format; + } + } + +} diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java new file mode 100644 index 000000000..a14825a67 --- /dev/null +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java @@ -0,0 +1,51 @@ +/** + * 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 com.dtstack.flink.sql.sink.rdb.table; + +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/27 + * Company: www.dtstack.com + * + * @author maqi + */ +public class RdbSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + RdbTableInfo rdbTableInfo = new RdbTableInfo(); + rdbTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, rdbTableInfo); + + rdbTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(RdbTableInfo.PARALLELISM_KEY.toLowerCase()))); + rdbTableInfo.setUrl(MathUtil.getString(props.get(RdbTableInfo.URL_KEY.toLowerCase()))); + rdbTableInfo.setTableName(MathUtil.getString(props.get(RdbTableInfo.TABLE_NAME_KEY.toLowerCase()))); + rdbTableInfo.setUserName(MathUtil.getString(props.get(RdbTableInfo.USER_NAME_KEY.toLowerCase()))); + rdbTableInfo.setPassword(MathUtil.getString(props.get(RdbTableInfo.PASSWORD_KEY.toLowerCase()))); + rdbTableInfo.setBatchSize(MathUtil.getIntegerVal(props.get(RdbTableInfo.BATCH_SIZE_KEY.toLowerCase()))); + rdbTableInfo.setBufferSize(MathUtil.getString(props.get(RdbTableInfo.BUFFER_SIZE_KEY.toLowerCase()))); + rdbTableInfo.setFlushIntervalMs(MathUtil.getString(props.get(RdbTableInfo.FLUSH_INTERVALMS_KEY.toLowerCase()))); + + return rdbTableInfo; + } +} diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlTableInfo.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java similarity index 81% rename from mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlTableInfo.java rename to rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java index 0576c466e..f170b88e1 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/table/MysqlTableInfo.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -6,33 +6,30 @@ * 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 - * + *

+ * 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 com.dtstack.flink.sql.sink.mysql.table; +package com.dtstack.flink.sql.sink.rdb.table; import com.dtstack.flink.sql.table.TargetTableInfo; import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; /** * Reason: - * Date: 2018/6/22 + * Date: 2018/11/27 * Company: www.dtstack.com - * @author xuchao + * + * @author maqi */ +public class RdbTableInfo extends TargetTableInfo { -public class MysqlTableInfo extends TargetTableInfo { - - private static final String CURR_TYPE = "mysql"; + //private static final String CURR_TYPE = "mysql"; public static final String URL_KEY = "url"; @@ -48,10 +45,6 @@ public class MysqlTableInfo extends TargetTableInfo { public static final String FLUSH_INTERVALMS_KEY = "flushIntervalMs"; - public MysqlTableInfo(){ - setType(CURR_TYPE); - } - private String url; private String tableName; @@ -124,10 +117,10 @@ public void setFlushIntervalMs(String flushIntervalMs) { @Override public boolean check() { - Preconditions.checkNotNull(url, "mysql field of URL is required"); - Preconditions.checkNotNull(tableName, "mysql field of tableName is required"); - Preconditions.checkNotNull(userName, "mysql field of userName is required"); - Preconditions.checkNotNull(password, "mysql field of password is required"); + Preconditions.checkNotNull(url, "rdb field of URL is required"); + Preconditions.checkNotNull(tableName, "rdb field of tableName is required"); + Preconditions.checkNotNull(userName, "rdb field of userName is required"); + Preconditions.checkNotNull(password, "rdb field of password is required"); return true; } diff --git a/sqlserver/pom.xml b/sqlserver/pom.xml index 463fd55c8..de00840e8 100644 --- a/sqlserver/pom.xml +++ b/sqlserver/pom.xml @@ -6,30 +6,25 @@ flink.sql com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 sql.sqlserver 1.0-SNAPSHOT - - sqlserver-side - pom 1.3.1 1.0-SNAPSHOT - 1.0-SNAPSHOT - - - com.dtstack.flink - sql.side.rdb - ${sql.side.rdb.version} - provided - + + sqlserver-side + sqlserver-sink + + com.dtstack.flink sql.core diff --git a/sqlserver/sqlserver-side/pom.xml b/sqlserver/sqlserver-side/pom.xml index dac53bed0..3848021e6 100644 --- a/sqlserver/sqlserver-side/pom.xml +++ b/sqlserver/sqlserver-side/pom.xml @@ -6,12 +6,12 @@ sql.sqlserver com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 sql.side.sqlserver 1.0-SNAPSHOT - sqlserver-side pom @@ -32,8 +32,6 @@ sql.side.rdb ${rdb.side.version} - - diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml b/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml index b188aaedf..7e3db7ecd 100644 --- a/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml +++ b/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml @@ -6,6 +6,7 @@ sql.side.sqlserver com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java index ef2786f57..d123c8f9c 100644 --- a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java @@ -1,3 +1,20 @@ +/** + * 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 com.dtstack.flink.sql.side.sqlserver; diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java index f671dc0e6..2b9913e7e 100644 --- a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java +++ b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllSideInfo.java @@ -1,3 +1,20 @@ +/** + * 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 com.dtstack.flink.sql.side.sqlserver; diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml b/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml index 08413dc43..902228d01 100644 --- a/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml +++ b/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml @@ -6,6 +6,7 @@ sql.side.sqlserver com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java index 1774d4388..4f39129ca 100644 --- a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java @@ -25,7 +25,6 @@ import io.vertx.core.Vertx; import io.vertx.core.VertxOptions; import io.vertx.core.json.JsonObject; -import io.vertx.ext.sql.SQLClient; import io.vertx.ext.jdbc.JDBCClient; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; diff --git a/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml b/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml index aed72e25f..dcff068b1 100644 --- a/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml +++ b/sqlserver/sqlserver-side/sqlserver-side-core/pom.xml @@ -6,13 +6,14 @@ sql.side.sqlserver com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 sql.side.sqlserver.core 1.0-SNAPSHOT jar - + sqlserver-side-core \ No newline at end of file diff --git a/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java b/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java index edd4820be..3631dcd8a 100644 --- a/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java +++ b/sqlserver/sqlserver-side/sqlserver-side-core/src/main/java/com/dtstack/flink/sql/side/sqlserver/table/SqlserverSideParser.java @@ -1,3 +1,20 @@ +/** + * 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 com.dtstack.flink.sql.side.sqlserver.table; import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; diff --git a/sqlserver/sqlserver-sink/pom.xml b/sqlserver/sqlserver-sink/pom.xml new file mode 100644 index 000000000..1180f1ca8 --- /dev/null +++ b/sqlserver/sqlserver-sink/pom.xml @@ -0,0 +1,92 @@ + + + + sql.sqlserver + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.sqlserver + 1.0-SNAPSHOT + sqlserver-sink + + + 1.0-SNAPSHOT + + + + + + com.dtstack.flink + sql.sink.rdb + ${sql.sink.rdb.version} + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java new file mode 100644 index 000000000..986486445 --- /dev/null +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java @@ -0,0 +1,47 @@ +/** + * 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 com.dtstack.flink.sql.sink.sqlserver; + +import com.dtstack.flink.sql.sink.rdb.RdbSink; + +import java.util.List; + +/** + * Reason: + * Date: 2018/11/27 + * Company: www.dtstack.com + * + * @author maqi + */ +public class SqlserverSink extends RdbSink { + private static final String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; + + @Override + public String getDriverName() { + return SQLSERVER_DRIVER; + } + + @Override + public void buildSql(String tableName, List fields) { + buildInsertSql(tableName, fields); + } + + private void buildInsertSql(String tableName, List fields) { + + } +} diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/table/SqlserverSinkParser.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/table/SqlserverSinkParser.java new file mode 100644 index 000000000..a695d6c3b --- /dev/null +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/table/SqlserverSinkParser.java @@ -0,0 +1,41 @@ +/** + * 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 com.dtstack.flink.sql.sink.sqlserver.table; + +import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; +import com.dtstack.flink.sql.table.TableInfo; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/27 + * Company: www.dtstack.com + * + * @author maqi + */ +public class SqlserverSinkParser extends RdbSinkParser { + private static final String CURR_TYPE = "sqlserver"; + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + TableInfo sqlserverTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + sqlserverTableInfo.setType(CURR_TYPE); + return sqlserverTableInfo; + } +} From e23bf6a7e73f59c553945d07bd5c6db8d0c2a3e6 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Tue, 27 Nov 2018 20:45:30 +0800 Subject: [PATCH 163/470] Update README.md --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 27dffc8dc..3abfabbc3 100644 --- a/README.md +++ b/README.md @@ -17,6 +17,7 @@ * 增加kafka结果表功能 * 增加SQL支持CEP * 维表快照 + * sql优化(谓词下移等) ## 1 快速起步 ### 1.1 运行模式 From 5721eda7a55a29b70fe7873d747f02714632dd42 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Tue, 27 Nov 2018 21:16:12 +0800 Subject: [PATCH 164/470] fetch size --- .../dtstack/flink/sql/side/mysql/MysqlAllReqRow.java | 5 +++++ .../dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java | 11 ++++++----- .../flink/sql/side/rdb/async/RdbAsyncReqRow.java | 3 +-- 3 files changed, 12 insertions(+), 7 deletions(-) diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java index fa4895ded..b56c3252b 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java @@ -67,4 +67,9 @@ public Connection getConn(String dbURL, String userName, String password) { throw new RuntimeException("", e); } } + + @Override + public int getFetchSize(){ + return Integer.MIN_VALUE; + } } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java index 118ee048f..54ea47a18 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java @@ -54,12 +54,8 @@ public abstract class RdbAllReqRow extends AllReqRow { private static final Logger LOG = LoggerFactory.getLogger(RdbAllReqRow.class); - //private static final String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; - private static final int CONN_RETRY_NUM = 3; - private static final int FETCH_SIZE = 1000; - private AtomicReference>>> cacheRef = new AtomicReference<>(); public RdbAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { @@ -195,7 +191,7 @@ private void loadData(Map>> tmpCache) throws SQ //load data from table String sql = sideInfo.getSqlCondition(); Statement statement = connection.createStatement(); - statement.setFetchSize(FETCH_SIZE); + statement.setFetchSize(getFetchSize()); ResultSet resultSet = statement.executeQuery(sql); String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); while (resultSet.next()) { @@ -216,4 +212,9 @@ private void loadData(Map>> tmpCache) throws SQ } } } + + public int getFetchSize(){ + return 1000; + } + } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index 66945c513..61aba3e40 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -105,8 +105,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except String sqlCondition = sideInfo.getSqlCondition(); connection.queryWithParams(sqlCondition, inputParams, rs -> { if (rs.failed()) { - LOG.error("Cannot retrieve the data from the database"); - LOG.error("", rs.cause()); + LOG.error("Cannot retrieve the data from the database", rs.cause()); resultFuture.complete(null); return; } From d1da5e6dd34e8fbc4e3f376ca64b6802a92d2c13 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Tue, 27 Nov 2018 21:19:25 +0800 Subject: [PATCH 165/470] add metric --- .../flink/sql/sink/hbase/HbaseOutputFormat.java | 15 +++++++++++++++ .../flink/sql/sink/mongo/MongoOutputFormat.java | 15 +++++++++++++++ 2 files changed, 30 insertions(+) diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java index cc95d1e75..ea47b5689 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java @@ -20,10 +20,14 @@ package com.dtstack.flink.sql.sink.hbase; +import com.dtstack.flink.sql.metric.MetricConstant; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -62,6 +66,10 @@ public class HbaseOutputFormat extends RichOutputFormat { private transient Connection conn; private transient Table table; + private transient Counter outRecords; + + private transient Meter outRecordsRate; + public final SimpleDateFormat ROWKEY_DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss"); public final SimpleDateFormat FIELD_DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); @@ -82,6 +90,7 @@ public void open(int taskNumber, int numTasks) throws IOException { conn = ConnectionFactory.createConnection(conf); table = conn.getTable(TableName.valueOf(tableName)); LOG.warn("---open end(get table from hbase) ---"); + initMetric(); } @Override @@ -133,7 +142,13 @@ public void writeRecord(Tuple2 tuple2) throws IOException { } table.put(put); + outRecords.inc(); + + } + private void initMetric() { + outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); } @Override diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java index f86be6212..b32177aab 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.sink.mongo; +import com.dtstack.flink.sql.metric.MetricConstant; import com.mongodb.MongoClient; import com.mongodb.MongoClientOptions; import com.mongodb.MongoCredential; @@ -31,6 +32,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; import org.apache.flink.types.Row; import org.bson.Document; import org.bson.types.ObjectId; @@ -64,6 +68,10 @@ public class MongoOutputFormat extends RichOutputFormat { private static String PK = "_ID"; + private transient Counter outRecords; + + private transient Meter outRecordsRate; + public final SimpleDateFormat ROWKEY_DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss"); @Override @@ -73,6 +81,7 @@ public void configure(Configuration parameters) { @Override public void open(int taskNumber, int numTasks) throws IOException { establishConnection(); + initMetric(); } @Override @@ -107,6 +116,7 @@ public void writeRecord(Tuple2 tuple2) throws IOException { } else { dbCollection.insertOne(doc); } + outRecords.inc(); } @Override @@ -147,6 +157,11 @@ private void establishConnection() { } } + private void initMetric() { + outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); + } + private MongoOutputFormat() { } From afd053823bfc945d77928f7ef1df8b7e573ee22d Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 28 Nov 2018 09:36:23 +0800 Subject: [PATCH 166/470] abstract output metric --- .../flink/sql/metric/MetricOutputFormat.java | 24 +++++++++++++++++++ .../sql/sink/hbase/HbaseOutputFormat.java | 18 ++------------ .../sql/sink/mongo/MongoOutputFormat.java | 18 ++------------ .../sql/sink/rdb/RetractJDBCOutputFormat.java | 19 +++------------ .../sql/sink/redis/RedisOutputFormat.java | 18 ++------------ 5 files changed, 33 insertions(+), 64 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java diff --git a/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java b/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java new file mode 100644 index 000000000..e6b3d88b0 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java @@ -0,0 +1,24 @@ +package com.dtstack.flink.sql.metric; + +import org.apache.flink.api.java.tuple.Tuple2; + +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; + +/** + * Created by sishu.yss on 2018/11/28. + */ +public abstract class MetricOutputFormat extends RichOutputFormat{ + + protected transient Counter outRecords; + + protected transient Meter outRecordsRate; + + public void initMetric() { + outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); + } + +} diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java index ea47b5689..5dbeea040 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java @@ -20,14 +20,10 @@ package com.dtstack.flink.sql.sink.hbase; -import com.dtstack.flink.sql.metric.MetricConstant; +import com.dtstack.flink.sql.metric.MetricOutputFormat; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.MeterView; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -38,7 +34,6 @@ import org.apache.hadoop.hbase.client.Table; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.io.IOException; import java.text.SimpleDateFormat; import java.util.ArrayList; @@ -48,7 +43,7 @@ * author: jingzhen@dtstack.com * date: 2017-6-29 */ -public class HbaseOutputFormat extends RichOutputFormat { +public class HbaseOutputFormat extends MetricOutputFormat { private static final Logger LOG = LoggerFactory.getLogger(HbaseOutputFormat.class); @@ -66,10 +61,6 @@ public class HbaseOutputFormat extends RichOutputFormat { private transient Connection conn; private transient Table table; - private transient Counter outRecords; - - private transient Meter outRecordsRate; - public final SimpleDateFormat ROWKEY_DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss"); public final SimpleDateFormat FIELD_DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); @@ -146,11 +137,6 @@ public void writeRecord(Tuple2 tuple2) throws IOException { } - private void initMetric() { - outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); - outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); - } - @Override public void close() throws IOException { if(conn != null) { diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java index b32177aab..10651871c 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.sink.mongo; -import com.dtstack.flink.sql.metric.MetricConstant; +import com.dtstack.flink.sql.metric.MetricOutputFormat; import com.mongodb.MongoClient; import com.mongodb.MongoClientOptions; import com.mongodb.MongoCredential; @@ -28,19 +28,14 @@ import com.mongodb.client.MongoDatabase; import com.mongodb.client.result.UpdateResult; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.MeterView; import org.apache.flink.types.Row; import org.bson.Document; import org.bson.types.ObjectId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.io.IOException; import java.text.SimpleDateFormat; import java.util.ArrayList; @@ -52,7 +47,7 @@ * * @author xuqianjin */ -public class MongoOutputFormat extends RichOutputFormat { +public class MongoOutputFormat extends MetricOutputFormat { private static final Logger LOG = LoggerFactory.getLogger(MongoOutputFormat.class); private String address; @@ -68,10 +63,6 @@ public class MongoOutputFormat extends RichOutputFormat { private static String PK = "_ID"; - private transient Counter outRecords; - - private transient Meter outRecordsRate; - public final SimpleDateFormat ROWKEY_DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss"); @Override @@ -157,11 +148,6 @@ private void establishConnection() { } } - private void initMetric() { - outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); - outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); - } - private MongoOutputFormat() { } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java index a47b6472c..2635268ff 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java @@ -18,23 +18,18 @@ package com.dtstack.flink.sql.sink.rdb; -import com.dtstack.flink.sql.metric.MetricConstant; -import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.MeterView; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.io.IOException; import java.sql.Connection; import java.sql.DriverManager; import java.sql.PreparedStatement; import java.sql.SQLException; +import com.dtstack.flink.sql.metric.MetricOutputFormat; /** * OutputFormat to write tuples into a database. @@ -43,7 +38,7 @@ * @see Tuple * @see DriverManager */ -public class RetractJDBCOutputFormat extends RichOutputFormat { +public class RetractJDBCOutputFormat extends MetricOutputFormat { private static final long serialVersionUID = 1L; private static final Logger LOG = LoggerFactory.getLogger(RetractJDBCOutputFormat.class); @@ -63,11 +58,8 @@ public class RetractJDBCOutputFormat extends RichOutputFormat { public int[] typesArray; - private transient Counter outRecords; - - private transient Meter outRecordsRate; - public RetractJDBCOutputFormat() { + } @Override @@ -100,11 +92,6 @@ public void open(int taskNumber, int numTasks) throws IOException { } } - private void initMetric() { - outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); - outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); - } - private void establishConnection() throws SQLException, ClassNotFoundException { Class.forName(drivername); if (username == null) { diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java index f43114918..ca305f046 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java @@ -18,23 +18,18 @@ package com.dtstack.flink.sql.sink.redis; -import com.dtstack.flink.sql.metric.MetricConstant; +import com.dtstack.flink.sql.metric.MetricOutputFormat; import org.apache.commons.pool2.impl.GenericObjectPoolConfig; -import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Meter; -import org.apache.flink.metrics.MeterView; import org.apache.flink.types.Row; import redis.clients.jedis.*; - import java.io.Closeable; import java.io.IOException; import java.util.*; -public class RedisOutputFormat extends RichOutputFormat { +public class RedisOutputFormat extends MetricOutputFormat { private String url; @@ -70,10 +65,6 @@ public class RedisOutputFormat extends RichOutputFormat { private GenericObjectPoolConfig poolConfig; - private transient Counter outRecords; - - private transient Meter outRecordsRate; - private RedisOutputFormat(){ } @Override @@ -101,11 +92,6 @@ private GenericObjectPoolConfig setPoolConfig(String maxTotal, String maxIdle, S return config; } - private void initMetric(){ - outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); - outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); - } - private void establishConnection() { poolConfig = setPoolConfig(maxTotal, maxIdle, minIdle); String[] nodes = url.split(","); From 96a7a2e5b8ae9fe47265c71e19d1ea41681fa540 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 28 Nov 2018 09:37:00 +0800 Subject: [PATCH 167/470] commnet --- .../flink/sql/metric/MetricOutputFormat.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java b/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java index e6b3d88b0..bbf1152a3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java +++ b/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java @@ -1,3 +1,20 @@ +/* + * 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 com.dtstack.flink.sql.metric; import org.apache.flink.api.java.tuple.Tuple2; From cea884af28e9cab92fe221c39ba9ed759ce14cc1 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 28 Nov 2018 09:49:47 +0800 Subject: [PATCH 168/470] Update README.md --- README.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/README.md b/README.md index 3abfabbc3..b6a39720c 100644 --- a/README.md +++ b/README.md @@ -5,6 +5,8 @@ > > * 自定义create function 语法 > > * 实现了流与维表的join > > * 支持原生FLinkSQL所有的语法 +> > * 扩展了输入和输出的性能指标到promethus + # 已支持 * 源表:kafka 0.9,1.x版本 From d74a9637a827c6bfac4c91078eaaf7311b19e54b Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 28 Nov 2018 09:58:57 +0800 Subject: [PATCH 169/470] rename package --- .../dtstack/flink/sql/{metric => sink}/MetricOutputFormat.java | 3 ++- .../com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java | 2 +- .../com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java | 2 +- .../dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java | 2 +- .../com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java | 2 +- 5 files changed, 6 insertions(+), 5 deletions(-) rename core/src/main/java/com/dtstack/flink/sql/{metric => sink}/MetricOutputFormat.java (94%) diff --git a/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java b/core/src/main/java/com/dtstack/flink/sql/sink/MetricOutputFormat.java similarity index 94% rename from core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java rename to core/src/main/java/com/dtstack/flink/sql/sink/MetricOutputFormat.java index bbf1152a3..4e11280e7 100644 --- a/core/src/main/java/com/dtstack/flink/sql/metric/MetricOutputFormat.java +++ b/core/src/main/java/com/dtstack/flink/sql/sink/MetricOutputFormat.java @@ -15,8 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flink.sql.metric; +package com.dtstack.flink.sql.sink; +import com.dtstack.flink.sql.metric.MetricConstant; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.common.io.RichOutputFormat; diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java index 5dbeea040..2ef2c6d52 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.sink.hbase; -import com.dtstack.flink.sql.metric.MetricOutputFormat; +import com.dtstack.flink.sql.sink.MetricOutputFormat; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java index 10651871c..87ffc7354 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.sink.mongo; -import com.dtstack.flink.sql.metric.MetricOutputFormat; +import com.dtstack.flink.sql.sink.MetricOutputFormat; import com.mongodb.MongoClient; import com.mongodb.MongoClientOptions; import com.mongodb.MongoCredential; diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java index 2635268ff..21c2a537d 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java @@ -29,7 +29,7 @@ import java.sql.DriverManager; import java.sql.PreparedStatement; import java.sql.SQLException; -import com.dtstack.flink.sql.metric.MetricOutputFormat; +import com.dtstack.flink.sql.sink.MetricOutputFormat; /** * OutputFormat to write tuples into a database. diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java index ca305f046..910882c9f 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.sink.redis; -import com.dtstack.flink.sql.metric.MetricOutputFormat; +import com.dtstack.flink.sql.sink.MetricOutputFormat; import org.apache.commons.pool2.impl.GenericObjectPoolConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; From 10f129279206bdaed6fcb5bda8ca992c16e462a3 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Wed, 28 Nov 2018 16:58:02 +0800 Subject: [PATCH 170/470] fix bugs --- .../sql/side/redis/table/RedisSideParser.java | 20 +++++++++---------- .../side/redis/table/RedisSideTableInfo.java | 8 ++++---- .../sql/sink/redis/table/RedisSinkParser.java | 10 +++++----- .../sql/sink/redis/table/RedisTableInfo.java | 4 ++-- 4 files changed, 21 insertions(+), 21 deletions(-) diff --git a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java index fdeedfdd1..f8a83b868 100644 --- a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java +++ b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideParser.java @@ -33,20 +33,20 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Wed, 28 Nov 2018 22:19:03 +0800 Subject: [PATCH 171/470] add netty --- redis5/redis5-side/redis-async-side/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/redis5/redis5-side/redis-async-side/pom.xml b/redis5/redis5-side/redis-async-side/pom.xml index 1aaacd4dc..e0e573904 100644 --- a/redis5/redis5-side/redis-async-side/pom.xml +++ b/redis5/redis5-side/redis-async-side/pom.xml @@ -23,6 +23,11 @@ lettuce-core 5.0.5.RELEASE + + io.netty + netty-all + 4.1.24.Final + From a605c1cff280df21383c279194ce1c4b141c795f Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 29 Nov 2018 10:29:22 +0800 Subject: [PATCH 172/470] rename jarname --- elasticsearch5/elasticsearch5-sink/pom.xml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/elasticsearch5/elasticsearch5-sink/pom.xml b/elasticsearch5/elasticsearch5-sink/pom.xml index 68f464e48..a6453f6ce 100644 --- a/elasticsearch5/elasticsearch5-sink/pom.xml +++ b/elasticsearch5/elasticsearch5-sink/pom.xml @@ -10,7 +10,7 @@ 4.0.0 sql.sink.elasticsearch5 - elasticsearch5-sink + elasticsearch-sink @@ -48,7 +48,6 @@ org.apache.logging.log4j:log4j-to-slf4j - org.slf4j:slf4j-api From 957b8966e1270d2d1b0c8d3434d7243ac7e5b933 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 29 Nov 2018 14:59:59 +0800 Subject: [PATCH 173/470] add elasticsearch metric --- .../sink/elasticsearch/CustomerSinkFunc.java | 8 +++ .../sink/elasticsearch/ElasticsearchSink.java | 3 +- .../MetricElasticsearchSink.java | 58 +++++++++++++++++++ 3 files changed, 68 insertions(+), 1 deletion(-) create mode 100644 elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/MetricElasticsearchSink.java diff --git a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java index 7ad2720b4..a95d477de 100644 --- a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java +++ b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/CustomerSinkFunc.java @@ -23,6 +23,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.metrics.Counter; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; import org.apache.flink.types.Row; @@ -56,6 +57,8 @@ public class CustomerSinkFunc implements ElasticsearchSinkFunction { private List fieldTypes; + public transient Counter outRecords; + /** 默认分隔符为'_' */ private char sp = '_'; @@ -79,11 +82,16 @@ public void process(Tuple2 tuple2, RuntimeContext ctx, RequestIndexer indexer) { indexer.add(createIndexRequest(element)); + outRecords.inc(); }catch (Throwable e){ logger.error("", e); } } + public void setOutRecords(Counter outRecords) { + this.outRecords = outRecords; + } + private IndexRequest createIndexRequest(Row element) { List idFieldList = new ArrayList<>(); diff --git a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java index 1429d1d90..f372e4d1e 100644 --- a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java +++ b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/ElasticsearchSink.java @@ -27,6 +27,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.metrics.Counter; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; @@ -131,7 +132,7 @@ private RichSinkFunction createEsSinkFunction(){ CustomerSinkFunc customerSinkFunc = new CustomerSinkFunc(index, type, Arrays.asList(fieldNames), Arrays.asList(columnTypes), idIndexList); - return new org.apache.flink.streaming.connectors.elasticsearch5.ElasticsearchSink(userConfig, transports, customerSinkFunc); + return new MetricElasticsearchSink(userConfig, transports, customerSinkFunc); } @Override diff --git a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/MetricElasticsearchSink.java b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/MetricElasticsearchSink.java new file mode 100644 index 000000000..709150848 --- /dev/null +++ b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/MetricElasticsearchSink.java @@ -0,0 +1,58 @@ +/* + * 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 com.dtstack.flink.sql.sink.elasticsearch; + +import com.dtstack.flink.sql.metric.MetricConstant; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; + +import java.util.List; +import java.util.Map; + +/** + * @Auther: jiangjunjie + * @Date: 2018/11/29 14:15 + * @Description: + */ +public class MetricElasticsearchSink extends org.apache.flink.streaming.connectors.elasticsearch5.ElasticsearchSink { + + protected CustomerSinkFunc customerSinkFunc; + + protected transient Meter outRecordsRate; + + public MetricElasticsearchSink(Map userConfig, List transportAddresses, ElasticsearchSinkFunction elasticsearchSinkFunction) { + super(userConfig, transportAddresses, elasticsearchSinkFunction); + this.customerSinkFunc = (CustomerSinkFunc) elasticsearchSinkFunction; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + initMetric(); + } + + public void initMetric() { + Counter counter = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + customerSinkFunc.setOutRecords(counter); + outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(counter, 20)); + } +} From 54492e8be7b10a7b3a80f88557ba182953010016 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Thu, 29 Nov 2018 20:25:08 +0800 Subject: [PATCH 174/470] add sqlserver result table --- .../flink/sql/enums/EDatabaseType.java | 33 +++++ .../flink/sql/side/mysql/MysqlAllReqRow.java | 2 +- .../sql/side/mysql/MysqlAsyncReqRow.java | 7 - .../flink/sql/sink/mysql/MysqlSink.java | 7 + .../sql/side/rdb/async/RdbAsyncReqRow.java | 7 +- .../dtstack/flink/sql/sink/rdb/DBSink.java | 25 +++- .../dtstack/flink/sql/sink/rdb/RdbSink.java | 2 + .../sql/sink/rdb/RetractJDBCOutputFormat.java | 122 +++++++++++++++++- .../side/sqlserver/SqlserverAsyncReqRow.java | 6 - .../sql/sink/sqlserver/SqlserverSink.java | 117 ++++++++++++++++- 10 files changed, 305 insertions(+), 23 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/enums/EDatabaseType.java diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/EDatabaseType.java b/core/src/main/java/com/dtstack/flink/sql/enums/EDatabaseType.java new file mode 100644 index 000000000..9b01bf052 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/enums/EDatabaseType.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 com.dtstack.flink.sql.enums; + +/** + * Database type + * + * Company: www.dtstack.com + * @author jiangbo + */ +public enum EDatabaseType { + + MYSQL, + SQLSERVER, + ORACLE, + +} diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java index b56c3252b..aca7593f7 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java @@ -69,7 +69,7 @@ public Connection getConn(String dbURL, String userName, String password) { } @Override - public int getFetchSize(){ + public int getFetchSize() { return Integer.MIN_VALUE; } } diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java index 86fe8c8fa..456451214 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -49,13 +49,6 @@ public class MysqlAsyncReqRow extends RdbAsyncReqRow { private final static String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; - private final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 10; - - private final static int DEFAULT_VERTX_WORKER_POOL_SIZE = 20; - - private final static int DEFAULT_MAX_DB_CONN_POOL_SIZE = 20; - - public MysqlAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java index fe190f10f..b4ff9dea1 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.sink.rdb.RdbSink; import java.util.List; +import java.util.Map; /** * Date: 2017/2/27 @@ -43,6 +44,11 @@ public void buildSql(String tableName, List fields) { buildInsertSql(tableName, fields); } + @Override + public String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField) { + return null; + } + private void buildInsertSql(String tableName, List fields) { String sqlTmp = "replace into " + tableName + " (${fields}) values (${placeholder})"; String fieldsStr = ""; @@ -66,4 +72,5 @@ public String getDriverName() { return MYSQL_DRIVER; } + } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index 61aba3e40..a7e5339e9 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -52,13 +52,18 @@ public class RdbAsyncReqRow extends AsyncReqRow { private static final Logger LOG = LoggerFactory.getLogger(RdbAsyncReqRow.class); + public final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 1; + + public final static int DEFAULT_VERTX_WORKER_POOL_SIZE = Runtime.getRuntime().availableProcessors() * 2; + + public final static int DEFAULT_MAX_DB_CONN_POOL_SIZE = DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE + DEFAULT_VERTX_WORKER_POOL_SIZE; + private transient SQLClient rdbSQLClient; public RdbAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(new RdbAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } - @Override public void asyncInvoke(Row input, ResultFuture resultFuture) throws Exception { diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java index 35ecf926c..4f6372d54 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java @@ -21,6 +21,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; @@ -29,10 +30,12 @@ import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; +import java.io.Serializable; import java.math.BigDecimal; import java.sql.Timestamp; import java.sql.Types; import java.util.List; +import java.util.Map; /** * Reason: @@ -41,7 +44,7 @@ * * @author maqi */ -public abstract class DBSink implements RetractStreamTableSink { +public abstract class DBSink implements RetractStreamTableSink, Serializable { protected String driverName; @@ -51,6 +54,8 @@ public abstract class DBSink implements RetractStreamTableSink { protected String password; + protected String dbType; + protected int batchInterval = 1; protected int[] sqlTypes; @@ -67,6 +72,7 @@ public abstract class DBSink implements RetractStreamTableSink { private int parallelism = -1; + public RichSinkFunction createJdbcSinkFunc() { if (driverName == null || dbURL == null || userName == null @@ -84,6 +90,9 @@ public RichSinkFunction createJdbcSinkFunc() { jdbcFormatBuild.setBatchInterval(batchInterval); jdbcFormatBuild.setSqlTypes(sqlTypes); jdbcFormatBuild.setTableName(tableName); + jdbcFormatBuild.setDBType(dbType); + jdbcFormatBuild.setDBSink(this); + RetractJDBCOutputFormat outputFormat = jdbcFormatBuild.finish(); OutputFormatSinkFunction outputFormatSinkFunc = new OutputFormatSinkFunction(outputFormat); @@ -180,6 +189,10 @@ public void setParallelism(int parallelism) { this.parallelism = parallelism; } + public void setDbType(String dbType) { + this.dbType = dbType; + } + /** * you need to implements this method in your own class. * @@ -187,4 +200,14 @@ public void setParallelism(int parallelism) { * @param fields */ public abstract void buildSql(String tableName, List fields); + + /** + * sqlserver and oracle maybe implement + * + * @param tableName + * @param fieldNames + * @param realIndexes + * @return + */ + public abstract String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java index ae1630a38..586d4fb85 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java @@ -61,6 +61,8 @@ public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { this.password = tmpPassword; this.tableName = tmpTableName; this.primaryKeys = rdbTableInfo.getPrimaryKeys(); + this.dbType=rdbTableInfo.getType(); + buildSql(tableName, fields); buildSqlTypes(fieldTypeArray); return this; diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java index 21c2a537d..ee1d03abc 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java @@ -18,17 +18,20 @@ package com.dtstack.flink.sql.sink.rdb; +import com.dtstack.flink.sql.enums.EDatabaseType; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.sql.*; +import java.util.*; import java.io.IOException; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.PreparedStatement; -import java.sql.SQLException; + import com.dtstack.flink.sql.sink.MetricOutputFormat; /** @@ -49,15 +52,33 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private String dbURL; private String insertQuery; private String tableName; + private String dbType; private int batchInterval = 5000; private Connection dbConn; private PreparedStatement upload; + //index field + private Map> realIndexes = Maps.newHashMap(); + //full field + private List fullField = Lists.newArrayList(); + + private DBSink dbSink; private int batchCount = 0; public int[] typesArray; + private final static String GET_ORACLE_INDEX_SQL = "SELECT " + + "t.INDEX_NAME," + + "t.COLUMN_NAME " + + "FROM " + + "user_ind_columns t," + + "user_indexes i " + + "WHERE " + + "t.index_name = i.index_name " + + "AND i.uniqueness = 'UNIQUE' " + + "AND t.table_name = '%s'"; + public RetractJDBCOutputFormat() { } @@ -77,9 +98,11 @@ public void configure(Configuration parameters) { public void open(int taskNumber, int numTasks) throws IOException { try { establishConnection(); - upload = dbConn.prepareStatement(insertQuery); initMetric(); if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { + if (!EDatabaseType.MYSQL.name().equalsIgnoreCase(dbType) && isReplaceInsertQuery()) { + insertQuery = dbSink.buildUpdateSql(tableName, Arrays.asList(dbSink.fieldNames), realIndexes, fullField); + } upload = dbConn.prepareStatement(insertQuery); } else { throw new SQLException("Table " + tableName + " doesn't exist"); @@ -92,6 +115,22 @@ public void open(int taskNumber, int numTasks) throws IOException { } } + private boolean isReplaceInsertQuery() throws SQLException { + getRealIndexes(); + getFullColumns(); + + if (!realIndexes.isEmpty()) { + for (List value : realIndexes.values()) { + for (String fieldName : dbSink.getFieldNames()) { + if (value.contains(fieldName)) { + return true; + } + } + } + } + return false; + } + private void establishConnection() throws SQLException, ClassNotFoundException { Class.forName(drivername); if (username == null) { @@ -149,7 +188,6 @@ private void insertWrite(Row row) throws SQLException { } } - private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLException { if (typesArray == null) { // no types provided @@ -269,6 +307,67 @@ public void close() throws IOException { } } + /** + * get db all index + * + * @throws SQLException + */ + public void getRealIndexes() throws SQLException { + Map> map = Maps.newHashMap(); + ResultSet rs; + if (EDatabaseType.ORACLE.name().equalsIgnoreCase(dbType)) { + PreparedStatement ps = dbConn.prepareStatement(String.format(GET_ORACLE_INDEX_SQL, tableName)); + rs = ps.executeQuery(); + } else { + rs = dbConn.getMetaData().getIndexInfo(null, null, tableName, true, false); + } + + while (rs.next()) { + String indexName = rs.getString("INDEX_NAME"); + if (!map.containsKey(indexName)) { + map.put(indexName, new ArrayList<>()); + } + String column_name = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(column_name)) { + column_name = column_name.toUpperCase(); + } + map.get(indexName).add(column_name); + } + + for (Map.Entry> entry : map.entrySet()) { + String k = entry.getKey(); + List v = entry.getValue(); + if (v != null && v.size() != 0 && v.get(0) != null) { + realIndexes.put(k, v); + } + } + } + + /** + * get db all column name + * + * @throws SQLException + */ + public void getFullColumns() throws SQLException { + String schema = null; + if (EDatabaseType.ORACLE.name().equalsIgnoreCase(dbType)) { + String[] parts = tableName.split("\\."); + if (parts.length == 2) { + schema = parts[0].toUpperCase(); + tableName = parts[1]; + } + } + + ResultSet rs = dbConn.getMetaData().getColumns(null, schema, tableName, null); + while (rs.next()) { + String columnName = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(columnName)) { + fullField.add(columnName.toUpperCase()); + } + } + + } + public static JDBCOutputFormatBuilder buildJDBCOutputFormat() { return new JDBCOutputFormatBuilder(); } @@ -321,6 +420,17 @@ public JDBCOutputFormatBuilder setTableName(String tableName) { return this; } + public JDBCOutputFormatBuilder setDBSink(DBSink dbSink) { + format.dbSink = dbSink; + return this; + } + + public JDBCOutputFormatBuilder setDBType(String dbType) { + format.dbType = dbType; + return this; + } + + /** * Finalizes the configuration and checks validity. * diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java index 4f39129ca..003d61b27 100644 --- a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java @@ -40,12 +40,6 @@ public class SqlserverAsyncReqRow extends RdbAsyncReqRow { private final static String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; - private final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 10; - - private final static int DEFAULT_VERTX_WORKER_POOL_SIZE = 20; - - private final static int DEFAULT_MAX_DB_CONN_POOL_SIZE = 20; - public SqlserverAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java index 986486445..164413216 100644 --- a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java @@ -18,8 +18,9 @@ package com.dtstack.flink.sql.sink.sqlserver; import com.dtstack.flink.sql.sink.rdb.RdbSink; +import org.apache.commons.lang3.StringUtils; -import java.util.List; +import java.util.*; /** * Reason: @@ -42,6 +43,120 @@ public void buildSql(String tableName, List fields) { } private void buildInsertSql(String tableName, List fields) { + String sqlTmp = "insert into " + tableName + " (${fields}) values (${placeholder})"; + String fieldsStr = StringUtils.join(fields, ","); + String placeholder = ""; + for (String fieldName : fields) { + placeholder += ",?"; + } + placeholder = placeholder.replaceFirst(",", ""); + sqlTmp = sqlTmp.replace("${fields}", fieldsStr).replace("${placeholder}", placeholder); + this.sql = sqlTmp; } + + @Override + public String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField) { + return "MERGE INTO " + tableName + " T1 USING " + + "(" + makeValues(fieldNames) + ") T2 ON (" + + updateKeySql(realIndexes) + ") WHEN MATCHED THEN UPDATE SET " + + getUpdateSql(fieldNames, fullField, "T1", "T2", keyColList(realIndexes)) + " WHEN NOT MATCHED THEN " + + "INSERT (" + quoteColumns(fieldNames) + ") VALUES (" + + quoteColumns(fieldNames, "T2") + ");"; + } + + + public String quoteColumns(List column) { + return quoteColumns(column, null); + } + + public String quoteColumns(List column, String table) { + String prefix = StringUtils.isBlank(table) ? "" : quoteTable(table) + "."; + List list = new ArrayList<>(); + for (String col : column) { + list.add(prefix + quoteColumn(col)); + } + return StringUtils.join(list, ","); + } + + protected List keyColList(Map> updateKey) { + List keyCols = new ArrayList<>(); + for (Map.Entry> entry : updateKey.entrySet()) { + List list = entry.getValue(); + for (String col : list) { + if (!keyCols.contains(col)) { + keyCols.add(col); + } + } + } + return keyCols; + } + + public String getUpdateSql(List column, List fullColumn, String leftTable, String rightTable, List keyCols) { + String prefixLeft = StringUtils.isBlank(leftTable) ? "" : quoteTable(leftTable) + "."; + String prefixRight = StringUtils.isBlank(rightTable) ? "" : quoteTable(rightTable) + "."; + List list = new ArrayList<>(); + for (String col : fullColumn) { + if (keyCols == null || keyCols.size() == 0 ) { + continue; + } + if (fullColumn == null || column.contains(col)) { + list.add(prefixLeft + col + "=" + prefixRight + col); + } else { + list.add(prefixLeft + col + "=null"); + } + } + return StringUtils.join(list, ","); + } + + public String quoteTable(String table) { + String[] parts = table.split("\\."); + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < parts.length; ++i) { + if (i != 0) { + sb.append("."); + } + sb.append(getStartQuote() + parts[i] + getEndQuote()); + } + return sb.toString(); + } + + + public String updateKeySql(Map> updateKey) { + List exprList = new ArrayList<>(); + for (Map.Entry> entry : updateKey.entrySet()) { + List colList = new ArrayList<>(); + for (String col : entry.getValue()) { + colList.add("T1." + quoteColumn(col) + "=T2." + quoteColumn(col)); + } + exprList.add(StringUtils.join(colList, " AND ")); + } + return StringUtils.join(exprList, " OR "); + } + + + public String makeValues(List column) { + StringBuilder sb = new StringBuilder("SELECT "); + for (int i = 0; i < column.size(); ++i) { + if (i != 0) { + sb.append(","); + } + sb.append("? " + quoteColumn(column.get(i))); + } + return sb.toString(); + } + + public String quoteColumn(String column) { + return getStartQuote() + column + getEndQuote(); + } + + public String getStartQuote() { + return "\""; + } + + public String getEndQuote() { + return "\""; + } + + } From 18fc2afe3bd8cb5df08bbdb181d27e5ba8382573 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Fri, 30 Nov 2018 17:57:21 +0800 Subject: [PATCH 175/470] extract outputformat --- .../flink/sql/sink/mysql/MysqlSink.java | 3 +- .../dtstack/flink/sql/sink/rdb/DBSink.java | 213 ---------------- .../dtstack/flink/sql/sink/rdb/RdbSink.java | 199 ++++++++++++++- .../sink/rdb/format/OracleOutputFormat.java | 115 +++++++++ .../{ => format}/RetractJDBCOutputFormat.java | 236 ++++++------------ .../rdb/format/SqlserverOutputFormat.java | 102 ++++++++ .../sql/sink/sqlserver/SqlserverSink.java | 5 +- 7 files changed, 492 insertions(+), 381 deletions(-) delete mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java create mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/OracleOutputFormat.java rename rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/{ => format}/RetractJDBCOutputFormat.java (66%) create mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/SqlserverOutputFormat.java diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java index b4ff9dea1..007c9a262 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.sink.mysql; +import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.rdb.RdbSink; import java.util.List; @@ -32,7 +33,7 @@ * @author xuchao */ -public class MysqlSink extends RdbSink { +public class MysqlSink extends RdbSink implements IStreamSinkGener { private static final String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java deleted file mode 100644 index 4f6372d54..000000000 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/DBSink.java +++ /dev/null @@ -1,213 +0,0 @@ -/** - * 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 com.dtstack.flink.sql.sink.rdb; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; -import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; -import org.apache.flink.table.sinks.RetractStreamTableSink; -import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.types.Row; - -import java.io.Serializable; -import java.math.BigDecimal; -import java.sql.Timestamp; -import java.sql.Types; -import java.util.List; -import java.util.Map; - -/** - * Reason: - * Date: 2018/11/27 - * Company: www.dtstack.com - * - * @author maqi - */ -public abstract class DBSink implements RetractStreamTableSink, Serializable { - - protected String driverName; - - protected String dbURL; - - protected String userName; - - protected String password; - - protected String dbType; - - protected int batchInterval = 1; - - protected int[] sqlTypes; - - protected String tableName; - - protected String sql; - - protected List primaryKeys; - - protected String[] fieldNames; - - private TypeInformation[] fieldTypes; - - private int parallelism = -1; - - - public RichSinkFunction createJdbcSinkFunc() { - - if (driverName == null || dbURL == null || userName == null - || password == null || sqlTypes == null || tableName == null) { - throw new RuntimeException("any of params in(driverName, dbURL, userName, password, type, tableName) " + - " must not be null. please check it!!!"); - } - - RetractJDBCOutputFormat.JDBCOutputFormatBuilder jdbcFormatBuild = RetractJDBCOutputFormat.buildJDBCOutputFormat(); - jdbcFormatBuild.setDBUrl(dbURL); - jdbcFormatBuild.setDrivername(driverName); - jdbcFormatBuild.setUsername(userName); - jdbcFormatBuild.setPassword(password); - jdbcFormatBuild.setInsertQuery(sql); - jdbcFormatBuild.setBatchInterval(batchInterval); - jdbcFormatBuild.setSqlTypes(sqlTypes); - jdbcFormatBuild.setTableName(tableName); - jdbcFormatBuild.setDBType(dbType); - jdbcFormatBuild.setDBSink(this); - - RetractJDBCOutputFormat outputFormat = jdbcFormatBuild.finish(); - - OutputFormatSinkFunction outputFormatSinkFunc = new OutputFormatSinkFunction(outputFormat); - return outputFormatSinkFunc; - } - - /** - * By now specified class type conversion. - * FIXME Follow-up has added a new type of time needs to be modified - * - * @param fieldTypeArray - */ - protected void buildSqlTypes(List fieldTypeArray) { - - int[] tmpFieldsType = new int[fieldTypeArray.size()]; - for (int i = 0; i < fieldTypeArray.size(); i++) { - String fieldType = fieldTypeArray.get(i).getName(); - if (fieldType.equals(Integer.class.getName())) { - tmpFieldsType[i] = Types.INTEGER; - } else if (fieldType.equals(Long.class.getName())) { - tmpFieldsType[i] = Types.BIGINT; - } else if (fieldType.equals(Byte.class.getName())) { - tmpFieldsType[i] = Types.TINYINT; - } else if (fieldType.equals(Short.class.getName())) { - tmpFieldsType[i] = Types.SMALLINT; - } else if (fieldType.equals(String.class.getName())) { - tmpFieldsType[i] = Types.CHAR; - } else if (fieldType.equals(Byte.class.getName())) { - tmpFieldsType[i] = Types.BINARY; - } else if (fieldType.equals(Float.class.getName())) { - tmpFieldsType[i] = Types.FLOAT; - } else if (fieldType.equals(Double.class.getName())) { - tmpFieldsType[i] = Types.DOUBLE; - } else if (fieldType.equals(Timestamp.class.getName())) { - tmpFieldsType[i] = Types.TIMESTAMP; - } else if (fieldType.equals(BigDecimal.class.getName())) { - tmpFieldsType[i] = Types.DECIMAL; - } else { - throw new RuntimeException("no support field type for sql. the input type:" + fieldType); - } - } - - this.sqlTypes = tmpFieldsType; - } - - /** - * Set the default frequency submit updated every submission - * - * @param batchInterval - */ - public void setBatchInterval(int batchInterval) { - this.batchInterval = batchInterval; - } - - @Override - public void emitDataStream(DataStream> dataStream) { - RichSinkFunction richSinkFunction = createJdbcSinkFunc(); - DataStreamSink streamSink = dataStream.addSink(richSinkFunction); - streamSink.name(tableName); - if (parallelism > 0) { - streamSink.setParallelism(parallelism); - } - } - - @Override - public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { - this.fieldNames = fieldNames; - this.fieldTypes = fieldTypes; - return this; - } - - @Override - public TupleTypeInfo> getOutputType() { - return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), getRecordType()); - } - - @Override - public TypeInformation getRecordType() { - return new RowTypeInfo(fieldTypes, fieldNames); - } - - @Override - public String[] getFieldNames() { - return fieldNames; - } - - @Override - public TypeInformation[] getFieldTypes() { - return fieldTypes; - } - - - public void setParallelism(int parallelism) { - this.parallelism = parallelism; - } - - public void setDbType(String dbType) { - this.dbType = dbType; - } - - /** - * you need to implements this method in your own class. - * - * @param tableName - * @param fields - */ - public abstract void buildSql(String tableName, List fields); - - /** - * sqlserver and oracle maybe implement - * - * @param tableName - * @param fieldNames - * @param realIndexes - * @return - */ - public abstract String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField); -} diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java index 586d4fb85..b4edd38c7 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java @@ -17,12 +17,32 @@ */ package com.dtstack.flink.sql.sink.rdb; +import com.dtstack.flink.sql.enums.EDatabaseType; import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.rdb.format.OracleOutputFormat; +import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; +import com.dtstack.flink.sql.sink.rdb.format.SqlserverOutputFormat; import com.dtstack.flink.sql.sink.rdb.table.RdbTableInfo; import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.table.sinks.RetractStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; +import java.io.Serializable; +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.sql.Types; import java.util.Arrays; import java.util.List; +import java.util.Map; /** * Reason: @@ -31,7 +51,67 @@ * * @author maqi */ -public abstract class RdbSink extends DBSink implements IStreamSinkGener { +public abstract class RdbSink implements RetractStreamTableSink, Serializable, IStreamSinkGener { + + protected String driverName; + + protected String dbURL; + + protected String userName; + + protected String password; + + protected String dbType; + + protected int batchInterval = 1; + + protected int[] sqlTypes; + + protected String tableName; + + protected String sql; + + protected List primaryKeys; + + protected String[] fieldNames; + + private TypeInformation[] fieldTypes; + + private int parallelism = -1; + + public RichSinkFunction createJdbcSinkFunc() { + if (driverName == null || dbURL == null || userName == null + || password == null || sqlTypes == null || tableName == null) { + throw new RuntimeException("any of params in(driverName, dbURL, userName, password, type, tableName) " + + " must not be null. please check it!!!"); + } + RetractJDBCOutputFormat outputFormat = getOutputFormat(dbType); + outputFormat.setDbURL(dbURL); + outputFormat.setDrivername(driverName); + outputFormat.setUsername(userName); + outputFormat.setPassword(password); + outputFormat.setInsertQuery(sql); + outputFormat.setBatchInterval(batchInterval); + outputFormat.setTypesArray(sqlTypes); + outputFormat.setTableName(tableName); + outputFormat.setDbType(dbType); + outputFormat.setDbSink(this); + + outputFormat.verifyField(); + OutputFormatSinkFunction outputFormatSinkFunc = new OutputFormatSinkFunction(outputFormat); + return outputFormatSinkFunc; + } + + protected RetractJDBCOutputFormat getOutputFormat(String dbType) { + if (dbType.equalsIgnoreCase(EDatabaseType.SQLSERVER.name())) { + return new SqlserverOutputFormat(); + } else if (dbType.equalsIgnoreCase(EDatabaseType.ORACLE.name())) { + return new OracleOutputFormat(); + } else { + return new RetractJDBCOutputFormat(); + } + } + @Override public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { @@ -61,13 +141,126 @@ public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { this.password = tmpPassword; this.tableName = tmpTableName; this.primaryKeys = rdbTableInfo.getPrimaryKeys(); - this.dbType=rdbTableInfo.getType(); + this.dbType = rdbTableInfo.getType(); buildSql(tableName, fields); buildSqlTypes(fieldTypeArray); return this; } - public abstract String getDriverName(); + /** + * By now specified class type conversion. + * FIXME Follow-up has added a new type of time needs to be modified + * + * @param fieldTypeArray + */ + protected void buildSqlTypes(List fieldTypeArray) { + + int[] tmpFieldsType = new int[fieldTypeArray.size()]; + for (int i = 0; i < fieldTypeArray.size(); i++) { + String fieldType = fieldTypeArray.get(i).getName(); + if (fieldType.equals(Integer.class.getName())) { + tmpFieldsType[i] = Types.INTEGER; + } else if (fieldType.equals(Long.class.getName())) { + tmpFieldsType[i] = Types.BIGINT; + } else if (fieldType.equals(Byte.class.getName())) { + tmpFieldsType[i] = Types.TINYINT; + } else if (fieldType.equals(Short.class.getName())) { + tmpFieldsType[i] = Types.SMALLINT; + } else if (fieldType.equals(String.class.getName())) { + tmpFieldsType[i] = Types.CHAR; + } else if (fieldType.equals(Byte.class.getName())) { + tmpFieldsType[i] = Types.BINARY; + } else if (fieldType.equals(Float.class.getName())) { + tmpFieldsType[i] = Types.FLOAT; + } else if (fieldType.equals(Double.class.getName())) { + tmpFieldsType[i] = Types.DOUBLE; + } else if (fieldType.equals(Timestamp.class.getName())) { + tmpFieldsType[i] = Types.TIMESTAMP; + } else if (fieldType.equals(BigDecimal.class.getName())) { + tmpFieldsType[i] = Types.DECIMAL; + } else { + throw new RuntimeException("no support field type for sql. the input type:" + fieldType); + } + } + + this.sqlTypes = tmpFieldsType; + } + + + @Override + public void emitDataStream(DataStream> dataStream) { + RichSinkFunction richSinkFunction = createJdbcSinkFunc(); + DataStreamSink streamSink = dataStream.addSink(richSinkFunction); + streamSink.name(tableName); + if (parallelism > 0) { + streamSink.setParallelism(parallelism); + } + } + + @Override + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + + /** + * Set the default frequency submit updated every submission + * + * @param batchInterval + */ + public void setBatchInterval(int batchInterval) { + this.batchInterval = batchInterval; + } + + @Override + public TupleTypeInfo> getOutputType() { + return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), getRecordType()); + } + + @Override + public TypeInformation getRecordType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + + public void setParallelism(int parallelism) { + this.parallelism = parallelism; + } + + public void setDbType(String dbType) { + this.dbType = dbType; + } + + /** + * you need to implements this method in your own class. + * + * @param tableName + * @param fields + */ + public abstract void buildSql(String tableName, List fields); + + /** + * sqlserver and oracle maybe implement + * + * @param tableName + * @param fieldNames + * @param realIndexes + * @return + */ + public abstract String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField); + + public abstract String getDriverName(); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/OracleOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/OracleOutputFormat.java new file mode 100644 index 000000000..a9d635bca --- /dev/null +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/OracleOutputFormat.java @@ -0,0 +1,115 @@ +/** + * 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 com.dtstack.flink.sql.sink.rdb.format; + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/30 + * Company: www.dtstack.com + * + * @author maqi + */ +public class OracleOutputFormat extends RetractJDBCOutputFormat { + + private final static String GET_ORACLE_INDEX_SQL = "SELECT " + + "t.INDEX_NAME," + + "t.COLUMN_NAME " + + "FROM " + + "user_ind_columns t," + + "user_indexes i " + + "WHERE " + + "t.index_name = i.index_name " + + "AND i.uniqueness = 'UNIQUE' " + + "AND t.table_name = '%s'"; + + + @Override + public boolean isReplaceInsertQuery() throws SQLException { + fillRealIndexes(); + fillFullColumns(); + + if (!getRealIndexes().isEmpty()) { + for (List value : getRealIndexes().values()) { + for (String fieldName : getDbSink().getFieldNames()) { + if (value.contains(fieldName)) { + return true; + } + } + } + } + return false; + } + + /** + * get db all index + * + * @throws SQLException + */ + public void fillRealIndexes() throws SQLException { + Map> map = Maps.newHashMap(); + + PreparedStatement ps = getDbConn().prepareStatement(String.format(GET_ORACLE_INDEX_SQL, getTableName())); + ResultSet rs = ps.executeQuery(); + + while (rs.next()) { + String indexName = rs.getString("INDEX_NAME"); + if (!map.containsKey(indexName)) { + map.put(indexName, new ArrayList<>()); + } + String column_name = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(column_name)) { + column_name = column_name.toUpperCase(); + } + map.get(indexName).add(column_name); + } + + for (Map.Entry> entry : map.entrySet()) { + String k = entry.getKey(); + List v = entry.getValue(); + if (v != null && v.size() != 0 && v.get(0) != null) { + getRealIndexes().put(k, v); + } + } + } + + /** + * get db all column name + * + * @throws SQLException + */ + public void fillFullColumns() throws SQLException { + ResultSet rs = getDbConn().getMetaData().getColumns(null, null, getTableName(), null); + while (rs.next()) { + String columnName = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(columnName)) { + getFullField().add(columnName.toUpperCase()); + } + } + } + + +} diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java similarity index 66% rename from rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java rename to rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index ee1d03abc..8f5329796 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -16,9 +16,9 @@ * limitations under the License. */ -package com.dtstack.flink.sql.sink.rdb; +package com.dtstack.flink.sql.sink.rdb.format; -import com.dtstack.flink.sql.enums.EDatabaseType; +import com.dtstack.flink.sql.sink.rdb.RdbSink; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; @@ -28,6 +28,7 @@ import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import java.sql.*; import java.util.*; import java.io.IOException; @@ -50,35 +51,23 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private String password; private String drivername; private String dbURL; - private String insertQuery; private String tableName; private String dbType; + private RdbSink dbSink; private int batchInterval = 5000; + private String insertQuery; + public int[] typesArray; private Connection dbConn; private PreparedStatement upload; + + private int batchCount = 0; + //index field private Map> realIndexes = Maps.newHashMap(); //full field private List fullField = Lists.newArrayList(); - private DBSink dbSink; - - private int batchCount = 0; - - public int[] typesArray; - - private final static String GET_ORACLE_INDEX_SQL = "SELECT " + - "t.INDEX_NAME," + - "t.COLUMN_NAME " + - "FROM " + - "user_ind_columns t," + - "user_indexes i " + - "WHERE " + - "t.index_name = i.index_name " + - "AND i.uniqueness = 'UNIQUE' " + - "AND t.table_name = '%s'"; - public RetractJDBCOutputFormat() { } @@ -100,8 +89,8 @@ public void open(int taskNumber, int numTasks) throws IOException { establishConnection(); initMetric(); if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { - if (!EDatabaseType.MYSQL.name().equalsIgnoreCase(dbType) && isReplaceInsertQuery()) { - insertQuery = dbSink.buildUpdateSql(tableName, Arrays.asList(dbSink.fieldNames), realIndexes, fullField); + if (isReplaceInsertQuery()) { + insertQuery = dbSink.buildUpdateSql(tableName, Arrays.asList(dbSink.getFieldNames()), realIndexes, fullField); } upload = dbConn.prepareStatement(insertQuery); } else { @@ -115,21 +104,6 @@ public void open(int taskNumber, int numTasks) throws IOException { } } - private boolean isReplaceInsertQuery() throws SQLException { - getRealIndexes(); - getFullColumns(); - - if (!realIndexes.isEmpty()) { - for (List value : realIndexes.values()) { - for (String fieldName : dbSink.getFieldNames()) { - if (value.contains(fieldName)) { - return true; - } - } - } - } - return false; - } private void establishConnection() throws SQLException, ClassNotFoundException { Class.forName(drivername); @@ -307,153 +281,91 @@ public void close() throws IOException { } } - /** - * get db all index - * - * @throws SQLException - */ - public void getRealIndexes() throws SQLException { - Map> map = Maps.newHashMap(); - ResultSet rs; - if (EDatabaseType.ORACLE.name().equalsIgnoreCase(dbType)) { - PreparedStatement ps = dbConn.prepareStatement(String.format(GET_ORACLE_INDEX_SQL, tableName)); - rs = ps.executeQuery(); - } else { - rs = dbConn.getMetaData().getIndexInfo(null, null, tableName, true, false); - } - - while (rs.next()) { - String indexName = rs.getString("INDEX_NAME"); - if (!map.containsKey(indexName)) { - map.put(indexName, new ArrayList<>()); - } - String column_name = rs.getString("COLUMN_NAME"); - if (StringUtils.isNotBlank(column_name)) { - column_name = column_name.toUpperCase(); - } - map.get(indexName).add(column_name); - } - for (Map.Entry> entry : map.entrySet()) { - String k = entry.getKey(); - List v = entry.getValue(); - if (v != null && v.size() != 0 && v.get(0) != null) { - realIndexes.put(k, v); - } - } + public boolean isReplaceInsertQuery() throws SQLException { + return false; } - /** - * get db all column name - * - * @throws SQLException - */ - public void getFullColumns() throws SQLException { - String schema = null; - if (EDatabaseType.ORACLE.name().equalsIgnoreCase(dbType)) { - String[] parts = tableName.split("\\."); - if (parts.length == 2) { - schema = parts[0].toUpperCase(); - tableName = parts[1]; - } + public void verifyField() { + if (StringUtils.isBlank(username)) { + LOG.info("Username was not supplied separately."); } - - ResultSet rs = dbConn.getMetaData().getColumns(null, schema, tableName, null); - while (rs.next()) { - String columnName = rs.getString("COLUMN_NAME"); - if (StringUtils.isNotBlank(columnName)) { - fullField.add(columnName.toUpperCase()); - } + if (StringUtils.isBlank(password)) { + LOG.info("Password was not supplied separately."); + } + if (StringUtils.isBlank(dbURL)) { + throw new IllegalArgumentException("No dababase URL supplied."); + } + if (StringUtils.isBlank(insertQuery)) { + throw new IllegalArgumentException("No insertQuery suplied"); + } + if (StringUtils.isBlank(drivername)) { + throw new IllegalArgumentException("No driver supplied"); } - } - public static JDBCOutputFormatBuilder buildJDBCOutputFormat() { - return new JDBCOutputFormatBuilder(); - } - public static class JDBCOutputFormatBuilder { - private final RetractJDBCOutputFormat format; + public void setUsername(String username) { + this.username = username; + } - protected JDBCOutputFormatBuilder() { - this.format = new RetractJDBCOutputFormat(); - } + public void setPassword(String password) { + this.password = password; + } - public JDBCOutputFormatBuilder setUsername(String username) { - format.username = username; - return this; - } + public void setDrivername(String drivername) { + this.drivername = drivername; + } - public JDBCOutputFormatBuilder setPassword(String password) { - format.password = password; - return this; - } + public void setDbURL(String dbURL) { + this.dbURL = dbURL; + } - public JDBCOutputFormatBuilder setDrivername(String drivername) { - format.drivername = drivername; - return this; - } + public void setTableName(String tableName) { + this.tableName = tableName; + } - public JDBCOutputFormatBuilder setDBUrl(String dbURL) { - format.dbURL = dbURL; - return this; - } + public void setDbType(String dbType) { + this.dbType = dbType; + } - public JDBCOutputFormatBuilder setInsertQuery(String query) { - format.insertQuery = query; - return this; - } + public void setDbSink(RdbSink dbSink) { + this.dbSink = dbSink; + } + public void setBatchInterval(int batchInterval) { + this.batchInterval = batchInterval; + } - public JDBCOutputFormatBuilder setBatchInterval(int batchInterval) { - format.batchInterval = batchInterval; - return this; - } + public void setInsertQuery(String insertQuery) { + this.insertQuery = insertQuery; + } - public JDBCOutputFormatBuilder setSqlTypes(int[] typesArray) { - format.typesArray = typesArray; - return this; - } + public void setTypesArray(int[] typesArray) { + this.typesArray = typesArray; + } - public JDBCOutputFormatBuilder setTableName(String tableName) { - format.tableName = tableName; - return this; - } + public String getDbType() { + return dbType; + } - public JDBCOutputFormatBuilder setDBSink(DBSink dbSink) { - format.dbSink = dbSink; - return this; - } + public RdbSink getDbSink() { + return dbSink; + } - public JDBCOutputFormatBuilder setDBType(String dbType) { - format.dbType = dbType; - return this; - } + public Connection getDbConn() { + return dbConn; + } + public String getTableName() { + return tableName; + } - /** - * Finalizes the configuration and checks validity. - * - * @return Configured RetractJDBCOutputFormat - */ - public RetractJDBCOutputFormat finish() { - if (format.username == null) { - LOG.info("Username was not supplied separately."); - } - if (format.password == null) { - LOG.info("Password was not supplied separately."); - } - if (format.dbURL == null) { - throw new IllegalArgumentException("No dababase URL supplied."); - } - if (format.insertQuery == null) { - throw new IllegalArgumentException("No insertQuery suplied"); - } - if (format.drivername == null) { - throw new IllegalArgumentException("No driver supplied"); - } - return format; - } + public Map> getRealIndexes() { + return realIndexes; } + public List getFullField() { + return fullField; + } } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/SqlserverOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/SqlserverOutputFormat.java new file mode 100644 index 000000000..612bf7f17 --- /dev/null +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/SqlserverOutputFormat.java @@ -0,0 +1,102 @@ +/** + * 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 com.dtstack.flink.sql.sink.rdb.format; + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/30 + * Company: www.dtstack.com + * + * @author maqi + */ +public class SqlserverOutputFormat extends RetractJDBCOutputFormat { + + + @Override + public boolean isReplaceInsertQuery() throws SQLException { + fillRealIndexes(); + fillFullColumns(); + + if (!getRealIndexes().isEmpty()) { + for (List value : getRealIndexes().values()) { + for (String fieldName : getDbSink().getFieldNames()) { + if (value.contains(fieldName)) { + return true; + } + } + } + } + return false; + } + + /** + * get db all index + * + * @throws SQLException + */ + public void fillRealIndexes() throws SQLException { + Map> map = Maps.newHashMap(); + ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, null, getTableName(), true, false); + + while (rs.next()) { + String indexName = rs.getString("INDEX_NAME"); + if (!map.containsKey(indexName)) { + map.put(indexName, new ArrayList<>()); + } + String column_name = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(column_name)) { + column_name = column_name.toUpperCase(); + } + map.get(indexName).add(column_name); + } + + for (Map.Entry> entry : map.entrySet()) { + String k = entry.getKey(); + List v = entry.getValue(); + if (v != null && v.size() != 0 && v.get(0) != null) { + getRealIndexes().put(k, v); + } + } + } + + /** + * get db all column name + * + * @throws SQLException + */ + public void fillFullColumns() throws SQLException { + String schema = null; + ResultSet rs = getDbConn().getMetaData().getColumns(null, schema, getTableName(), null); + while (rs.next()) { + String columnName = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(columnName)) { + getFullField().add(columnName.toUpperCase()); + } + } + } + + +} diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java index 164413216..dbd75635e 100644 --- a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java @@ -17,6 +17,7 @@ */ package com.dtstack.flink.sql.sink.sqlserver; +import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.rdb.RdbSink; import org.apache.commons.lang3.StringUtils; @@ -29,7 +30,7 @@ * * @author maqi */ -public class SqlserverSink extends RdbSink { +public class SqlserverSink extends RdbSink implements IStreamSinkGener { private static final String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; @Override @@ -97,7 +98,7 @@ public String getUpdateSql(List column, List fullColumn, String String prefixRight = StringUtils.isBlank(rightTable) ? "" : quoteTable(rightTable) + "."; List list = new ArrayList<>(); for (String col : fullColumn) { - if (keyCols == null || keyCols.size() == 0 ) { + if (keyCols == null || keyCols.size() == 0) { continue; } if (fullColumn == null || column.contains(col)) { From 59801ed5c9e6d7844845c462348fdaf3ff81d45d Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Sat, 1 Dec 2018 12:17:29 +0800 Subject: [PATCH 176/470] modify structure --- .../flink/sql/sink/mysql/MysqlSink.java | 6 + .../dtstack/flink/sql/sink/rdb/RdbSink.java | 15 +-- .../sink/rdb/format/OracleOutputFormat.java | 115 ------------------ .../sqlserver}/SqlserverOutputFormat.java | 12 +- .../sql/sink/sqlserver/SqlserverSink.java | 6 + 5 files changed, 21 insertions(+), 133 deletions(-) delete mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/OracleOutputFormat.java rename {rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format => sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver}/SqlserverOutputFormat.java (94%) diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java index 007c9a262..6d4d86d23 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java @@ -22,6 +22,7 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; import java.util.List; import java.util.Map; @@ -40,6 +41,11 @@ public class MysqlSink extends RdbSink implements IStreamSinkGener { public MysqlSink() { } + @Override + public RetractJDBCOutputFormat getOutputFormat() { + return new RetractJDBCOutputFormat(); + } + @Override public void buildSql(String tableName, List fields) { buildInsertSql(tableName, fields); diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java index b4edd38c7..d587639aa 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java @@ -17,11 +17,8 @@ */ package com.dtstack.flink.sql.sink.rdb; -import com.dtstack.flink.sql.enums.EDatabaseType; import com.dtstack.flink.sql.sink.IStreamSinkGener; -import com.dtstack.flink.sql.sink.rdb.format.OracleOutputFormat; import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; -import com.dtstack.flink.sql.sink.rdb.format.SqlserverOutputFormat; import com.dtstack.flink.sql.sink.rdb.table.RdbTableInfo; import com.dtstack.flink.sql.table.TargetTableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -85,7 +82,7 @@ public RichSinkFunction createJdbcSinkFunc() { throw new RuntimeException("any of params in(driverName, dbURL, userName, password, type, tableName) " + " must not be null. please check it!!!"); } - RetractJDBCOutputFormat outputFormat = getOutputFormat(dbType); + RetractJDBCOutputFormat outputFormat = getOutputFormat(); outputFormat.setDbURL(dbURL); outputFormat.setDrivername(driverName); outputFormat.setUsername(userName); @@ -102,15 +99,7 @@ public RichSinkFunction createJdbcSinkFunc() { return outputFormatSinkFunc; } - protected RetractJDBCOutputFormat getOutputFormat(String dbType) { - if (dbType.equalsIgnoreCase(EDatabaseType.SQLSERVER.name())) { - return new SqlserverOutputFormat(); - } else if (dbType.equalsIgnoreCase(EDatabaseType.ORACLE.name())) { - return new OracleOutputFormat(); - } else { - return new RetractJDBCOutputFormat(); - } - } + public abstract RetractJDBCOutputFormat getOutputFormat(); @Override diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/OracleOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/OracleOutputFormat.java deleted file mode 100644 index a9d635bca..000000000 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/OracleOutputFormat.java +++ /dev/null @@ -1,115 +0,0 @@ -/** - * 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 com.dtstack.flink.sql.sink.rdb.format; - -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; -import java.sql.PreparedStatement; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -/** - * Reason: - * Date: 2018/11/30 - * Company: www.dtstack.com - * - * @author maqi - */ -public class OracleOutputFormat extends RetractJDBCOutputFormat { - - private final static String GET_ORACLE_INDEX_SQL = "SELECT " + - "t.INDEX_NAME," + - "t.COLUMN_NAME " + - "FROM " + - "user_ind_columns t," + - "user_indexes i " + - "WHERE " + - "t.index_name = i.index_name " + - "AND i.uniqueness = 'UNIQUE' " + - "AND t.table_name = '%s'"; - - - @Override - public boolean isReplaceInsertQuery() throws SQLException { - fillRealIndexes(); - fillFullColumns(); - - if (!getRealIndexes().isEmpty()) { - for (List value : getRealIndexes().values()) { - for (String fieldName : getDbSink().getFieldNames()) { - if (value.contains(fieldName)) { - return true; - } - } - } - } - return false; - } - - /** - * get db all index - * - * @throws SQLException - */ - public void fillRealIndexes() throws SQLException { - Map> map = Maps.newHashMap(); - - PreparedStatement ps = getDbConn().prepareStatement(String.format(GET_ORACLE_INDEX_SQL, getTableName())); - ResultSet rs = ps.executeQuery(); - - while (rs.next()) { - String indexName = rs.getString("INDEX_NAME"); - if (!map.containsKey(indexName)) { - map.put(indexName, new ArrayList<>()); - } - String column_name = rs.getString("COLUMN_NAME"); - if (StringUtils.isNotBlank(column_name)) { - column_name = column_name.toUpperCase(); - } - map.get(indexName).add(column_name); - } - - for (Map.Entry> entry : map.entrySet()) { - String k = entry.getKey(); - List v = entry.getValue(); - if (v != null && v.size() != 0 && v.get(0) != null) { - getRealIndexes().put(k, v); - } - } - } - - /** - * get db all column name - * - * @throws SQLException - */ - public void fillFullColumns() throws SQLException { - ResultSet rs = getDbConn().getMetaData().getColumns(null, null, getTableName(), null); - while (rs.next()) { - String columnName = rs.getString("COLUMN_NAME"); - if (StringUtils.isNotBlank(columnName)) { - getFullField().add(columnName.toUpperCase()); - } - } - } - - -} diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/SqlserverOutputFormat.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java similarity index 94% rename from rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/SqlserverOutputFormat.java rename to sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java index 612bf7f17..21d1f3bd1 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/SqlserverOutputFormat.java +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -6,17 +6,19 @@ * 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 - *

+ * + * 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 com.dtstack.flink.sql.sink.rdb.format; +package com.dtstack.flink.sql.sink.sqlserver; + +import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; import org.apache.commons.lang3.StringUtils; import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import java.sql.ResultSet; diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java index dbd75635e..1e266e3bc 100644 --- a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java @@ -19,6 +19,7 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; import org.apache.commons.lang3.StringUtils; import java.util.*; @@ -38,6 +39,11 @@ public String getDriverName() { return SQLSERVER_DRIVER; } + @Override + public RetractJDBCOutputFormat getOutputFormat() { + return new SqlserverOutputFormat(); + } + @Override public void buildSql(String tableName, List fields) { buildInsertSql(tableName, fields); From 07924945f9cb8a1601e142dc7354e77569a485de Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Fri, 30 Nov 2018 19:07:10 +0800 Subject: [PATCH 177/470] extract outputformat --- .../java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java | 6 +++++- .../com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java | 2 +- .../main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java | 5 +++-- .../sql/sink/rdb/{format => }/RetractJDBCOutputFormat.java | 3 +-- .../flink/sql/sink/sqlserver/SqlserverOutputFormat.java | 1 - 5 files changed, 10 insertions(+), 7 deletions(-) rename rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/{format => }/RetractJDBCOutputFormat.java (99%) diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java index 6d4d86d23..b872ecccd 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java @@ -23,7 +23,6 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.rdb.RdbSink; import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; - import java.util.List; import java.util.Map; @@ -79,5 +78,10 @@ public String getDriverName() { return MYSQL_DRIVER; } + @Override + public RetractJDBCOutputFormat getOutputFormat() { + return new RetractJDBCOutputFormat(); + } + } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java index 54ea47a18..04ad57306 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java @@ -213,7 +213,7 @@ private void loadData(Map>> tmpCache) throws SQ } } - public int getFetchSize(){ + public int getFetchSize() { return 1000; } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java index d587639aa..a3611bfc6 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java @@ -99,8 +99,6 @@ public RichSinkFunction createJdbcSinkFunc() { return outputFormatSinkFunc; } - public abstract RetractJDBCOutputFormat getOutputFormat(); - @Override public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { @@ -252,4 +250,7 @@ public void setDbType(String dbType) { public abstract String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField); public abstract String getDriverName(); + + public abstract RetractJDBCOutputFormat getOutputFormat(); + } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java similarity index 99% rename from rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java rename to rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java index 8f5329796..a003cb69a 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java @@ -16,9 +16,8 @@ * limitations under the License. */ -package com.dtstack.flink.sql.sink.rdb.format; +package com.dtstack.flink.sql.sink.rdb; -import com.dtstack.flink.sql.sink.rdb.RdbSink; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java index 21d1f3bd1..7708ba163 100644 --- a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java @@ -17,7 +17,6 @@ */ package com.dtstack.flink.sql.sink.sqlserver; - import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; import org.apache.commons.lang3.StringUtils; import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; From c7976db667d5d9c8b8ea958e44ed76ca7f0b9ede Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Mon, 3 Dec 2018 09:47:19 +0800 Subject: [PATCH 178/470] add output package --- .../java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java | 5 ----- .../sql/sink/rdb/{ => format}/RetractJDBCOutputFormat.java | 4 ++-- 2 files changed, 2 insertions(+), 7 deletions(-) rename rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/{ => format}/RetractJDBCOutputFormat.java (99%) diff --git a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java index b872ecccd..d045729e8 100644 --- a/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java +++ b/mysql/mysql-sink/src/main/java/com/dtstack/flink/sql/sink/mysql/MysqlSink.java @@ -78,10 +78,5 @@ public String getDriverName() { return MYSQL_DRIVER; } - @Override - public RetractJDBCOutputFormat getOutputFormat() { - return new RetractJDBCOutputFormat(); - } - } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java similarity index 99% rename from rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java rename to rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index a003cb69a..382b687e1 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -16,8 +16,9 @@ * limitations under the License. */ -package com.dtstack.flink.sql.sink.rdb; +package com.dtstack.flink.sql.sink.rdb.format; +import com.dtstack.flink.sql.sink.rdb.RdbSink; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; @@ -27,7 +28,6 @@ import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.sql.*; import java.util.*; import java.io.IOException; From 863965af24c940b836adf78c0be2cf7311779103 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 3 Dec 2018 16:33:08 +0800 Subject: [PATCH 179/470] case sensitive --- .../flink/sql/sink/redis/table/RedisSinkParser.java | 10 +++++----- .../flink/sql/sink/redis/table/RedisTableInfo.java | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java index 6960dc249..5754bdfee 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java @@ -37,11 +37,11 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Tue, 4 Dec 2018 10:18:08 +0800 Subject: [PATCH 180/470] oracle side and sink --- oracle/oracle-side/oracle-all-side/pom.xml | 91 +++++++ .../sql/side/oracle/OracleAllReqRow.java | 62 +++++ .../sql/side/oracle/OracleAllSideInfo.java | 33 +++ oracle/oracle-side/oracle-async-side/pom.xml | 92 +++++++ .../sql/side/oracle/OracleAsyncReqRow.java | 66 +++++ .../sql/side/oracle/OracleAsyncSideInfo.java | 35 +++ oracle/oracle-side/oracle-side-core/pom.xml | 18 ++ .../side/oracle/table/OracleSideParser.java | 35 +++ oracle/oracle-side/pom.xml | 33 +++ oracle/oracle-sink/pom.xml | 90 +++++++ .../sql/sink/oracle/OracleOutputFormat.java | 105 ++++++++ .../flink/sql/sink/oracle/OracleSink.java | 171 +++++++++++++ .../sink/oracle/table/OracleSinkParser.java | 41 +++ oracle/pom.xml | 39 +++ pom.xml | 1 + .../flink/sql/side/rdb/all/RdbAllReqRow.java | 11 +- .../sql/side/rdb/async/RdbAsyncReqRow.java | 9 +- .../flink/sql/side/rdb/util/MathUtil.java | 236 ++++++++++++++++++ .../flink/sql/side/rdb/util/SwitchUtil.java | 58 +++++ 19 files changed, 1223 insertions(+), 3 deletions(-) create mode 100644 oracle/oracle-side/oracle-all-side/pom.xml create mode 100644 oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java create mode 100644 oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java create mode 100644 oracle/oracle-side/oracle-async-side/pom.xml create mode 100644 oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java create mode 100644 oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java create mode 100644 oracle/oracle-side/oracle-side-core/pom.xml create mode 100644 oracle/oracle-side/oracle-side-core/src/main/java/com/dtstack/flink/sql/side/oracle/table/OracleSideParser.java create mode 100644 oracle/oracle-side/pom.xml create mode 100644 oracle/oracle-sink/pom.xml create mode 100644 oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleOutputFormat.java create mode 100644 oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java create mode 100644 oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java create mode 100644 oracle/pom.xml create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java diff --git a/oracle/oracle-side/oracle-all-side/pom.xml b/oracle/oracle-side/oracle-all-side/pom.xml new file mode 100644 index 000000000..458077376 --- /dev/null +++ b/oracle/oracle-side/oracle-all-side/pom.xml @@ -0,0 +1,91 @@ + + + + sql.side.oracle + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.all.oracle + 1.0-SNAPSHOT + oracle-all-side + jar + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.side.oracle.core + ${sql.side.oracle.core.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java new file mode 100644 index 000000000..b6779529e --- /dev/null +++ b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.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 com.dtstack.flink.sql.side.oracle; + + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.util.List; +import java.util.Map; + +/** + * side operator with cache for all(period reload) + */ +public class OracleAllReqRow extends RdbAllReqRow { + + private static final Logger LOG = LoggerFactory.getLogger(OracleAllReqRow.class); + + private static final String ORACLE_DRIVER = "oracle.jdbc.driver.OracleDriver"; + + public OracleAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public Connection getConn(String dbURL, String userName, String password) { + try { + Class.forName(ORACLE_DRIVER); + Map addParams = Maps.newHashMap(); + String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); + return DriverManager.getConnection(targetDbUrl, userName, password); + } catch (Exception e) { + LOG.error("", e); + throw new RuntimeException("", e); + } + } + +} diff --git a/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java new file mode 100644 index 000000000..4063bf9bb --- /dev/null +++ b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.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 com.dtstack.flink.sql.side.oracle; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; + +import java.util.List; + +public class OracleAllSideInfo extends RdbAllSideInfo { + + public OracleAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } +} diff --git a/oracle/oracle-side/oracle-async-side/pom.xml b/oracle/oracle-side/oracle-async-side/pom.xml new file mode 100644 index 000000000..3a4b36559 --- /dev/null +++ b/oracle/oracle-side/oracle-async-side/pom.xml @@ -0,0 +1,92 @@ + + + + sql.side.oracle + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.async.oracle + 1.0-SNAPSHOT + oracle-async-side + jar + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.side.oracle.core + ${sql.side.oracle.core.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java new file mode 100644 index 000000000..c68850d21 --- /dev/null +++ b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java @@ -0,0 +1,66 @@ +/* + * 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 com.dtstack.flink.sql.side.oracle; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.async.RdbAsyncReqRow; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import io.vertx.core.Vertx; +import io.vertx.core.VertxOptions; +import io.vertx.core.json.JsonObject; +import io.vertx.ext.jdbc.JDBCClient; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + + +public class OracleAsyncReqRow extends RdbAsyncReqRow { + + private static final Logger LOG = LoggerFactory.getLogger(OracleAsyncReqRow.class); + + private static final String ORACLE_DRIVER = "oracle.jdbc.driver.OracleDriver"; + + public OracleAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + JsonObject sqlserverClientConfig = new JsonObject(); + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); + sqlserverClientConfig.put("url", rdbSideTableInfo.getUrl()) + .put("driver_class", ORACLE_DRIVER) + .put("max_pool_size", DEFAULT_MAX_DB_CONN_POOL_SIZE) + .put("user", rdbSideTableInfo.getUserName()) + .put("password", rdbSideTableInfo.getPassword()); + + VertxOptions vo = new VertxOptions(); + vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); + vo.setWorkerPoolSize(DEFAULT_VERTX_WORKER_POOL_SIZE); + Vertx vertx = Vertx.vertx(vo); + setRdbSQLClient(JDBCClient.createNonShared(vertx, sqlserverClientConfig)); + } +} diff --git a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java new file mode 100644 index 000000000..7ff63f3d9 --- /dev/null +++ b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java @@ -0,0 +1,35 @@ +/* + * 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 com.dtstack.flink.sql.side.oracle; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; + +import java.util.List; + + +public class OracleAsyncSideInfo extends RdbAsyncSideInfo { + + public OracleAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } +} diff --git a/oracle/oracle-side/oracle-side-core/pom.xml b/oracle/oracle-side/oracle-side-core/pom.xml new file mode 100644 index 000000000..cc954d9fd --- /dev/null +++ b/oracle/oracle-side/oracle-side-core/pom.xml @@ -0,0 +1,18 @@ + + + + sql.side.oracle + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.oracle.core + 1.0-SNAPSHOT + jar + oracle-side-core + + \ No newline at end of file diff --git a/oracle/oracle-side/oracle-side-core/src/main/java/com/dtstack/flink/sql/side/oracle/table/OracleSideParser.java b/oracle/oracle-side/oracle-side-core/src/main/java/com/dtstack/flink/sql/side/oracle/table/OracleSideParser.java new file mode 100644 index 000000000..f9124bd10 --- /dev/null +++ b/oracle/oracle-side/oracle-side-core/src/main/java/com/dtstack/flink/sql/side/oracle/table/OracleSideParser.java @@ -0,0 +1,35 @@ +/** + * 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 com.dtstack.flink.sql.side.oracle.table; + +import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; +import com.dtstack.flink.sql.table.TableInfo; + +import java.util.Map; + + +public class OracleSideParser extends RdbSideParser { + private static final String CURR_TYPE = "oracle"; + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + TableInfo oracleTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + oracleTableInfo.setType(CURR_TYPE); + return oracleTableInfo; + } +} diff --git a/oracle/oracle-side/pom.xml b/oracle/oracle-side/pom.xml new file mode 100644 index 000000000..e0ffe547b --- /dev/null +++ b/oracle/oracle-side/pom.xml @@ -0,0 +1,33 @@ + + + + sql.oracle + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.oracle + 1.0-SNAPSHOT + + oracle-side-core + oracle-all-side + oracle-async-side + + pom + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.side.rdb + ${rdb.side.version} + + + \ No newline at end of file diff --git a/oracle/oracle-sink/pom.xml b/oracle/oracle-sink/pom.xml new file mode 100644 index 000000000..c5807f806 --- /dev/null +++ b/oracle/oracle-sink/pom.xml @@ -0,0 +1,90 @@ + + + + sql.oracle + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.oracle + 1.0-SNAPSHOT + oracle-sink + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.sink.rdb + ${sql.sink.rdb.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleOutputFormat.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleOutputFormat.java new file mode 100644 index 000000000..dbbeea270 --- /dev/null +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleOutputFormat.java @@ -0,0 +1,105 @@ +/* + * 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 com.dtstack.flink.sql.sink.oracle; + +import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; + +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/30 + * Company: www.dtstack.com + * + * @author maqi + */ +public class OracleOutputFormat extends RetractJDBCOutputFormat { + + + @Override + public boolean isReplaceInsertQuery() throws SQLException { + fillRealIndexes(); + fillFullColumns(); + + if (!getRealIndexes().isEmpty()) { + for (List value : getRealIndexes().values()) { + for (String fieldName : getDbSink().getFieldNames()) { + if (value.contains(fieldName)) { + return true; + } + } + } + } + return false; + } + + /** + * get db all index + * + * @throws SQLException + */ + public void fillRealIndexes() throws SQLException { + Map> map = Maps.newHashMap(); + ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, null, getTableName(), true, false); + + while (rs.next()) { + String indexName = rs.getString("INDEX_NAME"); + if (!map.containsKey(indexName)) { + map.put(indexName, new ArrayList<>()); + } + String column_name = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(column_name)) { + column_name = column_name.toUpperCase(); + } + map.get(indexName).add(column_name); + } + + for (Map.Entry> entry : map.entrySet()) { + String k = entry.getKey(); + List v = entry.getValue(); + if (v != null && v.size() != 0 && v.get(0) != null) { + getRealIndexes().put(k, v); + } + } + } + + /** + * get db all column name + * + * @throws SQLException + */ + public void fillFullColumns() throws SQLException { + String schema = null; + ResultSet rs = getDbConn().getMetaData().getColumns(null, schema, getTableName(), null); + while (rs.next()) { + String columnName = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(columnName)) { + getFullField().add(columnName.toUpperCase()); + } + } + } + + +} diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java new file mode 100644 index 000000000..de1ffe8e5 --- /dev/null +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -0,0 +1,171 @@ +/** + * 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 com.dtstack.flink.sql.sink.oracle; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; +import org.apache.commons.lang3.StringUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/27 + * Company: www.dtstack.com + * + * @author maqi + */ +public class OracleSink extends RdbSink implements IStreamSinkGener { + private static final String ORACLE_DRIVER = "oracle.jdbc.driver.OracleDriver"; + + @Override + public String getDriverName() { + return ORACLE_DRIVER; + } + + @Override + public RetractJDBCOutputFormat getOutputFormat() { + return new OracleOutputFormat(); + } + + @Override + public void buildSql(String tableName, List fields) { + buildInsertSql(tableName, fields); + } + + private void buildInsertSql(String tableName, List fields) { + String sqlTmp = "insert into " + tableName + " (${fields}) values (${placeholder})"; + String fieldsStr = StringUtils.join(fields, ","); + String placeholder = ""; + + for (String fieldName : fields) { + placeholder += ",?"; + } + placeholder = placeholder.replaceFirst(",", ""); + sqlTmp = sqlTmp.replace("${fields}", fieldsStr).replace("${placeholder}", placeholder); + this.sql = sqlTmp; + } + + @Override + public String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField) { + return "MERGE INTO " + tableName + " T1 USING " + + "(" + makeValues(fieldNames) + ") T2 ON (" + + updateKeySql(realIndexes) + ") WHEN MATCHED THEN UPDATE SET " + + getUpdateSql(fieldNames, fullField, "T1", "T2", keyColList(realIndexes)) + " WHEN NOT MATCHED THEN " + + "INSERT (" + quoteColumns(fieldNames) + ") VALUES (" + + quoteColumns(fieldNames, "T2") + ");"; + } + + + public String quoteColumns(List column) { + return quoteColumns(column, null); + } + + public String quoteColumns(List column, String table) { + String prefix = StringUtils.isBlank(table) ? "" : quoteTable(table) + "."; + List list = new ArrayList<>(); + for (String col : column) { + list.add(prefix + quoteColumn(col)); + } + return StringUtils.join(list, ","); + } + + protected List keyColList(Map> updateKey) { + List keyCols = new ArrayList<>(); + for (Map.Entry> entry : updateKey.entrySet()) { + List list = entry.getValue(); + for (String col : list) { + if (!keyCols.contains(col)) { + keyCols.add(col); + } + } + } + return keyCols; + } + + public String getUpdateSql(List column, List fullColumn, String leftTable, String rightTable, List keyCols) { + String prefixLeft = StringUtils.isBlank(leftTable) ? "" : quoteTable(leftTable) + "."; + String prefixRight = StringUtils.isBlank(rightTable) ? "" : quoteTable(rightTable) + "."; + List list = new ArrayList<>(); + for (String col : fullColumn) { + if (keyCols == null || keyCols.size() == 0) { + continue; + } + if (fullColumn == null || column.contains(col)) { + list.add(prefixLeft + col + "=" + prefixRight + col); + } else { + list.add(prefixLeft + col + "=null"); + } + } + return StringUtils.join(list, ","); + } + + public String quoteTable(String table) { + String[] parts = table.split("\\."); + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < parts.length; ++i) { + if (i != 0) { + sb.append("."); + } + sb.append(getStartQuote() + parts[i] + getEndQuote()); + } + return sb.toString(); + } + + + public String updateKeySql(Map> updateKey) { + List exprList = new ArrayList<>(); + for (Map.Entry> entry : updateKey.entrySet()) { + List colList = new ArrayList<>(); + for (String col : entry.getValue()) { + colList.add("T1." + quoteColumn(col) + "=T2." + quoteColumn(col)); + } + exprList.add(StringUtils.join(colList, " AND ")); + } + return StringUtils.join(exprList, " OR "); + } + + + public String makeValues(List column) { + StringBuilder sb = new StringBuilder("SELECT "); + for (int i = 0; i < column.size(); ++i) { + if (i != 0) { + sb.append(","); + } + sb.append("? " + quoteColumn(column.get(i))); + } + return sb.toString(); + } + + public String quoteColumn(String column) { + return getStartQuote() + column + getEndQuote(); + } + + public String getStartQuote() { + return "\""; + } + + public String getEndQuote() { + return "\""; + } + + +} diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java new file mode 100644 index 000000000..6db2c9c06 --- /dev/null +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java @@ -0,0 +1,41 @@ +/** + * 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 com.dtstack.flink.sql.sink.oracle.table; + +import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; +import com.dtstack.flink.sql.table.TableInfo; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/27 + * Company: www.dtstack.com + * + * @author maqi + */ +public class OracleSinkParser extends RdbSinkParser { + private static final String CURR_TYPE = "oracle"; + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + TableInfo sqlserverTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + sqlserverTableInfo.setType(CURR_TYPE); + return sqlserverTableInfo; + } +} diff --git a/oracle/pom.xml b/oracle/pom.xml new file mode 100644 index 000000000..60565a149 --- /dev/null +++ b/oracle/pom.xml @@ -0,0 +1,39 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.oracle + 1.0-SNAPSHOT + + oracle-side + oracle-sink + + pom + + + 12.2.0.1 + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.core + ${sql.core.version} + provided + + + + com.github.noraui + ojdbc8 + ${ojdbc.version} + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index bde73d6df..a31cc7392 100644 --- a/pom.xml +++ b/pom.xml @@ -18,6 +18,7 @@ launcher rdb sqlserver + oracle pom diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java index 04ad57306..fce61e238 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java @@ -23,8 +23,11 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import com.dtstack.flink.sql.side.rdb.util.MathUtil; +import com.dtstack.flink.sql.side.rdb.util.SwitchUtil; import org.apache.calcite.sql.JoinType; import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; @@ -34,6 +37,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.math.BigDecimal; import java.sql.*; import java.util.Calendar; import java.util.List; @@ -74,6 +78,7 @@ protected Row fillData(Row input, Object sideInput) { if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { obj = ((Timestamp) obj).getTime(); } + row.setField(entry.getKey(), obj); } @@ -194,10 +199,14 @@ private void loadData(Map>> tmpCache) throws SQ statement.setFetchSize(getFetchSize()); ResultSet resultSet = statement.executeQuery(sql); String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); + String[] fields = sideInfo.getSideTableInfo().getFieldTypes(); while (resultSet.next()) { Map oneRow = Maps.newHashMap(); for (String fieldName : sideFieldNames) { - oneRow.put(fieldName.trim(), resultSet.getObject(fieldName.trim())); + Object object = resultSet.getObject(fieldName.trim()); + int fieldIndex = sideInfo.getRowTypeInfo().getFieldIndex(fieldName.trim()); + object = SwitchUtil.getTarget(object, fields[fieldIndex]); + oneRow.put(fieldName.trim(), object); } String cacheKey = buildKey(oneRow, sideInfo.getEqualFieldList()); diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index a7e5339e9..2c3d9b3f4 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -22,9 +22,11 @@ import com.dtstack.flink.sql.enums.ECacheContentType; import com.dtstack.flink.sql.side.*; import com.dtstack.flink.sql.side.cache.CacheObj; +import com.dtstack.flink.sql.side.rdb.util.SwitchUtil; import io.vertx.core.json.JsonArray; import io.vertx.ext.sql.SQLClient; import io.vertx.ext.sql.SQLConnection; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.configuration.Configuration; @@ -34,6 +36,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.math.BigInteger; import java.sql.Timestamp; import java.util.Collections; import java.util.List; @@ -120,6 +123,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except int resultSize = rs.result().getResults().size(); if (resultSize > 0) { for (JsonArray line : rs.result().getResults()) { + Row row = fillData(input, line); if (openCache()) { cacheContent.add(line); @@ -151,10 +155,10 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except public Row fillData(Row input, Object line) { JsonArray jsonArray = (JsonArray) line; Row row = new Row(sideInfo.getOutFieldInfoList().size()); + String[] fields = sideInfo.getSideTableInfo().getFieldTypes(); for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { Object obj = input.getField(entry.getValue()); boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); - if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { obj = ((Timestamp) obj).getTime(); } @@ -166,7 +170,8 @@ public Row fillData(Row input, Object line) { if (jsonArray == null) { row.setField(entry.getKey(), null); } else { - row.setField(entry.getKey(), jsonArray.getValue(entry.getValue())); + Object object = SwitchUtil.getTarget(jsonArray.getValue(entry.getValue()), fields[entry.getKey()]); + row.setField(entry.getKey(), object); } } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java new file mode 100644 index 000000000..aa7442586 --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java @@ -0,0 +1,236 @@ +/* + * 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 com.dtstack.flink.sql.side.rdb.util; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Date; + +/** + * Date: 2017/4/21 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class MathUtil { + + public static Long getLongVal(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof String) { + return Long.valueOf((String) obj); + } else if (obj instanceof Long) { + return (Long) obj; + } else if (obj instanceof Integer) { + return Long.valueOf(obj.toString()); + } else if (obj instanceof BigDecimal) { + return ((BigDecimal) obj).longValue(); + } + + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Long."); + } + + public static Long getLongVal(Object obj, long defaultVal) { + if (obj == null) { + return defaultVal; + } + + return getLongVal(obj); + } + + public static Integer getIntegerVal(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof String) { + return Integer.valueOf((String) obj); + } else if (obj instanceof Integer) { + return (Integer) obj; + } else if (obj instanceof Long) { + return ((Long) obj).intValue(); + } else if (obj instanceof Double) { + return ((Double) obj).intValue(); + } else if (obj instanceof BigDecimal) { + return ((BigDecimal) obj).intValue(); + } + + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Integer."); + } + + public static Integer getIntegerVal(Object obj, int defaultVal) { + if (obj == null) { + return defaultVal; + } + + return getIntegerVal(obj); + } + + public static Float getFloatVal(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof String) { + return Float.valueOf((String) obj); + } else if (obj instanceof Float) { + return (Float) obj; + } else if (obj instanceof BigDecimal) { + return ((BigDecimal) obj).floatValue(); + } + + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Float."); + } + + public static Float getFloatVal(Object obj, float defaultVal) { + if (obj == null) { + return defaultVal; + } + + return getFloatVal(obj); + } + + public static Double getDoubleVal(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof String) { + return Double.valueOf((String) obj); + } else if (obj instanceof Float) { + return (Double) obj; + } else if (obj instanceof BigDecimal) { + return ((BigDecimal) obj).doubleValue(); + } + + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Double."); + } + + public static Double getDoubleVal(Object obj, double defaultVal) { + if (obj == null) { + return defaultVal; + } + + return getDoubleVal(obj); + } + + + public static Boolean getBoolean(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof String) { + return Boolean.valueOf((String) obj); + } else if (obj instanceof Boolean) { + return (Boolean) obj; + } + + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Boolean."); + } + + public static Boolean getBoolean(Object obj, boolean defaultVal) { + if (obj == null) { + return defaultVal; + } + + return getBoolean(obj); + } + + public static String getString(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof String) { + return (String) obj; + } + + return obj.toString(); + } + + public static Byte getByte(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof String) { + return Byte.valueOf((String) obj); + } else if (obj instanceof Byte) { + return (Byte) obj; + } + + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Byte."); + } + + public static Short getShort(Object obj) { + if (obj == null) { + return null; + } + + if (obj instanceof String) { + return Short.valueOf((String) obj); + } else if (obj instanceof Short) { + return (Short) obj; + } + + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Short."); + } + + public static BigDecimal getBigDecimal(Object obj) { + if (obj == null) { + return null; + } + if (obj instanceof String) { + return new BigDecimal((String) obj); + } else if (obj instanceof BigDecimal) { + return (BigDecimal) obj; + } else if (obj instanceof BigInteger) { + return new BigDecimal((BigInteger) obj); + } else if (obj instanceof Number) { + return new BigDecimal(((Number) obj).doubleValue()); + } + throw new RuntimeException("not support type of " + obj.getClass() + " convert to BigDecimal."); + } + + public static Date getDate(Object obj) { + if (obj == null) { + return null; + } + SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + if (obj instanceof String) { + try { + return sdf.parse((String) obj); + } catch (ParseException e) { + e.printStackTrace(); + } + } else if (obj instanceof Date) { + return (Date) obj; + } + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Date."); + } + + +} \ No newline at end of file diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java new file mode 100644 index 000000000..eadbdc476 --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java @@ -0,0 +1,58 @@ +/** + * 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 com.dtstack.flink.sql.side.rdb.util; + +import java.text.ParseException; + +/** + * Reason: + * Date: 2018/12/3 + * Company: www.dtstack.com + * + * @author maqi + */ +public class SwitchUtil { + public static Object getTarget(Object obj, String targetType) { + targetType = targetType.toLowerCase(); + switch (targetType) { + case "int": + case "integer": + return MathUtil.getIntegerVal(obj); + case "bigint": + return MathUtil.getLongVal(obj); + case "boolean": + return MathUtil.getBoolean(obj); + case "tinyint": + return MathUtil.getByte(obj); + case "smallint": + return MathUtil.getShort(obj); + case "varchar": + return MathUtil.getString(obj); + case "real": + case "float": + return MathUtil.getFloatVal(obj); + case "double": + return MathUtil.getDoubleVal(obj); + case "decimal": + return MathUtil.getBigDecimal(obj); + case "date": + return MathUtil.getDate(obj); + } + return obj; + } +} From 4c9af4f9a5319ef54e760b1e037162e7e4afce00 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 4 Dec 2018 10:42:40 +0800 Subject: [PATCH 181/470] fix abstract tableInfo not Serializable --- .../dtstack/flink/sql/table/TableInfo.java | 26 ++++++++++++++++++- .../flink/sql/launcher/LauncherMain.java | 4 ++- .../sql/side/rdb/async/RdbAsyncSideInfo.java | 2 ++ .../sql/side/rdb/table/RdbSideTableInfo.java | 1 + 4 files changed, 31 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java index 71bea466f..b56b790ea 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java @@ -22,6 +22,7 @@ import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import java.io.Serializable; import java.util.List; /** @@ -31,7 +32,7 @@ * @author xuchao */ -public abstract class TableInfo { +public abstract class TableInfo implements Serializable { public static final String PARALLELISM_KEY = "parallelism"; @@ -121,6 +122,29 @@ public void addFieldType(String fieldType){ fieldTypeList.add(fieldType); } + public void setFields(String[] fields) { + this.fields = fields; + } + + public void setFieldTypes(String[] fieldTypes) { + this.fieldTypes = fieldTypes; + } + + public void setFieldClasses(Class[] fieldClasses) { + this.fieldClasses = fieldClasses; + } + + public List getFieldList() { + return fieldList; + } + + public List getFieldTypeList() { + return fieldTypeList; + } + + public List getFieldClassList() { + return fieldClassList; + } public void finish(){ this.fields = fieldList.toArray(new String[fieldList.size()]); diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 10bc09475..eea52e0af 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -59,13 +59,15 @@ private static String getLocalCoreJarPath(String localSqlRootJar){ } public static void main(String[] args) throws Exception { - if (args.length==1 && args[0].endsWith(".json")){ + if (args.length == 1 && args[0].endsWith(".json")){ args = parseJson(args); } + LauncherOptionParser optionParser = new LauncherOptionParser(args); LauncherOptions launcherOptions = optionParser.getLauncherOptions(); String mode = launcherOptions.getMode(); List argList = optionParser.getProgramExeArgList(); + if(mode.equals(ClusterMode.local.name())) { String[] localArgs = argList.toArray(new String[argList.size()]); Main.main(localArgs); diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java index b7ff94ea9..f652bf342 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java @@ -42,6 +42,8 @@ public class RdbAsyncSideInfo extends SideInfo { + private static final long serialVersionUID = 1942629132469918611L; + public RdbAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java index c08f9a3a9..44d0af11e 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java @@ -28,6 +28,7 @@ * @author maqi */ public class RdbSideTableInfo extends SideTableInfo { + private static final long serialVersionUID = -1L; public static final String URL_KEY = "url"; From 6b7c031477e5ba2475f244d12ca2cde7af9f4675 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 4 Dec 2018 11:29:39 +0800 Subject: [PATCH 182/470] modify reqRow to a interface --- .../com/dtstack/flink/sql/side/AllReqRow.java | 4 +- .../dtstack/flink/sql/side/AsyncReqRow.java | 5 +- .../dtstack/flink/sql/side/ISideReqRow.java | 33 +++++++++ .../flink/sql/side/hbase/HbaseAllReqRow.java | 2 +- .../sql/side/hbase/HbaseAsyncReqRow.java | 2 +- .../flink/sql/side/mongo/MongoAllReqRow.java | 2 +- .../flink/sql/side/rdb/all/RdbAllReqRow.java | 2 +- .../flink/sql/side/redis/RedisAllReqRow.java | 29 ++------ .../sql/side/redis/RedisAsyncReqRow.java | 29 ++------ .../sql/side/redis/table/RedisSideReqRow.java | 72 +++++++++++++++++++ 10 files changed, 122 insertions(+), 58 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/side/ISideReqRow.java create mode 100644 redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideReqRow.java diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java index 303260742..a185da1bd 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java @@ -37,7 +37,7 @@ * @author xuchao */ -public abstract class AllReqRow extends RichFlatMapFunction{ +public abstract class AllReqRow extends RichFlatMapFunction implements ISideReqRow { protected SideInfo sideInfo; @@ -48,8 +48,6 @@ public AllReqRow(SideInfo sideInfo){ } - protected abstract Row fillData(Row input, Object sideInput); - protected abstract void initCache() throws SQLException; protected abstract void reloadCache(); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java index 6617bb29f..6df1af1d5 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java @@ -40,7 +40,7 @@ * @author xuchao */ -public abstract class AsyncReqRow extends RichAsyncFunction { +public abstract class AsyncReqRow extends RichAsyncFunction implements ISideReqRow { private static final long serialVersionUID = 2098635244857937717L; @@ -79,9 +79,6 @@ protected boolean openCache(){ return sideInfo.getSideCache() != null; } - - protected abstract Row fillData(Row input, Object sideInput); - protected void dealMissKey(Row input, ResultFuture resultFuture){ if(sideInfo.getJoinType() == JoinType.LEFT){ //Reserved left table data diff --git a/core/src/main/java/com/dtstack/flink/sql/side/ISideReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/ISideReqRow.java new file mode 100644 index 000000000..88066e37f --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/side/ISideReqRow.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 com.dtstack.flink.sql.side; + +import org.apache.flink.types.Row; + +/** + * + * Date: 2018/12/4 + * Company: www.dtstack.com + * @author xuchao + */ +public interface ISideReqRow { + + Row fillData(Row input, Object sideInput); + +} diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java index 29fca4bee..de89f337e 100644 --- a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java @@ -66,7 +66,7 @@ public HbaseAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List sideInputList = (Map) sideInput; Row row = new Row(sideInfo.getOutFieldInfoList().size()); for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java index fe227bb6a..6e82e4109 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java @@ -160,7 +160,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except } @Override - protected Row fillData(Row input, Object sideInput){ + public Row fillData(Row input, Object sideInput){ List sideInputList = (List) sideInput; Row row = new Row(sideInfo.getOutFieldInfoList().size()); diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java index e1a398d4c..c1c812cd0 100644 --- a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java @@ -80,7 +80,7 @@ public MongoAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List cacheInfo = (Map) sideInput; Row row = new Row(sideInfo.getOutFieldInfoList().size()); for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java index fce61e238..c7e37c590 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java @@ -67,7 +67,7 @@ public RdbAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List } @Override - protected Row fillData(Row input, Object sideInput) { + public Row fillData(Row input, Object sideInput) { Map cacheInfo = (Map) sideInput; Row row = new Row(sideInfo.getOutFieldInfoList().size()); for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java index e0e0a7d17..54ecf8952 100644 --- a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.side.redis; import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.redis.table.RedisSideReqRow; import com.dtstack.flink.sql.side.redis.table.RedisSideTableInfo; import org.apache.calcite.sql.JoinType; import org.apache.commons.pool2.impl.GenericObjectPoolConfig; @@ -55,34 +56,16 @@ public class RedisAllReqRow extends AllReqRow{ private AtomicReference>> cacheRef = new AtomicReference<>(); + private RedisSideReqRow redisSideReqRow; + public RedisAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(new RedisAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + this.redisSideReqRow = new RedisSideReqRow(super.sideInfo); } @Override - protected Row fillData(Row input, Object sideInput) { - Map sideInputMap = (Map) sideInput; - Row row = new Row(sideInfo.getOutFieldInfoList().size()); - for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ - Object obj = input.getField(entry.getValue()); - boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); - - if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ - obj = ((Timestamp)obj).getTime(); - } - row.setField(entry.getKey(), obj); - } - - for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ - if(sideInputMap == null){ - row.setField(entry.getKey(), null); - }else{ - String key = sideInfo.getSideFieldNameIndex().get(entry.getKey()); - row.setField(entry.getKey(), sideInputMap.get(key)); - } - } - - return row; + public Row fillData(Row input, Object sideInput) { + return redisSideReqRow.fillData(input, sideInput); } @Override diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index 82532c527..3f9f40f24 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -21,6 +21,7 @@ import com.dtstack.flink.sql.enums.ECacheContentType; import com.dtstack.flink.sql.side.*; import com.dtstack.flink.sql.side.cache.CacheObj; +import com.dtstack.flink.sql.side.redis.table.RedisSideReqRow; import com.dtstack.flink.sql.side.redis.table.RedisSideTableInfo; import io.lettuce.core.KeyValue; import io.lettuce.core.RedisClient; @@ -60,9 +61,11 @@ public class RedisAsyncReqRow extends AsyncReqRow { private RedisSideTableInfo redisSideTableInfo; + private RedisSideReqRow redisSideReqRow; public RedisAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(new RedisAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + redisSideReqRow = new RedisSideReqRow(super.sideInfo); } @Override @@ -108,30 +111,8 @@ private void buildRedisClient(RedisSideTableInfo tableInfo){ } @Override - protected Row fillData(Row input, Object sideInput) { - Map keyValue = (Map) sideInput; - Row row = new Row(sideInfo.getOutFieldInfoList().size()); - for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ - Object obj = input.getField(entry.getValue()); - boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); - - if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ - obj = ((Timestamp)obj).getTime(); - } - - row.setField(entry.getKey(), obj); - } - - for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ - if(keyValue == null){ - row.setField(entry.getKey(), null); - }else{ - String key = sideInfo.getSideFieldNameIndex().get(entry.getKey()); - row.setField(entry.getKey(), keyValue.get(key)); - } - } - - return row; + public Row fillData(Row input, Object sideInput) { + return redisSideReqRow.fillData(input, sideInput); } @Override diff --git a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideReqRow.java b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideReqRow.java new file mode 100644 index 000000000..40dae6c68 --- /dev/null +++ b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideReqRow.java @@ -0,0 +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 com.dtstack.flink.sql.side.redis.table; + +import com.dtstack.flink.sql.side.ISideReqRow; +import com.dtstack.flink.sql.side.SideInfo; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; + +import java.io.Serializable; +import java.sql.Timestamp; +import java.util.Map; + +/** + * redis fill row data + * Date: 2018/12/4 + * Company: www.dtstack.com + * @author xuchao + */ + +public class RedisSideReqRow implements ISideReqRow, Serializable { + + private static final long serialVersionUID = 3751171828444748982L; + + private SideInfo sideInfo; + + public RedisSideReqRow(SideInfo sideInfo){ + this.sideInfo = sideInfo; + } + + @Override + public Row fillData(Row input, Object sideInput) { + Map sideInputMap = (Map) sideInput; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for(Map.Entry entry : sideInfo.getInFieldIndex().entrySet()){ + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + if(obj instanceof Timestamp && isTimeIndicatorTypeInfo){ + obj = ((Timestamp)obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for(Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()){ + if(sideInputMap == null){ + row.setField(entry.getKey(), null); + }else{ + String key = sideInfo.getSideFieldNameIndex().get(entry.getKey()); + row.setField(entry.getKey(), sideInputMap.get(key)); + } + } + + return row; + } +} From fca63ad710715c11e4d7ad366cd3ec8f14e5a8cf Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Tue, 4 Dec 2018 17:26:36 +0800 Subject: [PATCH 183/470] the value is case sensitive --- .../com/dtstack/flink/sql/parser/CreateTmpTableParser.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java index 9bd1374a0..8da89b2be 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.parser; +import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.calcite.sql.*; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; @@ -77,7 +78,8 @@ public void parseSql(String sql, SqlTree sqlTree) { parseNode(sqlNode, sqlParseResult); sqlParseResult.setTableName(tableName); - sqlParseResult.setExecSql(selectSql.toUpperCase()); + String transformSelectSql = DtStringUtil.replaceIgnoreQuota(sqlNode.toString(), "`", ""); + sqlParseResult.setExecSql(transformSelectSql); sqlTree.addTmpSql(sqlParseResult); sqlTree.addTmplTableInfo(tableName, sqlParseResult); } else { From 5f5d78e4fe77d1ddc9732e2cca009e92cf46f732 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Tue, 4 Dec 2018 21:29:58 +0800 Subject: [PATCH 184/470] oracle side and sink and modify jdbc getdate type --- .../sql/sink/oracle/OracleOutputFormat.java | 105 ------------------ .../flink/sql/sink/oracle/OracleSink.java | 8 +- .../flink/sql/side/rdb/all/RdbAllReqRow.java | 3 +- .../sql/side/rdb/async/RdbAsyncReqRow.java | 2 +- .../flink/sql/side/rdb/util/MathUtil.java | 29 +++-- .../flink/sql/side/rdb/util/SwitchUtil.java | 4 +- .../dtstack/flink/sql/sink/rdb/RdbSink.java | 7 +- .../sql/sink/rdb/table/RdbTableInfo.java | 2 - .../sink/sqlserver/SqlserverOutputFormat.java | 103 ----------------- .../sql/sink/sqlserver/SqlserverSink.java | 3 +- 10 files changed, 40 insertions(+), 226 deletions(-) delete mode 100644 oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleOutputFormat.java delete mode 100644 sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleOutputFormat.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleOutputFormat.java deleted file mode 100644 index dbbeea270..000000000 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleOutputFormat.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * 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 com.dtstack.flink.sql.sink.oracle; - -import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; - -import java.sql.ResultSet; -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -/** - * Reason: - * Date: 2018/11/30 - * Company: www.dtstack.com - * - * @author maqi - */ -public class OracleOutputFormat extends RetractJDBCOutputFormat { - - - @Override - public boolean isReplaceInsertQuery() throws SQLException { - fillRealIndexes(); - fillFullColumns(); - - if (!getRealIndexes().isEmpty()) { - for (List value : getRealIndexes().values()) { - for (String fieldName : getDbSink().getFieldNames()) { - if (value.contains(fieldName)) { - return true; - } - } - } - } - return false; - } - - /** - * get db all index - * - * @throws SQLException - */ - public void fillRealIndexes() throws SQLException { - Map> map = Maps.newHashMap(); - ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, null, getTableName(), true, false); - - while (rs.next()) { - String indexName = rs.getString("INDEX_NAME"); - if (!map.containsKey(indexName)) { - map.put(indexName, new ArrayList<>()); - } - String column_name = rs.getString("COLUMN_NAME"); - if (StringUtils.isNotBlank(column_name)) { - column_name = column_name.toUpperCase(); - } - map.get(indexName).add(column_name); - } - - for (Map.Entry> entry : map.entrySet()) { - String k = entry.getKey(); - List v = entry.getValue(); - if (v != null && v.size() != 0 && v.get(0) != null) { - getRealIndexes().put(k, v); - } - } - } - - /** - * get db all column name - * - * @throws SQLException - */ - public void fillFullColumns() throws SQLException { - String schema = null; - ResultSet rs = getDbConn().getMetaData().getColumns(null, schema, getTableName(), null); - while (rs.next()) { - String columnName = rs.getString("COLUMN_NAME"); - if (StringUtils.isNotBlank(columnName)) { - getFullField().add(columnName.toUpperCase()); - } - } - } - - -} diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index de1ffe8e5..4252f025c 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -19,6 +19,7 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.sink.rdb.format.ExtendOutputFormat; import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; import org.apache.commons.lang3.StringUtils; @@ -43,7 +44,7 @@ public String getDriverName() { @Override public RetractJDBCOutputFormat getOutputFormat() { - return new OracleOutputFormat(); + return new ExtendOutputFormat(); } @Override @@ -71,7 +72,7 @@ public String buildUpdateSql(String tableName, List fieldNames, Map column, List fullColumn, String String prefixRight = StringUtils.isBlank(rightTable) ? "" : quoteTable(rightTable) + "."; List list = new ArrayList<>(); for (String col : fullColumn) { - if (keyCols == null || keyCols.size() == 0) { + if (keyCols == null || keyCols.size() == 0 || keyCols.contains(col)) { continue; } if (fullColumn == null || column.contains(col)) { @@ -152,6 +153,7 @@ public String makeValues(List column) { } sb.append("? " + quoteColumn(column.get(i))); } + sb.append(" FROM DUAL"); return sb.toString(); } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java index c7e37c590..abae0cbb2 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java @@ -62,6 +62,7 @@ public abstract class RdbAllReqRow extends AllReqRow { private AtomicReference>>> cacheRef = new AtomicReference<>(); + public RdbAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(new RdbAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @@ -204,7 +205,7 @@ private void loadData(Map>> tmpCache) throws SQ Map oneRow = Maps.newHashMap(); for (String fieldName : sideFieldNames) { Object object = resultSet.getObject(fieldName.trim()); - int fieldIndex = sideInfo.getRowTypeInfo().getFieldIndex(fieldName.trim()); + int fieldIndex = sideInfo.getSideTableInfo().getFieldList().indexOf(fieldName.trim()); object = SwitchUtil.getTarget(object, fields[fieldIndex]); oneRow.put(fieldName.trim(), object); } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index 2c3d9b3f4..73fab0e9f 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -170,7 +170,7 @@ public Row fillData(Row input, Object line) { if (jsonArray == null) { row.setField(entry.getKey(), null); } else { - Object object = SwitchUtil.getTarget(jsonArray.getValue(entry.getValue()), fields[entry.getKey()]); + Object object = SwitchUtil.getTarget(jsonArray.getValue(entry.getValue()), fields[entry.getValue()]); row.setField(entry.getKey(), object); } } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java index aa7442586..fd6af5e0b 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java @@ -21,9 +21,10 @@ import java.math.BigDecimal; import java.math.BigInteger; +import java.sql.Date; +import java.sql.Timestamp; import java.text.ParseException; import java.text.SimpleDateFormat; -import java.util.Date; /** * Date: 2017/4/21 @@ -75,6 +76,8 @@ public static Integer getIntegerVal(Object obj) { return ((Double) obj).intValue(); } else if (obj instanceof BigDecimal) { return ((BigDecimal) obj).intValue(); + } else if (obj instanceof BigInteger) { + return ((BigInteger) obj).intValue(); } throw new RuntimeException("not support type of " + obj.getClass() + " convert to Integer."); @@ -219,18 +222,28 @@ public static Date getDate(Object obj) { if (obj == null) { return null; } - SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); if (obj instanceof String) { - try { - return sdf.parse((String) obj); - } catch (ParseException e) { - e.printStackTrace(); - } - } else if (obj instanceof Date) { + return Date.valueOf((String) obj); + } else if (obj instanceof Timestamp) { + return new Date(((Timestamp) obj).getTime()); + }else if (obj instanceof Date){ return (Date) obj; } throw new RuntimeException("not support type of " + obj.getClass() + " convert to Date."); } + public static Timestamp getTimestamp(Object obj) { + if (obj == null) { + return null; + } + if (obj instanceof Timestamp) { + return (Timestamp) obj; + } else if (obj instanceof Date) { + return Timestamp.valueOf((String) obj); + } else if (obj instanceof String) { + return Timestamp.valueOf((String) obj); + } + throw new RuntimeException("not support type of " + obj.getClass() + " convert to Date."); + } } \ No newline at end of file diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java index eadbdc476..1d06a4193 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java @@ -27,7 +27,7 @@ * @author maqi */ public class SwitchUtil { - public static Object getTarget(Object obj, String targetType) { + public static Object getTarget(Object obj, String targetType) { targetType = targetType.toLowerCase(); switch (targetType) { case "int": @@ -52,6 +52,8 @@ public static Object getTarget(Object obj, String targetType) { return MathUtil.getBigDecimal(obj); case "date": return MathUtil.getDate(obj); + case "timestamp": + return MathUtil.getTimestamp(obj); } return obj; } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java index a3611bfc6..5ee2c44c4 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java @@ -35,6 +35,7 @@ import java.io.Serializable; import java.math.BigDecimal; +import java.sql.Date; import java.sql.Timestamp; import java.sql.Types; import java.util.Arrays; @@ -149,7 +150,9 @@ protected void buildSqlTypes(List fieldTypeArray) { String fieldType = fieldTypeArray.get(i).getName(); if (fieldType.equals(Integer.class.getName())) { tmpFieldsType[i] = Types.INTEGER; - } else if (fieldType.equals(Long.class.getName())) { + }else if (fieldType.equals(Boolean.class.getName())) { + tmpFieldsType[i] = Types.BOOLEAN; + }else if (fieldType.equals(Long.class.getName())) { tmpFieldsType[i] = Types.BIGINT; } else if (fieldType.equals(Byte.class.getName())) { tmpFieldsType[i] = Types.TINYINT; @@ -167,6 +170,8 @@ protected void buildSqlTypes(List fieldTypeArray) { tmpFieldsType[i] = Types.TIMESTAMP; } else if (fieldType.equals(BigDecimal.class.getName())) { tmpFieldsType[i] = Types.DECIMAL; + } else if (fieldType.equals(Date.class.getName())) { + tmpFieldsType[i] = Types.DATE; } else { throw new RuntimeException("no support field type for sql. the input type:" + fieldType); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java index f170b88e1..9c2bbc9e1 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java @@ -29,8 +29,6 @@ */ public class RdbTableInfo extends TargetTableInfo { - //private static final String CURR_TYPE = "mysql"; - public static final String URL_KEY = "url"; public static final String TABLE_NAME_KEY = "tableName"; diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java deleted file mode 100644 index 7708ba163..000000000 --- a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverOutputFormat.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * 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 com.dtstack.flink.sql.sink.sqlserver; -import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -/** - * Reason: - * Date: 2018/11/30 - * Company: www.dtstack.com - * - * @author maqi - */ -public class SqlserverOutputFormat extends RetractJDBCOutputFormat { - - - @Override - public boolean isReplaceInsertQuery() throws SQLException { - fillRealIndexes(); - fillFullColumns(); - - if (!getRealIndexes().isEmpty()) { - for (List value : getRealIndexes().values()) { - for (String fieldName : getDbSink().getFieldNames()) { - if (value.contains(fieldName)) { - return true; - } - } - } - } - return false; - } - - /** - * get db all index - * - * @throws SQLException - */ - public void fillRealIndexes() throws SQLException { - Map> map = Maps.newHashMap(); - ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, null, getTableName(), true, false); - - while (rs.next()) { - String indexName = rs.getString("INDEX_NAME"); - if (!map.containsKey(indexName)) { - map.put(indexName, new ArrayList<>()); - } - String column_name = rs.getString("COLUMN_NAME"); - if (StringUtils.isNotBlank(column_name)) { - column_name = column_name.toUpperCase(); - } - map.get(indexName).add(column_name); - } - - for (Map.Entry> entry : map.entrySet()) { - String k = entry.getKey(); - List v = entry.getValue(); - if (v != null && v.size() != 0 && v.get(0) != null) { - getRealIndexes().put(k, v); - } - } - } - - /** - * get db all column name - * - * @throws SQLException - */ - public void fillFullColumns() throws SQLException { - String schema = null; - ResultSet rs = getDbConn().getMetaData().getColumns(null, schema, getTableName(), null); - while (rs.next()) { - String columnName = rs.getString("COLUMN_NAME"); - if (StringUtils.isNotBlank(columnName)) { - getFullField().add(columnName.toUpperCase()); - } - } - } - - -} diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java index 1e266e3bc..2bb53064e 100644 --- a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java @@ -19,6 +19,7 @@ import com.dtstack.flink.sql.sink.IStreamSinkGener; import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.sink.rdb.format.ExtendOutputFormat; import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; import org.apache.commons.lang3.StringUtils; @@ -41,7 +42,7 @@ public String getDriverName() { @Override public RetractJDBCOutputFormat getOutputFormat() { - return new SqlserverOutputFormat(); + return new ExtendOutputFormat(); } @Override From 2b7a5ead8124aef4bb4a431bfbe39cbb3f7c1a1e Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Tue, 4 Dec 2018 21:39:04 +0800 Subject: [PATCH 185/470] add extendOutputFormat --- .../sink/rdb/format/ExtendOutputFormat.java | 103 ++++++++++++++++++ 1 file changed, 103 insertions(+) create mode 100644 rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java new file mode 100644 index 000000000..3aa5e6f99 --- /dev/null +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -0,0 +1,103 @@ +/* + * 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 com.dtstack.flink.sql.sink.rdb.format; + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; + +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/30 + * Company: www.dtstack.com + * + * @author maqi + */ +public class ExtendOutputFormat extends RetractJDBCOutputFormat { + + + @Override + public boolean isReplaceInsertQuery() throws SQLException { + fillRealIndexes(); + fillFullColumns(); + + if (!getRealIndexes().isEmpty()) { + for (List value : getRealIndexes().values()) { + for (String fieldName : getDbSink().getFieldNames()) { + if (value.contains(fieldName)) { + return true; + } + } + } + } + return false; + } + + /** + * get db all index + * + * @throws SQLException + */ + public void fillRealIndexes() throws SQLException { + Map> map = Maps.newHashMap(); + ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, null, getTableName(), true, false); + + while (rs.next()) { + String indexName = rs.getString("INDEX_NAME"); + if (!map.containsKey(indexName)) { + map.put(indexName, new ArrayList<>()); + } + String column_name = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(column_name)) { + column_name = column_name.toUpperCase(); + } + map.get(indexName).add(column_name); + } + + for (Map.Entry> entry : map.entrySet()) { + String k = entry.getKey(); + List v = entry.getValue(); + if (v != null && v.size() != 0 && v.get(0) != null) { + getRealIndexes().put(k, v); + } + } + } + + /** + * get db all column name + * + * @throws SQLException + */ + public void fillFullColumns() throws SQLException { + ResultSet rs = getDbConn().getMetaData().getColumns(null, null, getTableName(), null); + while (rs.next()) { + String columnName = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(columnName)) { + getFullField().add(columnName.toUpperCase()); + } + } + } + + +} From 2a64d517881a6f8c49101ebf148d31ee1595d322 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Wed, 5 Dec 2018 11:39:05 +0800 Subject: [PATCH 186/470] ignore case --- .../flink/sql/sink/redis/table/RedisSinkParser.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java index 5754bdfee..c114b946e 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java @@ -33,15 +33,15 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Wed, 5 Dec 2018 15:41:49 +0800 Subject: [PATCH 187/470] modify MathUtil date parse --- .../dtstack/flink/sql/side/rdb/util/MathUtil.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java index fd6af5e0b..284d458b3 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java @@ -223,10 +223,15 @@ public static Date getDate(Object obj) { return null; } if (obj instanceof String) { - return Date.valueOf((String) obj); + SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd"); + try { + return new Date(format.parse((String) obj).getTime()); + } catch (ParseException e) { + throw new RuntimeException("String convert to Date fail."); + } } else if (obj instanceof Timestamp) { return new Date(((Timestamp) obj).getTime()); - }else if (obj instanceof Date){ + } else if (obj instanceof Date) { return (Date) obj; } throw new RuntimeException("not support type of " + obj.getClass() + " convert to Date."); @@ -239,9 +244,9 @@ public static Timestamp getTimestamp(Object obj) { if (obj instanceof Timestamp) { return (Timestamp) obj; } else if (obj instanceof Date) { - return Timestamp.valueOf((String) obj); + return new Timestamp(((Date) obj).getTime()); } else if (obj instanceof String) { - return Timestamp.valueOf((String) obj); + return new Timestamp(getDate(obj).getTime()); } throw new RuntimeException("not support type of " + obj.getClass() + " convert to Date."); } From 8412d882255a10fefded7ff6d10472ea0efef06d Mon Sep 17 00:00:00 2001 From: XuQianJin-Stars Date: Wed, 5 Dec 2018 18:52:20 +0800 Subject: [PATCH 188/470] add cassandra side and sink --- .../cassandra-side/cassandra-all-side/pom.xml | 88 ++++ .../side/cassandra/CassandraAllReqRow.java | 305 ++++++++++++++ .../side/cassandra/CassandraAllSideInfo.java | 113 ++++++ .../cassandra-async-side/pom.xml | 103 +++++ .../side/cassandra/CassandraAsyncReqRow.java | 309 ++++++++++++++ .../cassandra/CassandraAsyncSideInfo.java | 127 ++++++ .../cassandra-side-core/pom.xml | 24 ++ .../cassandra/table/CassandraSideParser.java | 99 +++++ .../table/CassandraSideTableInfo.java | 181 +++++++++ cassandra/cassandra-side/pom.xml | 23 ++ cassandra/cassandra-sink/pom.xml | 82 ++++ .../sink/cassandra/CassandraOutputFormat.java | 379 ++++++++++++++++++ .../sql/sink/cassandra/CassandraSink.java | 133 ++++++ .../cassandra/table/CassandraSinkParser.java | 84 ++++ .../cassandra/table/CassandraTableInfo.java | 161 ++++++++ .../test/java/com/dtstack/flinkx/AppTest.java | 58 +++ cassandra/pom.xml | 39 ++ docs/cassandraSide.md | 85 ++++ docs/cassandraSink.md | 63 +++ .../flink/sql/side/mongo/MongoAllReqRow.java | 1 - pom.xml | 1 + 21 files changed, 2457 insertions(+), 1 deletion(-) create mode 100644 cassandra/cassandra-side/cassandra-all-side/pom.xml create mode 100644 cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java create mode 100644 cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java create mode 100644 cassandra/cassandra-side/cassandra-async-side/pom.xml create mode 100644 cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java create mode 100644 cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java create mode 100644 cassandra/cassandra-side/cassandra-side-core/pom.xml create mode 100644 cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideParser.java create mode 100644 cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideTableInfo.java create mode 100644 cassandra/cassandra-side/pom.xml create mode 100644 cassandra/cassandra-sink/pom.xml create mode 100644 cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraOutputFormat.java create mode 100644 cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraSink.java create mode 100644 cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraSinkParser.java create mode 100644 cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java create mode 100644 cassandra/cassandra-sink/src/test/java/com/dtstack/flinkx/AppTest.java create mode 100644 cassandra/pom.xml create mode 100644 docs/cassandraSide.md create mode 100644 docs/cassandraSink.md diff --git a/cassandra/cassandra-side/cassandra-all-side/pom.xml b/cassandra/cassandra-side/cassandra-all-side/pom.xml new file mode 100644 index 000000000..74c62afdb --- /dev/null +++ b/cassandra/cassandra-side/cassandra-all-side/pom.xml @@ -0,0 +1,88 @@ + + + + sql.side.cassandra + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.all.cassandra + cassandra-all-side + + jar + + + + com.dtstack.flink + sql.side.cassandra.core + 1.0-SNAPSHOT + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java new file mode 100644 index 000000000..e8ccc739f --- /dev/null +++ b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java @@ -0,0 +1,305 @@ +/* + * 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 com.dtstack.flink.sql.side.cassandra; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ConsistencyLevel; +import com.datastax.driver.core.HostDistance; +import com.datastax.driver.core.PoolingOptions; +import com.datastax.driver.core.QueryOptions; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.SocketOptions; +import com.datastax.driver.core.policies.DowngradingConsistencyRetryPolicy; +import com.datastax.driver.core.policies.RetryPolicy; +import com.dtstack.flink.sql.side.AllReqRow; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo; +import org.apache.calcite.sql.JoinType; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.InetAddress; +import java.sql.SQLException; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Reason: + * Date: 2018/11/22 + * + * @author xuqianjin + */ +public class CassandraAllReqRow extends AllReqRow { + + private static final long serialVersionUID = 54015343561288219L; + + private static final Logger LOG = LoggerFactory.getLogger(CassandraAllReqRow.class); + + private static final String cassandra_DRIVER = "com.cassandra.jdbc.Driver"; + + private static final int CONN_RETRY_NUM = 3; + + private static final int FETCH_SIZE = 1000; + + private transient Cluster cluster; + private transient Session session = null; + + private AtomicReference>>> cacheRef = new AtomicReference<>(); + + public CassandraAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new com.dtstack.flink.sql.side.cassandra.CassandraAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + @Override + protected Row fillData(Row input, Object sideInput) { + Map cacheInfo = (Map) sideInput; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. + if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { + obj = ((Timestamp) obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for (Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()) { + if (cacheInfo == null) { + row.setField(entry.getKey(), null); + } else { + row.setField(entry.getKey(), cacheInfo.get(entry.getValue())); + } + } + + return row; + } + + @Override + protected void initCache() throws SQLException { + Map>> newCache = Maps.newConcurrentMap(); + cacheRef.set(newCache); + loadData(newCache); + } + + @Override + protected void reloadCache() { + //reload cacheRef and replace to old cacheRef + Map>> newCache = Maps.newConcurrentMap(); + try { + loadData(newCache); + } catch (SQLException e) { + LOG.error("", e); + } + + cacheRef.set(newCache); + LOG.info("----- cassandra all cacheRef reload end:{}", Calendar.getInstance()); + } + + + @Override + public void flatMap(Row value, Collector out) throws Exception { + List inputParams = Lists.newArrayList(); + for (Integer conValIndex : sideInfo.getEqualValIndex()) { + Object equalObj = value.getField(conValIndex); + if (equalObj == null) { + out.collect(null); + } + + inputParams.add(equalObj); + } + + String key = buildKey(inputParams); + List> cacheList = cacheRef.get().get(key); + if (CollectionUtils.isEmpty(cacheList)) { + if (sideInfo.getJoinType() == JoinType.LEFT) { + Row row = fillData(value, null); + out.collect(row); + } else { + return; + } + + return; + } + + for (Map one : cacheList) { + out.collect(fillData(value, one)); + } + + } + + private String buildKey(List equalValList) { + StringBuilder sb = new StringBuilder(""); + for (Object equalVal : equalValList) { + sb.append(equalVal).append("_"); + } + + return sb.toString(); + } + + private String buildKey(Map val, List equalFieldList) { + StringBuilder sb = new StringBuilder(""); + for (String equalField : equalFieldList) { + sb.append(val.get(equalField)).append("_"); + } + + return sb.toString(); + } + + private Session getConn(CassandraSideTableInfo tableInfo) { + try { + if (session == null) { + QueryOptions queryOptions = new QueryOptions(); + //The default consistency level for queries: ConsistencyLevel.TWO. + queryOptions.setConsistencyLevel(ConsistencyLevel.QUORUM); + Integer maxRequestsPerConnection = tableInfo.getMaxRequestsPerConnection() == null ? 1 : tableInfo.getMaxRequestsPerConnection(); + Integer coreConnectionsPerHost = tableInfo.getCoreConnectionsPerHost() == null ? 8 : tableInfo.getCoreConnectionsPerHost(); + Integer maxConnectionsPerHost = tableInfo.getMaxConnectionsPerHost() == null ? 32768 : tableInfo.getMaxConnectionsPerHost(); + Integer maxQueueSize = tableInfo.getMaxQueueSize() == null ? 100000 : tableInfo.getMaxQueueSize(); + Integer readTimeoutMillis = tableInfo.getReadTimeoutMillis() == null ? 60000 : tableInfo.getReadTimeoutMillis(); + Integer connectTimeoutMillis = tableInfo.getConnectTimeoutMillis() == null ? 60000 : tableInfo.getConnectTimeoutMillis(); + Integer poolTimeoutMillis = tableInfo.getPoolTimeoutMillis() == null ? 60000 : tableInfo.getPoolTimeoutMillis(); + Integer cassandraPort = 0; + String address = tableInfo.getAddress(); + String userName = tableInfo.getUserName(); + String password = tableInfo.getPassword(); + String database = tableInfo.getDatabase(); + + ArrayList serversList = new ArrayList(); + //Read timeout or connection timeout Settings + SocketOptions so = new SocketOptions() + .setReadTimeoutMillis(readTimeoutMillis) + .setConnectTimeoutMillis(connectTimeoutMillis); + + //The cluster USES hostdistance.local in the same machine room + //Hostdistance. REMOTE is used for different machine rooms + //Ignore use HostDistance. IGNORED + PoolingOptions poolingOptions = new PoolingOptions() + //Each connection allows a maximum of 64 concurrent requests + .setMaxRequestsPerConnection(HostDistance.LOCAL, maxRequestsPerConnection) + //Have at least two connections to each machine in the cluster + .setCoreConnectionsPerHost(HostDistance.LOCAL, coreConnectionsPerHost) + //There are up to eight connections to each machine in the cluster + .setMaxConnectionsPerHost(HostDistance.LOCAL, maxConnectionsPerHost) + .setMaxQueueSize(maxQueueSize) + .setPoolTimeoutMillis(poolTimeoutMillis); + //重试策略 + RetryPolicy retryPolicy = DowngradingConsistencyRetryPolicy.INSTANCE; + + for (String server : address.split(",")) { + cassandraPort = Integer.parseInt(server.split(":")[1]); + serversList.add(InetAddress.getByName(server.split(":")[0])); + } + + if (userName == null || userName.isEmpty() || password == null || password.isEmpty()) { + cluster = Cluster.builder().addContactPoints(serversList).withRetryPolicy(retryPolicy) + .withPort(cassandraPort) + .withPoolingOptions(poolingOptions).withSocketOptions(so) + .withQueryOptions(queryOptions).build(); + } else { + cluster = Cluster.builder().addContactPoints(serversList).withRetryPolicy(retryPolicy) + .withPort(cassandraPort) + .withPoolingOptions(poolingOptions).withSocketOptions(so) + .withCredentials(userName, password) + .withQueryOptions(queryOptions).build(); + } + // 建立连接 连接已存在的键空间 + session = cluster.connect(database); + LOG.info("connect cassandra is successed!"); + } + } catch (Exception e) { + LOG.error("connect cassandra is error:" + e.getMessage()); + } + return session; + } + + + private void loadData(Map>> tmpCache) throws SQLException { + CassandraSideTableInfo tableInfo = (CassandraSideTableInfo) sideInfo.getSideTableInfo(); + Session session = null; + + try { + for (int i = 0; i < CONN_RETRY_NUM; i++) { + try { + session = getConn(tableInfo); + break; + } catch (Exception e) { + if (i == CONN_RETRY_NUM - 1) { + throw new RuntimeException("", e); + } + try { + String connInfo = "address:" + tableInfo.getAddress() + ";userName:" + tableInfo.getUserName() + + ",pwd:" + tableInfo.getPassword(); + LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + connInfo); + Thread.sleep(5 * 1000); + } catch (InterruptedException e1) { + e1.printStackTrace(); + } + } + + } + + //load data from table + String sql = sideInfo.getSqlCondition() + " limit " + FETCH_SIZE; + ResultSet resultSet = session.execute(sql); + String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); + for (com.datastax.driver.core.Row row : resultSet) { + Map oneRow = Maps.newHashMap(); + for (String fieldName : sideFieldNames) { + oneRow.put(fieldName.trim(), row.getObject(fieldName.trim())); + } + String cacheKey = buildKey(oneRow, sideInfo.getEqualFieldList()); + List> list = tmpCache.computeIfAbsent(cacheKey, key -> Lists.newArrayList()); + list.add(oneRow); + } + } catch (Exception e) { + LOG.error("", e); + } finally { + try { + if (session != null) { + session.close(); + } + } catch (Exception e) { + LOG.error("Error while closing session.", e); + } + try { + if (cluster != null) { + cluster.close(); + } + } catch (Exception e) { + LOG.error("Error while closing cluster.", e); + } + } + } +} diff --git a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java new file mode 100644 index 000000000..bba39fc3e --- /dev/null +++ b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java @@ -0,0 +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 com.dtstack.flink.sql.side.cassandra; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.List; + +/** + * Reason: + * Date: 2018/11/22 + * + * @author xuqianjin + */ +public class CassandraAllSideInfo extends SideInfo { + + private static final long serialVersionUID = -8690814317653033557L; + + public CassandraAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + CassandraSideTableInfo cassandraSideTableInfo = (CassandraSideTableInfo) sideTableInfo; + + sqlCondition = "select ${selectField} from ${tableName} "; + sqlCondition = sqlCondition.replace("${tableName}", cassandraSideTableInfo.getDatabase() + "." + + cassandraSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); + System.out.println("---------side_exe_sql-----\n" + sqlCondition); + } + + @Override + public void parseSelectFields(JoinInfo joinInfo) { + String sideTableName = joinInfo.getSideTableName(); + String nonSideTableName = joinInfo.getNonSideTable(); + List fields = Lists.newArrayList(); + + int sideIndex = 0; + for (int i = 0; i < outFieldInfoList.size(); i++) { + FieldInfo fieldInfo = outFieldInfoList.get(i); + if (fieldInfo.getTable().equalsIgnoreCase(sideTableName)) { + fields.add(fieldInfo.getFieldName()); + sideFieldIndex.put(i, sideIndex); + sideFieldNameIndex.put(i, fieldInfo.getFieldName()); + sideIndex++; + } else if (fieldInfo.getTable().equalsIgnoreCase(nonSideTableName)) { + int nonSideIndex = rowTypeInfo.getFieldIndex(fieldInfo.getFieldName()); + inFieldIndex.put(i, nonSideIndex); + } else { + throw new RuntimeException("unknown table " + fieldInfo.getTable()); + } + } + + if (fields.size() == 0) { + throw new RuntimeException("select non field from table " + sideTableName); + } + + //add join on condition field to select fields + SqlNode conditionNode = joinInfo.getCondition(); + + List sqlNodeList = Lists.newArrayList(); + if (conditionNode.getKind() == SqlKind.AND) { + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall) conditionNode).getOperands())); + } else { + sqlNodeList.add(conditionNode); + } + + for (SqlNode sqlNode : sqlNodeList) { + dealOneEqualCon(sqlNode, sideTableName); + } + + if (CollectionUtils.isEmpty(equalFieldList)) { + throw new RuntimeException("no join condition found after table " + joinInfo.getLeftTableName()); + } + + for (String equalField : equalFieldList) { + if (fields.contains(equalField)) { + continue; + } + + fields.add(equalField); + } + + sideSelectFields = String.join(",", fields); + } +} diff --git a/cassandra/cassandra-side/cassandra-async-side/pom.xml b/cassandra/cassandra-side/cassandra-async-side/pom.xml new file mode 100644 index 000000000..cd709fecd --- /dev/null +++ b/cassandra/cassandra-side/cassandra-async-side/pom.xml @@ -0,0 +1,103 @@ + + + + sql.side.cassandra + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.async.cassandra + + cassandra-async-side + + jar + + + + + + io.vertx + vertx-jdbc-client + 3.5.2 + + + + io.vertx + vertx-core + 3.5.2 + + + + com.dtstack.flink + sql.side.cassandra.core + 1.0-SNAPSHOT + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java new file mode 100644 index 000000000..94c8e6fb6 --- /dev/null +++ b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java @@ -0,0 +1,309 @@ +/* + * 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 com.dtstack.flink.sql.side.cassandra; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ConsistencyLevel; +import com.datastax.driver.core.HostDistance; +import com.datastax.driver.core.PoolingOptions; +import com.datastax.driver.core.QueryOptions; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.SocketOptions; +import com.datastax.driver.core.policies.DowngradingConsistencyRetryPolicy; +import com.datastax.driver.core.policies.RetryPolicy; +import com.dtstack.flink.sql.enums.ECacheContentType; +import com.dtstack.flink.sql.side.AsyncReqRow; +import com.dtstack.flink.sql.side.CacheMissVal; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.cache.CacheObj; +import com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo; +import com.google.common.base.Function; +import com.google.common.util.concurrent.AsyncFunction; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import io.vertx.core.json.JsonArray; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.InetAddress; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Reason: + * Date: 2018/11/22 + * + * @author xuqianjin + */ +public class CassandraAsyncReqRow extends AsyncReqRow { + + private static final long serialVersionUID = 6631584128079864735L; + + private static final Logger LOG = LoggerFactory.getLogger(CassandraAsyncReqRow.class); + + private final static int DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE = 10; + + private final static int DEFAULT_VERTX_WORKER_POOL_SIZE = 20; + + private final static int DEFAULT_MAX_DB_CONN_POOL_SIZE = 20; + + private transient Cluster cluster; + private transient ListenableFuture session; + private transient CassandraSideTableInfo cassandraSideTableInfo; + + public CassandraAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new com.dtstack.flink.sql.side.cassandra.CassandraAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + cassandraSideTableInfo = (CassandraSideTableInfo) sideInfo.getSideTableInfo(); + connCassandraDB(cassandraSideTableInfo); + } + + private void connCassandraDB(CassandraSideTableInfo tableInfo) { + try { + if (session == null) { + QueryOptions queryOptions = new QueryOptions(); + //The default consistency level for queries: ConsistencyLevel.TWO. + queryOptions.setConsistencyLevel(ConsistencyLevel.QUORUM); + Integer maxRequestsPerConnection = tableInfo.getMaxRequestsPerConnection() == null ? 1 : tableInfo.getMaxRequestsPerConnection(); + Integer coreConnectionsPerHost = tableInfo.getCoreConnectionsPerHost() == null ? 8 : tableInfo.getCoreConnectionsPerHost(); + Integer maxConnectionsPerHost = tableInfo.getMaxConnectionsPerHost() == null ? 32768 : tableInfo.getMaxConnectionsPerHost(); + Integer maxQueueSize = tableInfo.getMaxQueueSize() == null ? 100000 : tableInfo.getMaxQueueSize(); + Integer readTimeoutMillis = tableInfo.getReadTimeoutMillis() == null ? 60000 : tableInfo.getReadTimeoutMillis(); + Integer connectTimeoutMillis = tableInfo.getConnectTimeoutMillis() == null ? 60000 : tableInfo.getConnectTimeoutMillis(); + Integer poolTimeoutMillis = tableInfo.getPoolTimeoutMillis() == null ? 60000 : tableInfo.getPoolTimeoutMillis(); + Integer cassandraPort = 0; + String address = tableInfo.getAddress(); + String userName = tableInfo.getUserName(); + String password = tableInfo.getPassword(); + String database = tableInfo.getDatabase(); + + ArrayList serversList = new ArrayList(); + //Read timeout or connection timeout Settings + SocketOptions so = new SocketOptions() + .setReadTimeoutMillis(readTimeoutMillis) + .setConnectTimeoutMillis(connectTimeoutMillis); + + //The cluster USES hostdistance.local in the same machine room + //Hostdistance. REMOTE is used for different machine rooms + //Ignore use HostDistance. IGNORED + PoolingOptions poolingOptions = new PoolingOptions() + //Each connection allows a maximum of 64 concurrent requests + .setMaxRequestsPerConnection(HostDistance.LOCAL, maxRequestsPerConnection) + //Have at least two connections to each machine in the cluster + .setCoreConnectionsPerHost(HostDistance.LOCAL, coreConnectionsPerHost) + //There are up to eight connections to each machine in the cluster + .setMaxConnectionsPerHost(HostDistance.LOCAL, maxConnectionsPerHost) + .setMaxQueueSize(maxQueueSize) + .setPoolTimeoutMillis(poolTimeoutMillis); + //重试策略 + RetryPolicy retryPolicy = DowngradingConsistencyRetryPolicy.INSTANCE; + + for (String server : address.split(",")) { + cassandraPort = Integer.parseInt(server.split(":")[1]); + serversList.add(InetAddress.getByName(server.split(":")[0])); + } + + if (userName == null || userName.isEmpty() || password == null || password.isEmpty()) { + cluster = Cluster.builder().addContactPoints(serversList).withRetryPolicy(retryPolicy) + .withPort(cassandraPort) + .withPoolingOptions(poolingOptions).withSocketOptions(so) + .withQueryOptions(queryOptions).build(); + } else { + cluster = Cluster.builder().addContactPoints(serversList).withRetryPolicy(retryPolicy) + .withPort(cassandraPort) + .withPoolingOptions(poolingOptions).withSocketOptions(so) + .withCredentials(userName, password) + .withQueryOptions(queryOptions).build(); + } + // 建立连接 连接已存在的键空间 + session = cluster.connectAsync(database); + LOG.info("connect cassandra is successed!"); + } + } catch (Exception e) { + LOG.error("connect cassandra is error:" + e.getMessage()); + } + } + + @Override + public void asyncInvoke(Row input, ResultFuture resultFuture) throws Exception { + + JsonArray inputParams = new JsonArray(); + StringBuffer stringBuffer = new StringBuffer(); + String sqlWhere = " where "; + + for (int i = 0; i < sideInfo.getEqualFieldList().size(); i++) { + Integer conValIndex = sideInfo.getEqualValIndex().get(i); + Object equalObj = input.getField(conValIndex); + if (equalObj == null) { + resultFuture.complete(null); + } + inputParams.add(equalObj); + stringBuffer.append(sideInfo.getEqualFieldList().get(i)) + .append(" = ").append("'" + equalObj + "'") + .append(" and "); + } + + String key = buildCacheKey(inputParams); + sqlWhere = sqlWhere + stringBuffer.toString().substring(0, stringBuffer.lastIndexOf(" and ")); + + if (openCache()) { + CacheObj val = getFromCache(key); + if (val != null) { + + if (ECacheContentType.MissVal == val.getType()) { + dealMissKey(input, resultFuture); + return; + } else if (ECacheContentType.MultiLine == val.getType()) { + + for (Object rowArray : (List) val.getContent()) { + Row row = fillData(input, rowArray); + resultFuture.complete(Collections.singleton(row)); + } + + } else { + throw new RuntimeException("not support cache obj type " + val.getType()); + } + return; + } + } + + //connect Cassandra + connCassandraDB(cassandraSideTableInfo); + + String sqlCondition = sideInfo.getSqlCondition() + " " + sqlWhere; + System.out.println("sqlCondition:" + sqlCondition); + + ListenableFuture resultSet = Futures.transformAsync(session, + new AsyncFunction() { + @Override + public ListenableFuture apply(Session session) throws Exception { + return session.executeAsync(sqlCondition); + } + }); + + ListenableFuture> data = Futures.transform(resultSet, + new Function>() { + @Override + public List apply(ResultSet rs) { + return rs.all(); + } + }); + + Futures.addCallback(data, new FutureCallback>() { + @Override + public void onSuccess(List rows) { + cluster.closeAsync(); + if (rows.size() > 0) { + List cacheContent = Lists.newArrayList(); + for (com.datastax.driver.core.Row line : rows) { + Row row = fillData(input, line); + if (openCache()) { + cacheContent.add(line); + } + resultFuture.complete(Collections.singleton(row)); + } + + if (openCache()) { + putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); + } + } else { + dealMissKey(input, resultFuture); + if (openCache()) { + putCache(key, CacheMissVal.getMissKeyObj()); + } + resultFuture.complete(null); + } + } + + @Override + public void onFailure(Throwable t) { + LOG.error("Failed to retrieve the data: %s%n", + t.getMessage()); + System.out.println("Failed to retrieve the data: " + t.getMessage()); + cluster.closeAsync(); + resultFuture.complete(null); + } + }); + } + + @Override + public Row fillData(Row input, Object line) { + com.datastax.driver.core.Row rowArray = (com.datastax.driver.core.Row) line; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { + obj = ((Timestamp) obj).getTime(); + } + + row.setField(entry.getKey(), obj); + } + + for (Map.Entry entry : sideInfo.getSideFieldIndex().entrySet()) { + if (rowArray == null) { + row.setField(entry.getKey(), null); + } else { + row.setField(entry.getKey(), rowArray.getObject(entry.getValue())); + } + } + + System.out.println("row:" + row.toString()); + return row; + } + + @Override + public void close() throws Exception { + super.close(); + if (cluster != null) { + cluster.close(); + cluster = null; + } + } + + public String buildCacheKey(JsonArray jsonArray) { + StringBuilder sb = new StringBuilder(); + for (Object ele : jsonArray.getList()) { + sb.append(ele.toString()) + .append("_"); + } + + return sb.toString(); + } +} diff --git a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java new file mode 100644 index 000000000..b1d239440 --- /dev/null +++ b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java @@ -0,0 +1,127 @@ +/* + * 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 com.dtstack.flink.sql.side.cassandra; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.List; + +/** + * Reason: + * Date: 2018/11/22 + * + * @author xuqianjin + */ +public class CassandraAsyncSideInfo extends SideInfo { + + private static final long serialVersionUID = -4403313049809013362L; + + public CassandraAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + CassandraSideTableInfo cassandraSideTableInfo = (CassandraSideTableInfo) sideTableInfo; + + String sideTableName = joinInfo.getSideTableName(); + + SqlNode conditionNode = joinInfo.getCondition(); + + List sqlNodeList = Lists.newArrayList(); + if (conditionNode.getKind() == SqlKind.AND) { + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall) conditionNode).getOperands())); + } else { + sqlNodeList.add(conditionNode); + } + + for (SqlNode sqlNode : sqlNodeList) { + dealOneEqualCon(sqlNode, sideTableName); + } + + sqlCondition = "select ${selectField} from ${tableName}"; + + sqlCondition = sqlCondition.replace("${tableName}", cassandraSideTableInfo.getDatabase()+"."+cassandraSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); + System.out.println("---------side_exe_sql-----\n" + sqlCondition); + } + + + @Override + public void dealOneEqualCon(SqlNode sqlNode, String sideTableName) { + if (sqlNode.getKind() != SqlKind.EQUALS) { + throw new RuntimeException("not equal operator."); + } + + SqlIdentifier left = (SqlIdentifier) ((SqlBasicCall) sqlNode).getOperands()[0]; + SqlIdentifier right = (SqlIdentifier) ((SqlBasicCall) sqlNode).getOperands()[1]; + + String leftTableName = left.getComponent(0).getSimple(); + String leftField = left.getComponent(1).getSimple(); + + String rightTableName = right.getComponent(0).getSimple(); + String rightField = right.getComponent(1).getSimple(); + + if (leftTableName.equalsIgnoreCase(sideTableName)) { + equalFieldList.add(leftField); + int equalFieldIndex = -1; + for (int i = 0; i < rowTypeInfo.getFieldNames().length; i++) { + String fieldName = rowTypeInfo.getFieldNames()[i]; + if (fieldName.equalsIgnoreCase(rightField)) { + equalFieldIndex = i; + } + } + if (equalFieldIndex == -1) { + throw new RuntimeException("can't deal equal field: " + sqlNode); + } + + equalValIndex.add(equalFieldIndex); + + } else if (rightTableName.equalsIgnoreCase(sideTableName)) { + + equalFieldList.add(rightField); + int equalFieldIndex = -1; + for (int i = 0; i < rowTypeInfo.getFieldNames().length; i++) { + String fieldName = rowTypeInfo.getFieldNames()[i]; + if (fieldName.equalsIgnoreCase(leftField)) { + equalFieldIndex = i; + } + } + if (equalFieldIndex == -1) { + throw new RuntimeException("can't deal equal field: " + sqlNode.toString()); + } + + equalValIndex.add(equalFieldIndex); + + } else { + throw new RuntimeException("resolve equalFieldList error:" + sqlNode.toString()); + } + + } + +} diff --git a/cassandra/cassandra-side/cassandra-side-core/pom.xml b/cassandra/cassandra-side/cassandra-side-core/pom.xml new file mode 100644 index 000000000..a3137b763 --- /dev/null +++ b/cassandra/cassandra-side/cassandra-side-core/pom.xml @@ -0,0 +1,24 @@ + + + + sql.side.cassandra + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.cassandra.core + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + jar + + \ No newline at end of file diff --git a/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideParser.java b/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideParser.java new file mode 100644 index 000000000..6403a225b --- /dev/null +++ b/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideParser.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package com.dtstack.flink.sql.side.cassandra.table; + +import com.dtstack.flink.sql.table.AbsSideTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; + +/** + * Reason: + * Date: 2018/11/22 + * + * @author xuqianjin + */ +public class CassandraSideParser extends AbsSideTableParser { + + private final static String SIDE_SIGN_KEY = "sideSignKey"; + + private final static Pattern SIDE_TABLE_SIGN = Pattern.compile("(?i)^PERIOD\\s+FOR\\s+SYSTEM_TIME$"); + + public static final String ADDRESS_KEY = "address"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String USER_NAME_KEY = "userName"; + + public static final String PASSWORD_KEY = "password"; + + public static final String DATABASE_KEY = "database"; + + public static final String MAX_REQUEST_PER_CONNECTION_KEY = "maxRequestsPerConnection"; + + public static final String CORE_CONNECTIONS_PER_HOST_KEY = "coreConnectionsPerHost"; + + public static final String MAX_CONNECTIONS_PER_HOST_KEY = "maxConnectionsPerHost"; + + public static final String MAX_QUEUE_SIZE_KEY = "maxQueueSize"; + + public static final String READ_TIMEOUT_MILLIS_KEY = "readTimeoutMillis"; + + public static final String CONNECT_TIMEOUT_MILLIS_KEY = "connectTimeoutMillis"; + + public static final String POOL_TIMEOUT_MILLIS_KEY = "poolTimeoutMillis"; + + static { + keyPatternMap.put(SIDE_SIGN_KEY, SIDE_TABLE_SIGN); + keyHandlerMap.put(SIDE_SIGN_KEY, CassandraSideParser::dealSideSign); + } + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo cassandraSideTableInfo = new com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo(); + cassandraSideTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, cassandraSideTableInfo); + parseCacheProp(cassandraSideTableInfo, props); + + cassandraSideTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase()))); + cassandraSideTableInfo.setAddress(MathUtil.getString(props.get(ADDRESS_KEY.toLowerCase()))); + cassandraSideTableInfo.setTableName(MathUtil.getString(props.get(TABLE_NAME_KEY.toLowerCase()))); + cassandraSideTableInfo.setDatabase(MathUtil.getString(props.get(DATABASE_KEY.toLowerCase()))); + cassandraSideTableInfo.setUserName(MathUtil.getString(props.get(USER_NAME_KEY.toLowerCase()))); + cassandraSideTableInfo.setPassword(MathUtil.getString(props.get(PASSWORD_KEY.toLowerCase()))); + cassandraSideTableInfo.setMaxRequestsPerConnection(MathUtil.getIntegerVal(props.get(MAX_REQUEST_PER_CONNECTION_KEY.toLowerCase()))); + cassandraSideTableInfo.setCoreConnectionsPerHost(MathUtil.getIntegerVal(props.get(CORE_CONNECTIONS_PER_HOST_KEY.toLowerCase()))); + cassandraSideTableInfo.setMaxConnectionsPerHost(MathUtil.getIntegerVal(props.get(MAX_CONNECTIONS_PER_HOST_KEY.toLowerCase()))); + cassandraSideTableInfo.setMaxQueueSize(MathUtil.getIntegerVal(props.get(MAX_QUEUE_SIZE_KEY.toLowerCase()))); + cassandraSideTableInfo.setReadTimeoutMillis(MathUtil.getIntegerVal(props.get(READ_TIMEOUT_MILLIS_KEY.toLowerCase()))); + cassandraSideTableInfo.setConnectTimeoutMillis(MathUtil.getIntegerVal(props.get(CONNECT_TIMEOUT_MILLIS_KEY.toLowerCase()))); + cassandraSideTableInfo.setPoolTimeoutMillis(MathUtil.getIntegerVal(props.get(POOL_TIMEOUT_MILLIS_KEY.toLowerCase()))); + + return cassandraSideTableInfo; + } + + private static void dealSideSign(Matcher matcher, TableInfo tableInfo) { + } +} diff --git a/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideTableInfo.java b/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideTableInfo.java new file mode 100644 index 000000000..b1b36f7e8 --- /dev/null +++ b/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideTableInfo.java @@ -0,0 +1,181 @@ +/* + * 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 com.dtstack.flink.sql.side.cassandra.table; + +import com.dtstack.flink.sql.side.SideTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * Reason: + * Date: 2018/11/22 + * + * @author xuqianjin + */ +public class CassandraSideTableInfo extends SideTableInfo { + + private static final long serialVersionUID = -5556431094535478915L; + + private static final String CURR_TYPE = "cassandra"; + + public static final String ADDRESS_KEY = "address"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String USER_NAME_KEY = "userName"; + + public static final String PASSWORD_KEY = "password"; + + public static final String DATABASE_KEY = "database"; + + public static final String MAX_REQUEST_PER_CONNECTION_KEY = "maxRequestsPerConnection"; + + public static final String CORE_CONNECTIONS_PER_HOST_KEY = "coreConnectionsPerHost"; + + public static final String MAX_CONNECTIONS_PER_HOST_KEY = "maxConnectionsPerHost"; + + public static final String MAX_QUEUE_SIZE_KEY = "maxQueueSize"; + + public static final String READ_TIMEOUT_MILLIS_KEY = "readTimeoutMillis"; + + public static final String CONNECT_TIMEOUT_MILLIS_KEY = "connectTimeoutMillis"; + + public static final String POOL_TIMEOUT_MILLIS_KEY = "poolTimeoutMillis"; + + private String address; + private String tableName; + private String userName; + private String password; + private String database; + private Integer maxRequestsPerConnection; + private Integer coreConnectionsPerHost; + private Integer maxConnectionsPerHost; + private Integer maxQueueSize; + private Integer readTimeoutMillis; + private Integer connectTimeoutMillis; + private Integer poolTimeoutMillis; + + public String getAddress() { + return address; + } + + public void setAddress(String address) { + this.address = address; + } + + public String getDatabase() { + return database; + } + + public void setDatabase(String database) { + this.database = database; + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } + + public Integer getMaxRequestsPerConnection() { + return maxRequestsPerConnection; + } + + public void setMaxRequestsPerConnection(Integer maxRequestsPerConnection) { + this.maxRequestsPerConnection = maxRequestsPerConnection; + } + + public Integer getCoreConnectionsPerHost() { + return coreConnectionsPerHost; + } + + public void setCoreConnectionsPerHost(Integer coreConnectionsPerHost) { + this.coreConnectionsPerHost = coreConnectionsPerHost; + } + + public Integer getMaxConnectionsPerHost() { + return maxConnectionsPerHost; + } + + public void setMaxConnectionsPerHost(Integer maxConnectionsPerHost) { + this.maxConnectionsPerHost = maxConnectionsPerHost; + } + + public Integer getMaxQueueSize() { + return maxQueueSize; + } + + public void setMaxQueueSize(Integer maxQueueSize) { + this.maxQueueSize = maxQueueSize; + } + + public Integer getReadTimeoutMillis() { + return readTimeoutMillis; + } + + public void setReadTimeoutMillis(Integer readTimeoutMillis) { + this.readTimeoutMillis = readTimeoutMillis; + } + + public Integer getConnectTimeoutMillis() { + return connectTimeoutMillis; + } + + public void setConnectTimeoutMillis(Integer connectTimeoutMillis) { + this.connectTimeoutMillis = connectTimeoutMillis; + } + + public Integer getPoolTimeoutMillis() { + return poolTimeoutMillis; + } + + public void setPoolTimeoutMillis(Integer poolTimeoutMillis) { + this.poolTimeoutMillis = poolTimeoutMillis; + } + + public CassandraSideTableInfo() { + setType(CURR_TYPE); + } + + @Override + public boolean check() { + Preconditions.checkNotNull(address, "Cassandra field of ADDRESS is required"); + Preconditions.checkNotNull(database, "Cassandra field of database is required"); + Preconditions.checkNotNull(tableName, "Cassandra field of tableName is required"); + return true; + } +} diff --git a/cassandra/cassandra-side/pom.xml b/cassandra/cassandra-side/pom.xml new file mode 100644 index 000000000..92d058900 --- /dev/null +++ b/cassandra/cassandra-side/pom.xml @@ -0,0 +1,23 @@ + + + + sql.cassandra + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.cassandra + cassandra-side + + cassandra-side-core + cassandra-async-side + cassandra-all-side + + + pom + + \ No newline at end of file diff --git a/cassandra/cassandra-sink/pom.xml b/cassandra/cassandra-sink/pom.xml new file mode 100644 index 000000000..4fb20c373 --- /dev/null +++ b/cassandra/cassandra-sink/pom.xml @@ -0,0 +1,82 @@ + + + sql.cassandra + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.cassandra + jar + + cassandra-sink + http://maven.apache.org + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + diff --git a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraOutputFormat.java b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraOutputFormat.java new file mode 100644 index 000000000..11fe24b81 --- /dev/null +++ b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraOutputFormat.java @@ -0,0 +1,379 @@ +/* + * 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. + */ + + + +/* + * 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 com.dtstack.flink.sql.sink.cassandra; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ConsistencyLevel; +import com.datastax.driver.core.HostDistance; +import com.datastax.driver.core.PoolingOptions; +import com.datastax.driver.core.QueryOptions; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.SocketOptions; +import com.datastax.driver.core.policies.DowngradingConsistencyRetryPolicy; +import com.datastax.driver.core.policies.RetryPolicy; +import com.dtstack.flink.sql.metric.MetricConstant; +import org.apache.flink.api.common.io.RichOutputFormat; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.net.InetAddress; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.util.ArrayList; + +/** + * OutputFormat to write tuples into a database. + * The OutputFormat has to be configured using the supplied OutputFormatBuilder. + * + * @see Tuple + * @see DriverManager + */ +public class CassandraOutputFormat extends RichOutputFormat { + private static final long serialVersionUID = -7994311331389155692L; + + private static final Logger LOG = LoggerFactory.getLogger(CassandraOutputFormat.class); + + private String address; + private String tableName; + private String userName; + private String password; + private String database; + private Integer maxRequestsPerConnection; + private Integer coreConnectionsPerHost; + private Integer maxConnectionsPerHost; + private Integer maxQueueSize; + private Integer readTimeoutMillis; + private Integer connectTimeoutMillis; + private Integer poolTimeoutMillis; + + protected String[] fieldNames; + TypeInformation[] fieldTypes; + + private int batchInterval = 5000; + + private Cluster cluster; + private Session session = null; + + private int batchCount = 0; + + private transient Counter outRecords; + + private transient Meter outRecordsRate; + + public CassandraOutputFormat() { + } + + @Override + public void configure(Configuration parameters) { + } + + /** + * Connects to the target database and initializes the prepared statement. + * + * @param taskNumber The number of the parallel instance. + * @throws IOException Thrown, if the output could not be opened due to an + * I/O problem. + */ + @Override + public void open(int taskNumber, int numTasks) throws IOException { + try { + if (session == null) { + QueryOptions queryOptions = new QueryOptions(); + //The default consistency level for queries: ConsistencyLevel.TWO. + queryOptions.setConsistencyLevel(ConsistencyLevel.QUORUM); + Integer maxRequestsPerConnection = this.maxRequestsPerConnection == null ? 1 : this.maxRequestsPerConnection; + Integer coreConnectionsPerHost = this.coreConnectionsPerHost == null ? 8 : this.coreConnectionsPerHost; + Integer maxConnectionsPerHost = this.maxConnectionsPerHost == null ? 32768 : this.maxConnectionsPerHost; + Integer maxQueueSize = this.maxQueueSize == null ? 100000 : this.maxQueueSize; + Integer readTimeoutMillis = this.readTimeoutMillis == null ? 60000 : this.readTimeoutMillis; + Integer connectTimeoutMillis = this.connectTimeoutMillis == null ? 60000 : this.connectTimeoutMillis; + Integer poolTimeoutMillis = this.poolTimeoutMillis == null ? 60000 : this.poolTimeoutMillis; + Integer cassandraPort = 0; + + ArrayList serversList = new ArrayList(); + //Read timeout or connection timeout Settings + SocketOptions so = new SocketOptions() + .setReadTimeoutMillis(readTimeoutMillis) + .setConnectTimeoutMillis(connectTimeoutMillis); + + //The cluster USES hostdistance.local in the same machine room + //Hostdistance. REMOTE is used for different machine rooms + //Ignore use HostDistance. IGNORED + PoolingOptions poolingOptions = new PoolingOptions() + //Each connection allows a maximum of 64 concurrent requests + .setMaxRequestsPerConnection(HostDistance.LOCAL, maxRequestsPerConnection) + //Have at least two connections to each machine in the cluster + .setCoreConnectionsPerHost(HostDistance.LOCAL, coreConnectionsPerHost) + //There are up to eight connections to each machine in the cluster + .setMaxConnectionsPerHost(HostDistance.LOCAL, maxConnectionsPerHost) + .setMaxQueueSize(maxQueueSize) + .setPoolTimeoutMillis(poolTimeoutMillis); + //重试策略 + RetryPolicy retryPolicy = DowngradingConsistencyRetryPolicy.INSTANCE; + + for (String server : address.split(",")) { + cassandraPort = Integer.parseInt(server.split(":")[1]); + serversList.add(InetAddress.getByName(server.split(":")[0])); + } + + if (userName == null || userName.isEmpty() || password == null || password.isEmpty()) { + cluster = Cluster.builder().addContactPoints(serversList).withRetryPolicy(retryPolicy) + .withPort(cassandraPort) + .withPoolingOptions(poolingOptions).withSocketOptions(so) + .withQueryOptions(queryOptions).build(); + } else { + cluster = Cluster.builder().addContactPoints(serversList).withRetryPolicy(retryPolicy) + .withPort(cassandraPort) + .withPoolingOptions(poolingOptions).withSocketOptions(so) + .withCredentials(userName, password) + .withQueryOptions(queryOptions).build(); + } + // 建立连接 连接已存在的键空间 + session = cluster.connect(database); + LOG.info("connect cassandra is successed!"); + initMetric(); + } + } catch (Exception e) { + LOG.error("connect cassandra is error:" + e.getMessage()); + } + } + + private void initMetric() { + outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); + } + + /** + * Adds a record to the prepared statement. + *

+ * When this method is called, the output format is guaranteed to be opened. + *

+ *

+ * WARNING: this may fail when no column types specified (because a best effort approach is attempted in order to + * insert a null value but it's not guaranteed that the JDBC driver handles PreparedStatement.setObject(pos, null)) + * + * @param tuple2 The records to add to the output. + * @throws IOException Thrown, if the records could not be added due to an I/O problem. + * @see PreparedStatement + */ + @Override + public void writeRecord(Tuple2 tuple2) throws IOException { + Tuple2 tupleTrans = tuple2; + Boolean retract = tupleTrans.getField(0); + Row row = tupleTrans.getField(1); + try { + if (retract) { + insertWrite(row); + outRecords.inc(); + } else { + //do nothing + } + } catch (Exception e) { + throw new IllegalArgumentException("writeRecord() failed", e); + } + } + + private void insertWrite(Row row) { + try { + String cql = buildSql(row); + if (cql != null) { + ResultSet resultSet = session.execute(cql); + resultSet.wasApplied(); + } + } catch (Exception e) { + LOG.error("[upsert] is error:" + e.getMessage()); + } + } + + private String buildSql(Row row) { + StringBuffer fields = new StringBuffer(); + StringBuffer values = new StringBuffer(); + for (int index = 0; index < row.getArity(); index++) { + if (row.getField(index) == null) { + } else { + fields.append(fieldNames[index] + ","); + values.append("'" + row.getField(index) + "'" + ","); + } + } + fields.deleteCharAt(fields.length() - 1); + values.deleteCharAt(values.length() - 1); + String cql = "INSERT INTO " + database + "." + tableName + " (" + fields.toString() + ") " + + " VALUES (" + values.toString() + ")"; + return cql; + } + + /** + * Executes prepared statement and closes all resources of this instance. + * + * @throws IOException Thrown, if the input could not be closed properly. + */ + @Override + public void close() throws IOException { + try { + if (session != null) { + session.close(); + } + } catch (Exception e) { + LOG.error("Error while closing session.", e); + } + try { + if (cluster != null) { + cluster.close(); + } + } catch (Exception e) { + LOG.error("Error while closing cluster.", e); + } + LOG.info("close cassandra is successed!"); + } + + public static CassandraFormatBuilder buildOutputFormat() { + return new CassandraFormatBuilder(); + } + + public static class CassandraFormatBuilder { + private final CassandraOutputFormat format; + + protected CassandraFormatBuilder() { + this.format = new CassandraOutputFormat(); + } + + public CassandraFormatBuilder setUsername(String username) { + format.userName = username; + return this; + } + + public CassandraFormatBuilder setPassword(String password) { + format.password = password; + return this; + } + + public CassandraFormatBuilder setAddress(String address) { + format.address = address; + return this; + } + + public CassandraFormatBuilder setTableName(String tableName) { + format.tableName = tableName; + return this; + } + + public CassandraFormatBuilder setDatabase(String database) { + format.database = database; + return this; + } + + public CassandraFormatBuilder setFieldNames(String[] fieldNames) { + format.fieldNames = fieldNames; + return this; + } + + public CassandraFormatBuilder setFieldTypes(TypeInformation[] fieldTypes) { + format.fieldTypes = fieldTypes; + return this; + } + + public CassandraFormatBuilder setMaxRequestsPerConnection(Integer maxRequestsPerConnection) { + format.maxRequestsPerConnection = maxRequestsPerConnection; + return this; + } + + public CassandraFormatBuilder setCoreConnectionsPerHost(Integer coreConnectionsPerHost) { + format.coreConnectionsPerHost = coreConnectionsPerHost; + return this; + } + + public CassandraFormatBuilder setMaxConnectionsPerHost(Integer maxConnectionsPerHost) { + format.maxConnectionsPerHost = maxConnectionsPerHost; + return this; + } + + public CassandraFormatBuilder setMaxQueueSize(Integer maxQueueSize) { + format.maxQueueSize = maxQueueSize; + return this; + } + + public CassandraFormatBuilder setReadTimeoutMillis(Integer readTimeoutMillis) { + format.readTimeoutMillis = readTimeoutMillis; + return this; + } + + public CassandraFormatBuilder setConnectTimeoutMillis(Integer connectTimeoutMillis) { + format.connectTimeoutMillis = connectTimeoutMillis; + return this; + } + + public CassandraFormatBuilder setPoolTimeoutMillis(Integer poolTimeoutMillis) { + format.poolTimeoutMillis = poolTimeoutMillis; + return this; + } + + /** + * Finalizes the configuration and checks validity. + * + * @return Configured RetractJDBCOutputFormat + */ + public CassandraOutputFormat finish() { + if (format.userName == null) { + LOG.info("Username was not supplied separately."); + } + if (format.password == null) { + LOG.info("Password was not supplied separately."); + } + if (format.address == null) { + throw new IllegalArgumentException("No address URL supplied."); + } + if (format.database == null) { + throw new IllegalArgumentException("No dababase suplied"); + } + if (format.tableName == null) { + throw new IllegalArgumentException("No tableName supplied"); + } + return format; + } + } +} diff --git a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraSink.java b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraSink.java new file mode 100644 index 000000000..eb7b23b53 --- /dev/null +++ b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/CassandraSink.java @@ -0,0 +1,133 @@ +/* + * 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 com.dtstack.flink.sql.sink.cassandra; + + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.cassandra.table.CassandraTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.table.sinks.RetractStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +/** + * Reason: + * Date: 2018/11/22 + * + * @author xuqianjin + */ +public class CassandraSink implements RetractStreamTableSink, IStreamSinkGener { + + + protected String[] fieldNames; + TypeInformation[] fieldTypes; + protected String address; + protected String tableName; + protected String userName; + protected String password; + protected String database; + protected Integer maxRequestsPerConnection; + protected Integer coreConnectionsPerHost; + protected Integer maxConnectionsPerHost; + protected Integer maxQueueSize; + protected Integer readTimeoutMillis; + protected Integer connectTimeoutMillis; + protected Integer poolTimeoutMillis; + + public CassandraSink() { + // TO DO NOTHING + } + + @Override + public CassandraSink genStreamSink(TargetTableInfo targetTableInfo) { + CassandraTableInfo cassandraTableInfo = (CassandraTableInfo) targetTableInfo; + this.address = cassandraTableInfo.getAddress(); + this.tableName = cassandraTableInfo.getTableName(); + this.userName = cassandraTableInfo.getUserName(); + this.password = cassandraTableInfo.getPassword(); + this.database = cassandraTableInfo.getDatabase(); + this.maxRequestsPerConnection = cassandraTableInfo.getMaxRequestsPerConnection(); + this.coreConnectionsPerHost = cassandraTableInfo.getCoreConnectionsPerHost(); + this.maxConnectionsPerHost = cassandraTableInfo.getMaxConnectionsPerHost(); + this.maxQueueSize = cassandraTableInfo.getMaxQueueSize(); + this.readTimeoutMillis = cassandraTableInfo.getReadTimeoutMillis(); + this.connectTimeoutMillis = cassandraTableInfo.getConnectTimeoutMillis(); + this.poolTimeoutMillis = cassandraTableInfo.getPoolTimeoutMillis(); + return this; + } + + @Override + public void emitDataStream(DataStream> dataStream) { + CassandraOutputFormat.CassandraFormatBuilder builder = CassandraOutputFormat.buildOutputFormat(); + builder.setAddress(this.address) + .setDatabase(this.database) + .setTableName(this.tableName) + .setPassword(this.password) + .setUsername(this.userName) + .setMaxRequestsPerConnection(this.maxRequestsPerConnection) + .setCoreConnectionsPerHost(this.coreConnectionsPerHost) + .setMaxConnectionsPerHost(this.maxConnectionsPerHost) + .setMaxQueueSize(this.maxQueueSize) + .setReadTimeoutMillis(this.readTimeoutMillis) + .setConnectTimeoutMillis(this.connectTimeoutMillis) + .setPoolTimeoutMillis(this.poolTimeoutMillis) + .setFieldNames(this.fieldNames) + .setFieldTypes(this.fieldTypes); + + CassandraOutputFormat outputFormat = builder.finish(); + RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(outputFormat); + dataStream.addSink(richSinkFunction); + } + + @Override + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + + @Override + public TupleTypeInfo> getOutputType() { + return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), getRecordType()); + } + + @Override + public TypeInformation getRecordType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + +} diff --git a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraSinkParser.java b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraSinkParser.java new file mode 100644 index 000000000..4c68e71ae --- /dev/null +++ b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraSinkParser.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 com.dtstack.flink.sql.sink.cassandra.table; + +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; + +/** + * Reason: + * Date: 2018/11/22 + * + * @author xuqianjin + */ +public class CassandraSinkParser extends AbsTableParser { + + public static final String ADDRESS_KEY = "address"; + + public static final String TABLE_NAME_KEY = "tableName"; + + public static final String USER_NAME_KEY = "userName"; + + public static final String PASSWORD_KEY = "password"; + + public static final String DATABASE_KEY = "database"; + + public static final String MAX_REQUEST_PER_CONNECTION_KEY = "maxRequestsPerConnection"; + + public static final String CORE_CONNECTIONS_PER_HOST_KEY = "coreConnectionsPerHost"; + + public static final String MAX_CONNECTIONS_PER_HOST_KEY = "maxConnectionsPerHost"; + + public static final String MAX_QUEUE_SIZE_KEY = "maxQueueSize"; + + public static final String READ_TIMEOUT_MILLIS_KEY = "readTimeoutMillis"; + + public static final String CONNECT_TIMEOUT_MILLIS_KEY = "connectTimeoutMillis"; + + public static final String POOL_TIMEOUT_MILLIS_KEY = "poolTimeoutMillis"; + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + CassandraTableInfo cassandraTableInfo = new CassandraTableInfo(); + cassandraTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, cassandraTableInfo); + + cassandraTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase()))); + cassandraTableInfo.setAddress(MathUtil.getString(props.get(ADDRESS_KEY.toLowerCase()))); + cassandraTableInfo.setTableName(MathUtil.getString(props.get(TABLE_NAME_KEY.toLowerCase()))); + cassandraTableInfo.setDatabase(MathUtil.getString(props.get(DATABASE_KEY.toLowerCase()))); + cassandraTableInfo.setUserName(MathUtil.getString(props.get(USER_NAME_KEY.toLowerCase()))); + cassandraTableInfo.setPassword(MathUtil.getString(props.get(PASSWORD_KEY.toLowerCase()))); + cassandraTableInfo.setMaxRequestsPerConnection(MathUtil.getIntegerVal(props.get(MAX_REQUEST_PER_CONNECTION_KEY.toLowerCase()))); + cassandraTableInfo.setCoreConnectionsPerHost(MathUtil.getIntegerVal(props.get(CORE_CONNECTIONS_PER_HOST_KEY.toLowerCase()))); + cassandraTableInfo.setMaxConnectionsPerHost(MathUtil.getIntegerVal(props.get(MAX_CONNECTIONS_PER_HOST_KEY.toLowerCase()))); + cassandraTableInfo.setMaxQueueSize(MathUtil.getIntegerVal(props.get(MAX_QUEUE_SIZE_KEY.toLowerCase()))); + cassandraTableInfo.setReadTimeoutMillis(MathUtil.getIntegerVal(props.get(READ_TIMEOUT_MILLIS_KEY.toLowerCase()))); + cassandraTableInfo.setConnectTimeoutMillis(MathUtil.getIntegerVal(props.get(CONNECT_TIMEOUT_MILLIS_KEY.toLowerCase()))); + cassandraTableInfo.setPoolTimeoutMillis(MathUtil.getIntegerVal(props.get(POOL_TIMEOUT_MILLIS_KEY.toLowerCase()))); + + return cassandraTableInfo; + } +} diff --git a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java new file mode 100644 index 000000000..7d52b23bb --- /dev/null +++ b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java @@ -0,0 +1,161 @@ +/* + * 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 com.dtstack.flink.sql.sink.cassandra.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * Reason: + * Date: 2018/11/22 + * + * @author xuqianjin + */ +public class CassandraTableInfo extends TargetTableInfo { + + private static final String CURR_TYPE = "cassandra"; + + private String address; + private String tableName; + private String userName; + private String password; + private String database; + private Integer maxRequestsPerConnection; + private Integer coreConnectionsPerHost; + private Integer maxConnectionsPerHost; + private Integer maxQueueSize; + private Integer readTimeoutMillis; + private Integer connectTimeoutMillis; + private Integer poolTimeoutMillis; + + public CassandraTableInfo() { + setType(CURR_TYPE); + } + + public String getAddress() { + return address; + } + + public void setAddress(String address) { + this.address = address; + } + + public String getDatabase() { + return database; + } + + public void setDatabase(String database) { + this.database = database; + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } + + public Integer getMaxRequestsPerConnection() { + return maxRequestsPerConnection; + } + + public void setMaxRequestsPerConnection(Integer maxRequestsPerConnection) { + this.maxRequestsPerConnection = maxRequestsPerConnection; + } + + public Integer getCoreConnectionsPerHost() { + return coreConnectionsPerHost; + } + + public void setCoreConnectionsPerHost(Integer coreConnectionsPerHost) { + this.coreConnectionsPerHost = coreConnectionsPerHost; + } + + public Integer getMaxConnectionsPerHost() { + return maxConnectionsPerHost; + } + + public void setMaxConnectionsPerHost(Integer maxConnectionsPerHost) { + this.maxConnectionsPerHost = maxConnectionsPerHost; + } + + public Integer getMaxQueueSize() { + return maxQueueSize; + } + + public void setMaxQueueSize(Integer maxQueueSize) { + this.maxQueueSize = maxQueueSize; + } + + public Integer getReadTimeoutMillis() { + return readTimeoutMillis; + } + + public void setReadTimeoutMillis(Integer readTimeoutMillis) { + this.readTimeoutMillis = readTimeoutMillis; + } + + public Integer getConnectTimeoutMillis() { + return connectTimeoutMillis; + } + + public void setConnectTimeoutMillis(Integer connectTimeoutMillis) { + this.connectTimeoutMillis = connectTimeoutMillis; + } + + public Integer getPoolTimeoutMillis() { + return poolTimeoutMillis; + } + + public void setPoolTimeoutMillis(Integer poolTimeoutMillis) { + this.poolTimeoutMillis = poolTimeoutMillis; + } + + @Override + public boolean check() { + Preconditions.checkNotNull(address, "Cassandra field of ADDRESS is required"); + Preconditions.checkNotNull(database, "Cassandra field of database is required"); + Preconditions.checkNotNull(tableName, "Cassandra field of tableName is required"); + return true; + } + + @Override + public String getType() { + // return super.getType().toLowerCase() + TARGET_SUFFIX; + return super.getType().toLowerCase(); + } +} diff --git a/cassandra/cassandra-sink/src/test/java/com/dtstack/flinkx/AppTest.java b/cassandra/cassandra-sink/src/test/java/com/dtstack/flinkx/AppTest.java new file mode 100644 index 000000000..33a0233ac --- /dev/null +++ b/cassandra/cassandra-sink/src/test/java/com/dtstack/flinkx/AppTest.java @@ -0,0 +1,58 @@ +/* + * 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 com.dtstack.flinkx; + +import junit.framework.Test; +import junit.framework.TestCase; +import junit.framework.TestSuite; + +/** + * Unit test for simple App. + */ +public class AppTest + extends TestCase +{ + /** + * Create the test case + * + * @param testName name of the test case + */ + public AppTest( String testName ) + { + super( testName ); + } + + /** + * @return the suite of tests being tested + */ + public static Test suite() + { + return new TestSuite( AppTest.class ); + } + + /** + * Rigourous Test :-) + */ + public void testApp() + { + assertTrue( true ); + } +} diff --git a/cassandra/pom.xml b/cassandra/pom.xml new file mode 100644 index 000000000..f49de388b --- /dev/null +++ b/cassandra/pom.xml @@ -0,0 +1,39 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + sql.cassandra + pom + + + cassandra-sink + cassandra-side + + + + + junit + junit + 3.8.1 + test + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + com.datastax.cassandra + cassandra-driver-core + 3.6.0 + + + + \ No newline at end of file diff --git a/docs/cassandraSide.md b/docs/cassandraSide.md new file mode 100644 index 000000000..131560047 --- /dev/null +++ b/docs/cassandraSide.md @@ -0,0 +1,85 @@ + +## 1.格式: +``` + CREATE TABLE tableName( + colName cloType, + ... + PRIMARY KEY(keyInfo), + PERIOD FOR SYSTEM_TIME + )WITH( + type ='cassandra', + address ='ip:port[,ip:port]', + userName='dbUserName', + password='dbPwd', + tableName='tableName', + database='database', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' + ); +``` + +# 2.支持版本 + cassandra-3.6.x + +## 3.表结构定义 + + |参数名称|含义| + |----|---| + | tableName | 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同)| + | colName | 列名称| + | colType | 列类型 [colType支持的类型](colType.md)| + | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| + | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| + +## 4.参数 + + |参数名称|含义|是否必填|默认值| + |----|---|---|----| + | type |表明 输出表类型 cassandra|是|| + | address | 连接cassandra数据库 jdbcUrl |是|| + | userName | cassandra连接用户名|否|| + | password | cassandra连接密码|否|| + | tableName | cassandra表名称|是|| + | database | cassandra表名称|是|| + | cache | 维表缓存策略(NONE/LRU)|否|NONE| + | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| + | maxRequestsPerConnection | 每个连接最多允许64个并发请求|否|NONE| + | coreConnectionsPerHost | 和Cassandra集群里的每个机器都至少有2个连接|否|NONE| + | maxConnectionsPerHost | 和Cassandra集群里的每个机器都最多有6个连接|否|NONE| + | maxQueueSize | Cassandra队列大小|否|NONE| + | readTimeoutMillis | Cassandra读超时|否|NONE| + | connectTimeoutMillis | Cassandra连接超时|否|NONE| + | poolTimeoutMillis | Cassandra线程池超时|否|NONE| + + ---------- + > 缓存策略 + * NONE: 不做内存缓存 + * LRU: + * cacheSize: 缓存的条目数量 + * cacheTTLMs:缓存的过期时间(ms) + + +## 5.样例 +``` +create table sideTable( + CHANNEL varchar, + XCCOUNT int, + PRIMARY KEY(channel), + PERIOD FOR SYSTEM_TIME + )WITH( + type ='cassandra', + address ='172.21.32.1:9042,172.21.32.1:9042', + database ='test', + tableName ='sidetest', + cache ='LRU', + parallelism ='1', + partitionedJoin='false' + ); + + +``` + + diff --git a/docs/cassandraSink.md b/docs/cassandraSink.md new file mode 100644 index 000000000..8ea38e104 --- /dev/null +++ b/docs/cassandraSink.md @@ -0,0 +1,63 @@ +## 1.格式: +``` +CREATE TABLE tableName( + colName colType, + ... + colNameX colType + )WITH( + type ='cassandra', + address ='ip:port[,ip:port]', + userName ='userName', + password ='pwd', + database ='databaseName', + tableName ='tableName', + parallelism ='parllNum' + ); + +``` + +## 2.支持版本 + cassandra-3.6.x + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName| 在 sql 中使用的名称;即注册到flink-table-env上的名称| +| colName | 列名称| +| colType | 列类型 [colType支持的类型](colType.md)| + +## 4.参数: + +|参数名称|含义|是否必填|默认值| +|----|----|----|----| +|type |表明 输出表类型 cassandra|是|| +|address | 连接cassandra数据库 jdbcUrl |是|| +|userName | cassandra连接用户名|否|| +|password | cassandra连接密码|否|| +|tableName | cassandra表名称|是|| +|database | cassandra表名称|是|| +|parallelism | 并行度设置|否|1| +|maxRequestsPerConnection | 每个连接最多允许64个并发请求|否|NONE| +|coreConnectionsPerHost | 和Cassandra集群里的每个机器都至少有2个连接|否|NONE| +|maxConnectionsPerHost | 和Cassandra集群里的每个机器都最多有6个连接|否|NONE| +|maxQueueSize | Cassandra队列大小|否|NONE| +|readTimeoutMillis | Cassandra读超时|否|NONE| +|connectTimeoutMillis | Cassandra连接超时|否|NONE| +|poolTimeoutMillis | Cassandra线程池超时|否|NONE| + +## 5.样例: +``` +CREATE TABLE MyResult( + channel VARCHAR, + pv VARCHAR + )WITH( + type ='cassandra', + address ='172.21.32.1:9042,172.21.32.1:9042', + userName ='dtstack', + password ='abc123', + database ='test', + tableName ='pv', + parallelism ='1' + ) + ``` \ No newline at end of file diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java index e1a398d4c..cb142944d 100644 --- a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java @@ -152,7 +152,6 @@ public void flatMap(Row value, Collector out) throws Exception { } for (Map one : cacheList) { - System.out.println(fillData(value, one)); out.collect(fillData(value, one)); } } diff --git a/pom.xml b/pom.xml index 07694589c..837ade4b2 100644 --- a/pom.xml +++ b/pom.xml @@ -16,6 +16,7 @@ mongo redis5 launcher + cassandra pom From 59701dc7f7cc3b32fc531f1169be5db654419381 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 5 Dec 2018 20:50:10 +0800 Subject: [PATCH 189/470] bugfix --- .../dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java index e8ccc739f..3a7e56902 100644 --- a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java +++ b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java @@ -81,7 +81,7 @@ public CassandraAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List cacheInfo = (Map) sideInput; Row row = new Row(sideInfo.getOutFieldInfoList().size()); for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { From 9003e2b5ee28a33949a1e1a2cf31513181e7894a Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 5 Dec 2018 20:53:51 +0800 Subject: [PATCH 190/470] bugfix --- .../dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java index e8ccc739f..3a7e56902 100644 --- a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java +++ b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java @@ -81,7 +81,7 @@ public CassandraAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List cacheInfo = (Map) sideInput; Row row = new Row(sideInfo.getOutFieldInfoList().size()); for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { From d17d235793e3bf0decd3e98f9976c4415f0f81fe Mon Sep 17 00:00:00 2001 From: XuQianJin-Stars Date: Thu, 6 Dec 2018 09:54:18 +0800 Subject: [PATCH 191/470] add cassandra md --- README.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 27dffc8dc..f7c86bfab 100644 --- a/README.md +++ b/README.md @@ -8,8 +8,8 @@ # 已支持 * 源表:kafka 0.9,1.x版本 - * 维表:mysql,hbase,mongo,redis - * 结果表:mysql,hbase,elasticsearch5.x,mongo,redis + * 维表:mysql,hbase,mongo,redis,cassandra + * 结果表:mysql,hbase,elasticsearch5.x,mongo,redis,cassandra # 后续开发计划 * 增加oracle维表,结果表功能 @@ -149,12 +149,14 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * [mysql 结果表插件](docs/mysqlSink.md) * [mongo 结果表插件](docs/mongoSink.md) * [redis 结果表插件](docs/redisSink.md) +* [cassandra 结果表插件](docs/cassandraSink.md) ### 2.3 维表插件 * [hbase 维表插件](docs/hbaseSide.md) * [mysql 维表插件](docs/mysqlSide.md) * [mongo 维表插件](docs/mongoSide.md) * [redis 维表插件](docs/redisSide.md) +* [cassandra 维表插件](docs/cassandraSide.md) ## 3 性能指标(新增) From d5f8b6792ad79eb4d61a87698805c274f87b326f Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 6 Dec 2018 10:24:58 +0800 Subject: [PATCH 192/470] Update README.md --- README.md | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index fc9f278c6..ad14222ca 100644 --- a/README.md +++ b/README.md @@ -10,12 +10,10 @@ # 已支持 * 源表:kafka 0.9,1.x版本 - * 维表:mysql,hbase,mongo,redis,cassandra - * 结果表:mysql,hbase,elasticsearch5.x,mongo,redis,cassandra + * 维表:mysql,SQlServer,oracle,hbase,mongo,redis,cassandra + * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra # 后续开发计划 - * 增加oracle维表,结果表功能 - * 增加SQlServer维表,结果表功能 * 增加kafka结果表功能 * 增加SQL支持CEP * 维表快照 From 416009689a4d8970f697e042bed00f642f6aa964 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 6 Dec 2018 14:22:29 +0800 Subject: [PATCH 193/470] fix flink-1.6 local-mode run bug --- .../dtstack/flink/sql/MyLocalStreamEnvironment.java | 4 ++-- .../flink/sql/launcher/LauncherOptionParser.java | 13 +++++++++---- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java b/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java index f41ecf00b..acd28e3f3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java +++ b/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java @@ -100,8 +100,8 @@ public JobExecutionResult execute(String jobName) throws Exception { Configuration configuration = new Configuration(); configuration.addAll(jobGraph.getJobConfiguration()); - configuration.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "-1L"); - configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, jobGraph.getMaximumParallelism()); + configuration.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "512M"); + configuration.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, jobGraph.getMaximumParallelism()); // add (and override) the settings with what the user defined configuration.addAll(this.conf); diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java index 75c5c4f0f..a801b193e 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java @@ -101,21 +101,25 @@ public LauncherOptionParser(String[] args) { byte[] filecontent = new byte[(int) file.length()]; in.read(filecontent); String content = new String(filecontent, "UTF-8"); + String sql = URLEncoder.encode(content, Charsets.UTF_8.name()); properties.setSql(sql); + String localPlugin = Preconditions.checkNotNull(cl.getOptionValue(OPTION_LOCAL_SQL_PLUGIN_PATH)); properties.setLocalSqlPluginPath(localPlugin); + String remotePlugin = cl.getOptionValue(OPTION_REMOTE_SQL_PLUGIN_PATH); - if(!ClusterMode.local.name().equals(mode)){ - Preconditions.checkNotNull(remotePlugin); - properties.setRemoteSqlPluginPath(remotePlugin); - } + Preconditions.checkNotNull(remotePlugin); + properties.setRemoteSqlPluginPath(remotePlugin); + String name = Preconditions.checkNotNull(cl.getOptionValue(OPTION_NAME)); properties.setName(name); + String addJar = cl.getOptionValue(OPTION_ADDJAR); if(StringUtils.isNotBlank(addJar)){ properties.setAddjar(addJar); } + String confProp = cl.getOptionValue(OPTION_CONF_PROP); if(StringUtils.isNotBlank(confProp)){ properties.setConfProp(confProp); @@ -157,6 +161,7 @@ public LauncherOptions getLauncherOptions(){ public List getProgramExeArgList() throws Exception { Map mapConf = PluginUtil.ObjectToMap(properties); List args = Lists.newArrayList(); + for(Map.Entry one : mapConf.entrySet()){ String key = one.getKey(); if(OPTION_FLINK_CONF_DIR.equalsIgnoreCase(key) From c15edc99be9c7c239b7b3fddc1a837665427d8ce Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 10 Dec 2018 11:36:58 +0800 Subject: [PATCH 194/470] password is null --- .../java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java | 2 ++ redis5/redis5-sink/pom.xml | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index 3f9f40f24..1a6aa0425 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -80,6 +80,8 @@ private void buildRedisClient(RedisSideTableInfo tableInfo){ String password = redisSideTableInfo.getPassword(); if (password != null){ password = password + "@"; + } else { + password = ""; } String database = redisSideTableInfo.getDatabase(); if (database == null){ diff --git a/redis5/redis5-sink/pom.xml b/redis5/redis5-sink/pom.xml index 994c3972a..2919d8685 100644 --- a/redis5/redis5-sink/pom.xml +++ b/redis5/redis5-sink/pom.xml @@ -26,7 +26,7 @@ redis.clients jedis - 2.8.0 + 2.9.0 From 66813312e0c9fe8f459ee0dda23a904f26c986b7 Mon Sep 17 00:00:00 2001 From: Hongtao Zhang <553780043@qq.com> Date: Mon, 10 Dec 2018 16:51:06 +0800 Subject: [PATCH 195/470] fix kafka offset reset option --- .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 ++ .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 ++ .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 ++ .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 ++ .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 ++ .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 ++ 6 files changed, 12 insertions(+) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 898083826..606d42c9d 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -46,6 +46,8 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Mon, 10 Dec 2018 17:05:34 +0800 Subject: [PATCH 196/470] fix bug for kafka offset --- .../flink/sql/source/kafka/table/KafkaSourceParser.java | 3 ++- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 ++ .../flink/sql/source/kafka/table/KafkaSourceParser.java | 1 + .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 3 ++- .../flink/sql/source/kafka/table/KafkaSourceParser.java | 1 + .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 ++ 6 files changed, 10 insertions(+), 2 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 898083826..301b0b423 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -16,7 +16,7 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.source.kafka.table; @@ -46,6 +46,7 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Mon, 10 Dec 2018 18:12:36 +0800 Subject: [PATCH 197/470] =?UTF-8?q?fix=20up=20=E7=94=A8=E6=88=B7=E6=A0=87?= =?UTF-8?q?=E8=AF=86=E7=AC=A6SqlParse=E5=BC=BA=E5=88=B6=E4=B8=BA=E5=A4=A7?= =?UTF-8?q?=E5=86=99=EF=BC=8C=E5=BA=94=E8=AF=A5=E4=BF=9D=E6=8C=81=E7=94=A8?= =?UTF-8?q?=E6=88=B7=E8=BE=93=E5=85=A5=E4=B8=8D=E5=8F=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/src/main/java/com/dtstack/flink/sql/Main.java | 9 +++++++-- .../dtstack/flink/sql/parser/CreateTableParser.java | 2 +- .../flink/sql/parser/CreateTmpTableParser.java | 12 +++++++++--- .../dtstack/flink/sql/parser/InsertSqlParser.java | 7 ++++++- .../com/dtstack/flink/sql/side/SideSQLParser.java | 8 ++++++-- .../java/com/dtstack/flink/sql/side/SideSqlExec.java | 2 +- .../com/dtstack/flink/sql/table/AbsTableParser.java | 3 --- 7 files changed, 30 insertions(+), 13 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 610abf21f..2e9c979d1 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -34,6 +34,7 @@ import com.dtstack.flink.sql.watermarker.WaterMarkerAssigner; import com.dtstack.flink.sql.util.FlinkUtil; import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.calcite.config.Lex; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlNode; import org.apache.commons.cli.CommandLine; @@ -185,7 +186,11 @@ public static void main(String[] args) throws Exception { if (sqlTree.getTmpTableMap().containsKey(tableName)) { CreateTmpTableParser.SqlParserResult tmp = sqlTree.getTmpTableMap().get(tableName); String realSql = DtStringUtil.replaceIgnoreQuota(result.getExecSql(), "`", ""); - SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql).parseStmt(); + org.apache.calcite.sql.parser.SqlParser.Config config = org.apache.calcite.sql.parser.SqlParser + .configBuilder() + .setLex(Lex.JAVA) + .build(); + SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql,config).parseStmt(); String tmpSql = ((SqlInsert) sqlNode).getSource().toString(); tmp.setExecSql(tmpSql); sideSqlExec.registerTmpTable(tmp, sideTableMap, tableEnv, registerTableCache); @@ -246,7 +251,7 @@ private static void registerUDF(SqlTree sqlTree, List jarURList, URLClassLo classLoader = FlinkUtil.loadExtraJar(jarURList, parentClassloader); } classLoader.loadClass(funcInfo.getClassName()); - FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName().toUpperCase(), + FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName(), tableEnv, classLoader); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java index d10d6825e..b5e4a4aa4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java @@ -54,7 +54,7 @@ public boolean verify(String sql) { public void parseSql(String sql, SqlTree sqlTree) { Matcher matcher = PATTERN.matcher(sql); if(matcher.find()){ - String tableName = matcher.group(1).toUpperCase(); + String tableName = matcher.group(1); String fieldsInfoStr = matcher.group(2); String propsStr = matcher.group(3); Map props = parseProp(propsStr); diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java index 8da89b2be..d205c59b6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.parser; import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.calcite.config.Lex; import org.apache.calcite.sql.*; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; @@ -62,11 +63,16 @@ public void parseSql(String sql, SqlTree sqlTree) { String tableName = null; String selectSql = null; if(matcher.find()) { - tableName = matcher.group(1).toUpperCase(); + tableName = matcher.group(1); selectSql = "select " + matcher.group(2); } - SqlParser sqlParser = SqlParser.create(selectSql); + SqlParser.Config config = SqlParser + .configBuilder() + .setLex(Lex.JAVA) + .build(); + SqlParser sqlParser = SqlParser.create(selectSql,config); + SqlNode sqlNode = null; try { sqlNode = sqlParser.parseStmt(); @@ -89,7 +95,7 @@ public void parseSql(String sql, SqlTree sqlTree) { String tableName = null; String fieldsInfoStr = null; if (matcher.find()){ - tableName = matcher.group(1).toUpperCase(); + tableName = matcher.group(1); fieldsInfoStr = matcher.group(2); } CreateTmpTableParser.SqlParserResult sqlParseResult = new CreateTmpTableParser.SqlParserResult(); diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index e08540c92..7d5c61e7a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.parser; +import org.apache.calcite.config.Lex; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlJoin; @@ -57,7 +58,11 @@ public static InsertSqlParser newInstance(){ @Override public void parseSql(String sql, SqlTree sqlTree) { - SqlParser sqlParser = SqlParser.create(sql); + SqlParser.Config config = SqlParser + .configBuilder() + .setLex(Lex.JAVA) + .build(); + SqlParser sqlParser = SqlParser.create(sql,config); SqlNode sqlNode = null; try { sqlNode = sqlParser.parseStmt(); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 7f165ac30..472146c11 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.side; import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.calcite.config.Lex; import org.apache.calcite.sql.JoinType; import org.apache.calcite.sql.SqlAsOperator; import org.apache.calcite.sql.SqlBasicCall; @@ -52,11 +53,14 @@ public class SideSQLParser { public Queue getExeQueue(String exeSql, Set sideTableSet) throws SqlParseException { - exeSql = DtStringUtil.replaceIgnoreQuota(exeSql, "`", ""); System.out.println("---exeSql---"); System.out.println(exeSql); Queue queueInfo = Queues.newLinkedBlockingQueue(); - SqlParser sqlParser = SqlParser.create(exeSql); + SqlParser.Config config = SqlParser + .configBuilder() + .setLex(Lex.JAVA) + .build(); + SqlParser sqlParser = SqlParser.create(exeSql,config); SqlNode sqlNode = sqlParser.parseStmt(); parseSql(sqlNode, sideTableSet, queueInfo); queueInfo.offer(sqlNode); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 080d3d48b..120bd4ee6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -642,7 +642,7 @@ private boolean checkFieldsInfo(CreateTmpTableParser.SqlParserResult result, Tab String[] filedNameArr = new String[filed.length - 1]; System.arraycopy(filed, 0, filedNameArr, 0, filed.length - 1); String fieldName = String.join(" ", filedNameArr); - fieldNames.add(fieldName.toUpperCase()); + fieldNames.add(fieldName); String fieldType = filed[filed.length - 1 ].trim(); Class fieldClass = ClassUtil.stringConvertClass(fieldType); Class tableField = table.getSchema().getType(i).get().getTypeClass(); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index 2fc2d799b..c3ccdd789 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -82,9 +82,6 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ String[] fieldRows = DtStringUtil.splitIgnoreQuotaBrackets(fieldsInfo, ","); for(String fieldRow : fieldRows){ fieldRow = fieldRow.trim(); - if(fieldNameNeedsUpperCase()) { - fieldRow = fieldRow.toUpperCase(); - } boolean isMatcherKey = dealKeyPattern(fieldRow, tableInfo); From 3f08046e7a3dd5975649bd32576f0fc696a94cf9 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 10 Dec 2018 20:36:55 +0800 Subject: [PATCH 198/470] add default database --- .../com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java index 910882c9f..c2a33255d 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java @@ -108,6 +108,10 @@ private void establishConnection() { if (timeout == 0){ timeout = 10000; } + if (database == null) + { + database = "0"; + } switch (redisType){ //单机 @@ -134,7 +138,7 @@ public void writeRecord(Tuple2 record) throws IOException { return; } Row row = tupleTrans.getField(1); - if (record.getArity() != fieldNames.length) { + if (row.getArity() != fieldNames.length) { return; } From 8ec6d1d4a35d8e21e89b058c4ce28a52cbd38573 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Tue, 11 Dec 2018 09:34:06 +0800 Subject: [PATCH 199/470] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index ad14222ca..5dfa80efb 100644 --- a/README.md +++ b/README.md @@ -188,7 +188,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack ``` -CREATE (scala|table) FUNCTION CHARACTER_LENGTH WITH com.dtstack.Kun +CREATE (scala|table) FUNCTION CHARACTER_LENGTH WITH com.dtstack.Kun; CREATE TABLE MyTable( From 7e5b70643f350523f7daa1c7ccd705dba4a728d6 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 11 Dec 2018 14:43:53 +0800 Subject: [PATCH 200/470] add pattern match for kafka topic and fix groupId invalid, check method not called bug --- docs/kafkaSource.md | 7 +- .../source/kafka/CustomerKafka09Consumer.java | 8 +- .../flink/sql/source/kafka/KafkaSource.java | 14 +- .../source/kafka/table/KafkaSourceParser.java | 2 + .../kafka/table/KafkaSourceTableInfo.java | 138 +++++++++-------- .../kafka/CustomerKafka010Consumer.java | 8 +- .../flink/sql/source/kafka/KafkaSource.java | 102 +++++++------ .../source/kafka/table/KafkaSourceParser.java | 2 + .../kafka/table/KafkaSourceTableInfo.java | 15 ++ .../kafka/CustomerKafka011Consumer.java | 10 +- .../flink/sql/source/kafka/KafkaSource.java | 113 +++++++------- .../source/kafka/table/KafkaSourceParser.java | 2 + .../kafka/table/KafkaSourceTableInfo.java | 139 ++++++++++-------- .../sql/side/rdb/table/RdbSideParser.java | 2 +- .../sql/sink/rdb/table/RdbSinkParser.java | 1 + 15 files changed, 337 insertions(+), 226 deletions(-) diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index f382ba9ad..f1518aaec 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -38,6 +38,8 @@ CREATE TABLE tableName( |bootstrapServers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| |zookeeperQuorum | kafka zk地址信息(多个之间用逗号分隔)|是|| |topic | 需要读取的 topic 名称|是|| +|topicIsPattern | topic是否是正则表达式格式|否| false +|groupId | 需要读取的 groupId 名称|是|| |offsetReset | 读取的topic 的offset初始位置[latest\|earliest\|指定offset值({"0":12312,"1":12321,"2":12312},{"partition_no":offset_value})]|否|latest| |parallelism | 并行度设置|否|1| @@ -54,7 +56,10 @@ CREATE TABLE MyTable( bootstrapServers ='172.16.8.198:9092', zookeeperQuorum ='172.16.8.198:2181/kafka', offsetReset ='latest', - topic ='nbTest1', + groupId='nbTest', + topic ='nbTest1,nbTest2,nbTest3', + --- topic ='mqTest.*', + ---topicIsPattern='true', parallelism ='1' ); ``` diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java index 052b4f048..09e684c75 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java @@ -30,8 +30,10 @@ import org.apache.flink.types.Row; import org.apache.flink.util.SerializedValue; +import java.util.Arrays; import java.util.Map; import java.util.Properties; +import java.util.regex.Pattern; /** * Reason: @@ -47,7 +49,11 @@ public class CustomerKafka09Consumer extends FlinkKafkaConsumer09 { private CustomerJsonDeserialization customerJsonDeserialization; public CustomerKafka09Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { - super(topic, valueDeserializer, props); + super(Arrays.asList(topic.split(",")), valueDeserializer, props); + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + } + public CustomerKafka09Consumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index cbc697c82..782022270 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -25,6 +25,7 @@ import com.dtstack.flink.sql.table.SourceTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.commons.lang3.BooleanUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -39,6 +40,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Properties; +import java.util.regex.Pattern; /** * If eventtime field is specified, the default time field rowtime @@ -67,6 +69,8 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv Properties props = new Properties(); props.setProperty("bootstrap.servers", kafka09SourceTableInfo.getBootstrapServers()); props.setProperty("auto.offset.reset", kafka09SourceTableInfo.getOffsetReset()); + props.setProperty("group.id", kafka09SourceTableInfo.getGroupId()); + // only required for Kafka 0.8 //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) TypeInformation[] types = new TypeInformation[kafka09SourceTableInfo.getFields().length]; @@ -75,8 +79,14 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } TypeInformation typeInformation = new RowTypeInfo(types, kafka09SourceTableInfo.getFields()); - FlinkKafkaConsumer09 kafkaSrc = new CustomerKafka09Consumer(topicName, - new CustomerJsonDeserialization(typeInformation), props); + FlinkKafkaConsumer09 kafkaSrc; + if (BooleanUtils.isTrue(kafka09SourceTableInfo.getTopicIsPattern())) { + kafkaSrc = new CustomerKafka09Consumer(Pattern.compile(topicName), + new CustomerJsonDeserialization(typeInformation), props); + } else { + kafkaSrc = new CustomerKafka09Consumer(topicName, + new CustomerJsonDeserialization(typeInformation), props); + } //earliest,latest if("earliest".equalsIgnoreCase(kafka09SourceTableInfo.getOffsetReset())){ diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 301b0b423..664e8c4b1 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -47,6 +47,8 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map { private CustomerJsonDeserialization customerJsonDeserialization; public CustomerKafka010Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { - super(topic, valueDeserializer, props); + super(Arrays.asList(topic.split(",")), valueDeserializer, props); this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } + public CustomerKafka010Consumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + } @Override public void run(SourceContext sourceContext) throws Exception { customerJsonDeserialization.setRuntimeContext(getRuntimeContext()); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 2953d0f86..3fb175804 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package com.dtstack.flink.sql.source.kafka; @@ -25,6 +24,7 @@ import com.dtstack.flink.sql.table.SourceTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.commons.lang3.BooleanUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -38,66 +38,78 @@ import java.util.HashMap; import java.util.Map; import java.util.Properties; +import java.util.regex.Pattern; /** * If eventtime field is specified, the default time field rowtime * Date: 2018/09/18 * Company: www.dtstack.com + * * @author sishu.yss */ public class KafkaSource implements IStreamSourceGener

{ - private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; + private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; - /** - * Get kafka data source, you need to provide the data field names, data types - * If you do not specify auto.offset.reset, the default use groupoffset - * @param sourceTableInfo - * @return - */ - @SuppressWarnings("rawtypes") - @Override + /** + * Get kafka data source, you need to provide the data field names, data types + * If you do not specify auto.offset.reset, the default use groupoffset + * + * @param sourceTableInfo + * @return + */ + @SuppressWarnings("rawtypes") + @Override public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { - KafkaSourceTableInfo kafka010SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; - String topicName = kafka010SourceTableInfo.getTopic(); + KafkaSourceTableInfo kafka010SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; + String topicName = kafka010SourceTableInfo.getTopic(); + + Properties props = new Properties(); + props.setProperty("bootstrap.servers", kafka010SourceTableInfo.getBootstrapServers()); + props.setProperty("auto.offset.reset", kafka010SourceTableInfo.getOffsetReset()); + props.setProperty("group.id", kafka010SourceTableInfo.getGroupId()); + // only required for Kafka 0.8 + //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) - Properties props = new Properties(); - props.setProperty("bootstrap.servers", kafka010SourceTableInfo.getBootstrapServers()); - props.setProperty("auto.offset.reset", kafka010SourceTableInfo.getOffsetReset()); - //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) + TypeInformation[] types = new TypeInformation[kafka010SourceTableInfo.getFields().length]; + for (int i = 0; i < kafka010SourceTableInfo.getFieldClasses().length; i++) { + types[i] = TypeInformation.of(kafka010SourceTableInfo.getFieldClasses()[i]); + } - TypeInformation[] types = new TypeInformation[kafka010SourceTableInfo.getFields().length]; - for(int i = 0; i< kafka010SourceTableInfo.getFieldClasses().length; i++){ - types[i] = TypeInformation.of(kafka010SourceTableInfo.getFieldClasses()[i]); - } + TypeInformation typeInformation = new RowTypeInfo(types, kafka010SourceTableInfo.getFields()); - TypeInformation typeInformation = new RowTypeInfo(types, kafka010SourceTableInfo.getFields()); - FlinkKafkaConsumer010 kafkaSrc = new CustomerKafka010Consumer(topicName, - new CustomerJsonDeserialization(typeInformation), props); + FlinkKafkaConsumer010 kafkaSrc; + if (BooleanUtils.isTrue(kafka010SourceTableInfo.getTopicIsPattern())) { + kafkaSrc = new CustomerKafka010Consumer(Pattern.compile(topicName), + new CustomerJsonDeserialization(typeInformation), props); + } else { + kafkaSrc = new CustomerKafka010Consumer(topicName, + new CustomerJsonDeserialization(typeInformation), props); + } - //earliest,latest - if("earliest".equalsIgnoreCase(kafka010SourceTableInfo.getOffsetReset())){ - kafkaSrc.setStartFromEarliest(); - }else if(DtStringUtil.isJosn(kafka010SourceTableInfo.getOffsetReset())){// {"0":12312,"1":12321,"2":12312} - try { - Properties properties = PluginUtil.jsonStrToObject(kafka010SourceTableInfo.getOffsetReset(), Properties.class); - Map offsetMap = PluginUtil.ObjectToMap(properties); - Map specificStartupOffsets = new HashMap<>(); - for(Map.Entry entry:offsetMap.entrySet()){ - specificStartupOffsets.put(new KafkaTopicPartition(topicName,Integer.valueOf(entry.getKey())),Long.valueOf(entry.getValue().toString())); - } - kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); - } catch (Exception e) { - throw new RuntimeException("not support offsetReset type:" + kafka010SourceTableInfo.getOffsetReset()); - } - }else { - kafkaSrc.setStartFromLatest(); - } + //earliest,latest + if ("earliest".equalsIgnoreCase(kafka010SourceTableInfo.getOffsetReset())) { + kafkaSrc.setStartFromEarliest(); + } else if (DtStringUtil.isJosn(kafka010SourceTableInfo.getOffsetReset())) {// {"0":12312,"1":12321,"2":12312} + try { + Properties properties = PluginUtil.jsonStrToObject(kafka010SourceTableInfo.getOffsetReset(), Properties.class); + Map offsetMap = PluginUtil.ObjectToMap(properties); + Map specificStartupOffsets = new HashMap<>(); + for (Map.Entry entry : offsetMap.entrySet()) { + specificStartupOffsets.put(new KafkaTopicPartition(topicName, Integer.valueOf(entry.getKey())), Long.valueOf(entry.getValue().toString())); + } + kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); + } catch (Exception e) { + throw new RuntimeException("not support offsetReset type:" + kafka010SourceTableInfo.getOffsetReset()); + } + } else { + kafkaSrc.setStartFromLatest(); + } - String fields = StringUtils.join(kafka010SourceTableInfo.getFields(), ","); - String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); - return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); - } + String fields = StringUtils.join(kafka010SourceTableInfo.getFields(), ","); + String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); + return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); + } } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 65778d22c..4a2590536 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -47,6 +47,8 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map { private CustomerJsonDeserialization customerJsonDeserialization; public CustomerKafka011Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { - super(topic, valueDeserializer, props); + super(Arrays.asList(topic.split(",")), valueDeserializer, props); + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + } + + public CustomerKafka011Consumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index d10151920..5f312fa33 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package com.dtstack.flink.sql.source.kafka; @@ -25,6 +24,7 @@ import com.dtstack.flink.sql.table.SourceTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.commons.lang3.BooleanUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -38,66 +38,79 @@ import java.util.HashMap; import java.util.Map; import java.util.Properties; +import java.util.regex.Pattern; /** * If eventtime field is specified, the default time field rowtime * Date: 2018/09/18 * Company: www.dtstack.com + * * @author sishu.yss */ public class KafkaSource implements IStreamSourceGener
{ - private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; + private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; - /** - * Get kafka data source, you need to provide the data field names, data types - * If you do not specify auto.offset.reset, the default use groupoffset - * @param sourceTableInfo - * @return - */ - @SuppressWarnings("rawtypes") - @Override + /** + * Get kafka data source, you need to provide the data field names, data types + * If you do not specify auto.offset.reset, the default use groupoffset + * + * @param sourceTableInfo + * @return + */ + @SuppressWarnings("rawtypes") + @Override public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { - KafkaSourceTableInfo kafka011SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; - String topicName = kafka011SourceTableInfo.getTopic(); - - Properties props = new Properties(); - props.setProperty("bootstrap.servers", kafka011SourceTableInfo.getBootstrapServers()); - props.setProperty("auto.offset.reset", kafka011SourceTableInfo.getOffsetReset()); - //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) - - TypeInformation[] types = new TypeInformation[kafka011SourceTableInfo.getFields().length]; - for(int i = 0; i< kafka011SourceTableInfo.getFieldClasses().length; i++){ - types[i] = TypeInformation.of(kafka011SourceTableInfo.getFieldClasses()[i]); - } - - TypeInformation typeInformation = new RowTypeInfo(types, kafka011SourceTableInfo.getFields()); - FlinkKafkaConsumer011 kafkaSrc = new CustomerKafka011Consumer(topicName, - new CustomerJsonDeserialization(typeInformation), props); - - //earliest,latest - if("earliest".equalsIgnoreCase(kafka011SourceTableInfo.getOffsetReset())){ - kafkaSrc.setStartFromEarliest(); - }else if(DtStringUtil.isJosn(kafka011SourceTableInfo.getOffsetReset())){// {"0":12312,"1":12321,"2":12312} - try { - Properties properties = PluginUtil.jsonStrToObject(kafka011SourceTableInfo.getOffsetReset(), Properties.class); - Map offsetMap = PluginUtil.ObjectToMap(properties); - Map specificStartupOffsets = new HashMap<>(); - for(Map.Entry entry:offsetMap.entrySet()){ - specificStartupOffsets.put(new KafkaTopicPartition(topicName,Integer.valueOf(entry.getKey())),Long.valueOf(entry.getValue().toString())); - } - kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); - } catch (Exception e) { - throw new RuntimeException("not support offsetReset type:" + kafka011SourceTableInfo.getOffsetReset()); - } - }else{ - kafkaSrc.setStartFromLatest(); - } - - String fields = StringUtils.join(kafka011SourceTableInfo.getFields(), ","); - String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); - return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); - } + KafkaSourceTableInfo kafka011SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; + String topicName = kafka011SourceTableInfo.getTopic(); + + Properties props = new Properties(); + props.setProperty("bootstrap.servers", kafka011SourceTableInfo.getBootstrapServers()); + props.setProperty("auto.offset.reset", kafka011SourceTableInfo.getOffsetReset()); + props.setProperty("group.id", kafka011SourceTableInfo.getGroupId()); + // only required for Kafka 0.8 + //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) + + TypeInformation[] types = new TypeInformation[kafka011SourceTableInfo.getFields().length]; + for (int i = 0; i < kafka011SourceTableInfo.getFieldClasses().length; i++) { + types[i] = TypeInformation.of(kafka011SourceTableInfo.getFieldClasses()[i]); + } + + TypeInformation typeInformation = new RowTypeInfo(types, kafka011SourceTableInfo.getFields()); + + FlinkKafkaConsumer011 kafkaSrc; + if (BooleanUtils.isTrue(kafka011SourceTableInfo.getTopicIsPattern())) { + kafkaSrc = new CustomerKafka011Consumer(Pattern.compile(topicName), + new CustomerJsonDeserialization(typeInformation), props); + } else { + kafkaSrc = new CustomerKafka011Consumer(topicName, + new CustomerJsonDeserialization(typeInformation), props); + } + + + //earliest,latest + if ("earliest".equalsIgnoreCase(kafka011SourceTableInfo.getOffsetReset())) { + kafkaSrc.setStartFromEarliest(); + } else if (DtStringUtil.isJosn(kafka011SourceTableInfo.getOffsetReset())) {// {"0":12312,"1":12321,"2":12312} + try { + Properties properties = PluginUtil.jsonStrToObject(kafka011SourceTableInfo.getOffsetReset(), Properties.class); + Map offsetMap = PluginUtil.ObjectToMap(properties); + Map specificStartupOffsets = new HashMap<>(); + for (Map.Entry entry : offsetMap.entrySet()) { + specificStartupOffsets.put(new KafkaTopicPartition(topicName, Integer.valueOf(entry.getKey())), Long.valueOf(entry.getValue().toString())); + } + kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); + } catch (Exception e) { + throw new RuntimeException("not support offsetReset type:" + kafka011SourceTableInfo.getOffsetReset()); + } + } else { + kafkaSrc.setStartFromLatest(); + } + + String fields = StringUtils.join(kafka011SourceTableInfo.getFields(), ","); + String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); + return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); + } } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index f71d44f74..47d453adc 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -47,6 +47,8 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Tue, 11 Dec 2018 16:37:49 +0800 Subject: [PATCH 201/470] add password --- .../com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java index c2a33255d..71607e918 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java @@ -126,7 +126,7 @@ private void establishConnection() { break; //集群 case 3: - jedis = new JedisCluster(addresses, timeout, timeout,1, poolConfig); + jedis = new JedisCluster(addresses, timeout, timeout,10, password, poolConfig); } } From 13060e4a3b7ae39cc0333f0ce5354fa257934661 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 11 Dec 2018 17:18:42 +0800 Subject: [PATCH 202/470] remove groupId check --- docs/kafkaSource.md | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index f1518aaec..748948a7b 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -39,7 +39,7 @@ CREATE TABLE tableName( |zookeeperQuorum | kafka zk地址信息(多个之间用逗号分隔)|是|| |topic | 需要读取的 topic 名称|是|| |topicIsPattern | topic是否是正则表达式格式|否| false -|groupId | 需要读取的 groupId 名称|是|| +|groupId | 需要读取的 groupId 名称|否|| |offsetReset | 读取的topic 的offset初始位置[latest\|earliest\|指定offset值({"0":12312,"1":12321,"2":12312},{"partition_no":offset_value})]|否|latest| |parallelism | 并行度设置|否|1| diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index a17848e97..4daf83d6f 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -121,7 +121,7 @@ public void setOffset(String offset) { public boolean check() { Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); Preconditions.checkNotNull(topic, "kafka of topic is required"); - Preconditions.checkNotNull(groupId, "kafka of groupId is required"); + //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 2d948fa74..93e7d7642 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -121,7 +121,7 @@ public void setTopicIsPattern(Boolean topicIsPattern) { public boolean check() { Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); Preconditions.checkNotNull(topic, "kafka of topic is required"); - Preconditions.checkNotNull(groupId, "kafka of groupId is required"); + //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 36d78bdd8..b151a93e9 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -122,7 +122,7 @@ public void setOffset(String offset) { public boolean check() { Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); Preconditions.checkNotNull(topic, "kafka of topic is required"); - Preconditions.checkNotNull(groupId, "kafka of groupId is required"); + //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); From e5cf94ec034a3f47d8986cedc1749e71935f15ac Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 11 Dec 2018 18:15:30 +0800 Subject: [PATCH 203/470] groupid check null --- .../java/com/dtstack/flink/sql/source/kafka/KafkaSource.java | 4 +++- .../java/com/dtstack/flink/sql/source/kafka/KafkaSource.java | 4 +++- .../java/com/dtstack/flink/sql/source/kafka/KafkaSource.java | 4 +++- .../dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java | 4 ++-- 4 files changed, 11 insertions(+), 5 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 782022270..b08485e0b 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -69,7 +69,9 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv Properties props = new Properties(); props.setProperty("bootstrap.servers", kafka09SourceTableInfo.getBootstrapServers()); props.setProperty("auto.offset.reset", kafka09SourceTableInfo.getOffsetReset()); - props.setProperty("group.id", kafka09SourceTableInfo.getGroupId()); + if (StringUtils.isNotBlank(kafka09SourceTableInfo.getGroupId())){ + props.setProperty("group.id", kafka09SourceTableInfo.getGroupId()); + } // only required for Kafka 0.8 //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 3fb175804..523eb25dc 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -69,7 +69,9 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv Properties props = new Properties(); props.setProperty("bootstrap.servers", kafka010SourceTableInfo.getBootstrapServers()); props.setProperty("auto.offset.reset", kafka010SourceTableInfo.getOffsetReset()); - props.setProperty("group.id", kafka010SourceTableInfo.getGroupId()); + if (StringUtils.isNotBlank(kafka010SourceTableInfo.getGroupId())){ + props.setProperty("group.id", kafka010SourceTableInfo.getGroupId()); + } // only required for Kafka 0.8 //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 5f312fa33..5c9f5eb49 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -69,7 +69,9 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv Properties props = new Properties(); props.setProperty("bootstrap.servers", kafka011SourceTableInfo.getBootstrapServers()); props.setProperty("auto.offset.reset", kafka011SourceTableInfo.getOffsetReset()); - props.setProperty("group.id", kafka011SourceTableInfo.getGroupId()); + if (StringUtils.isNotBlank(kafka011SourceTableInfo.getGroupId())){ + props.setProperty("group.id", kafka011SourceTableInfo.getGroupId()); + } // only required for Kafka 0.8 //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index 3aa5e6f99..8e513adfc 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -70,7 +70,7 @@ public void fillRealIndexes() throws SQLException { } String column_name = rs.getString("COLUMN_NAME"); if (StringUtils.isNotBlank(column_name)) { - column_name = column_name.toUpperCase(); + column_name = column_name; } map.get(indexName).add(column_name); } @@ -94,7 +94,7 @@ public void fillFullColumns() throws SQLException { while (rs.next()) { String columnName = rs.getString("COLUMN_NAME"); if (StringUtils.isNotBlank(columnName)) { - getFullField().add(columnName.toUpperCase()); + getFullField().add(columnName); } } } From e04b47c464514ee52ce6b50237fe670ed00b11c1 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 11 Dec 2018 18:40:19 +0800 Subject: [PATCH 204/470] reset ExtendOutputFormat --- .../dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index 8e513adfc..3aa5e6f99 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -70,7 +70,7 @@ public void fillRealIndexes() throws SQLException { } String column_name = rs.getString("COLUMN_NAME"); if (StringUtils.isNotBlank(column_name)) { - column_name = column_name; + column_name = column_name.toUpperCase(); } map.get(indexName).add(column_name); } @@ -94,7 +94,7 @@ public void fillFullColumns() throws SQLException { while (rs.next()) { String columnName = rs.getString("COLUMN_NAME"); if (StringUtils.isNotBlank(columnName)) { - getFullField().add(columnName); + getFullField().add(columnName.toUpperCase()); } } } From 75e0b8bcaa56291215408e572424422a0f80f19b Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 11 Dec 2018 19:01:05 +0800 Subject: [PATCH 205/470] remove toUppercase --- .../dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index 3aa5e6f99..8e513adfc 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -70,7 +70,7 @@ public void fillRealIndexes() throws SQLException { } String column_name = rs.getString("COLUMN_NAME"); if (StringUtils.isNotBlank(column_name)) { - column_name = column_name.toUpperCase(); + column_name = column_name; } map.get(indexName).add(column_name); } @@ -94,7 +94,7 @@ public void fillFullColumns() throws SQLException { while (rs.next()) { String columnName = rs.getString("COLUMN_NAME"); if (StringUtils.isNotBlank(columnName)) { - getFullField().add(columnName.toUpperCase()); + getFullField().add(columnName); } } } From 6bcab67b0a91b5b4ce392b4643334c9939d8a25d Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 11 Dec 2018 21:04:21 +0800 Subject: [PATCH 206/470] modify compare fun for repalce sql --- .../main/java/com/dtstack/flink/sql/Main.java | 2 +- .../sql/parser/CreateTmpTableParser.java | 2 +- .../flink/sql/parser/InsertSqlParser.java | 2 +- .../dtstack/flink/sql/side/SideSQLParser.java | 2 +- .../flink/sql/sink/oracle/OracleSink.java | 16 ++- .../sink/rdb/format/ExtendOutputFormat.java | 121 ++++++++++-------- .../sql/sink/sqlserver/SqlserverSink.java | 14 +- 7 files changed, 94 insertions(+), 65 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 2e9c979d1..19f660eb0 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -188,7 +188,7 @@ public static void main(String[] args) throws Exception { String realSql = DtStringUtil.replaceIgnoreQuota(result.getExecSql(), "`", ""); org.apache.calcite.sql.parser.SqlParser.Config config = org.apache.calcite.sql.parser.SqlParser .configBuilder() - .setLex(Lex.JAVA) + .setLex(Lex.MYSQL) .build(); SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql,config).parseStmt(); String tmpSql = ((SqlInsert) sqlNode).getSource().toString(); diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java index d205c59b6..5840b26a8 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -69,7 +69,7 @@ public void parseSql(String sql, SqlTree sqlTree) { SqlParser.Config config = SqlParser .configBuilder() - .setLex(Lex.JAVA) + .setLex(Lex.MYSQL) .build(); SqlParser sqlParser = SqlParser.create(selectSql,config); diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index 7d5c61e7a..52541385f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -60,7 +60,7 @@ public static InsertSqlParser newInstance(){ public void parseSql(String sql, SqlTree sqlTree) { SqlParser.Config config = SqlParser .configBuilder() - .setLex(Lex.JAVA) + .setLex(Lex.MYSQL) .build(); SqlParser sqlParser = SqlParser.create(sql,config); SqlNode sqlNode = null; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 472146c11..388bb5497 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -58,7 +58,7 @@ public Queue getExeQueue(String exeSql, Set sideTableSet) throws Queue queueInfo = Queues.newLinkedBlockingQueue(); SqlParser.Config config = SqlParser .configBuilder() - .setLex(Lex.JAVA) + .setLex(Lex.MYSQL) .build(); SqlParser sqlParser = SqlParser.create(exeSql,config); SqlNode sqlNode = sqlParser.parseStmt(); diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index 4252f025c..8d2399469 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -24,6 +24,7 @@ import org.apache.commons.lang3.StringUtils; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -94,7 +95,7 @@ protected List keyColList(Map> updateKey) { for (Map.Entry> entry : updateKey.entrySet()) { List list = entry.getValue(); for (String col : list) { - if (!keyCols.contains(col)) { + if (!containsIgnoreCase(keyCols,col)) { keyCols.add(col); } } @@ -107,10 +108,10 @@ public String getUpdateSql(List column, List fullColumn, String String prefixRight = StringUtils.isBlank(rightTable) ? "" : quoteTable(rightTable) + "."; List list = new ArrayList<>(); for (String col : fullColumn) { - if (keyCols == null || keyCols.size() == 0 || keyCols.contains(col)) { + if (keyCols == null || keyCols.size() == 0 || containsIgnoreCase(keyCols,col)) { continue; } - if (fullColumn == null || column.contains(col)) { + if (fullColumn == null || containsIgnoreCase(column,col)) { list.add(prefixLeft + col + "=" + prefixRight + col); } else { list.add(prefixLeft + col + "=null"); @@ -157,6 +158,15 @@ public String makeValues(List column) { return sb.toString(); } + public boolean containsIgnoreCase(List l, String s) { + Iterator it = l.iterator(); + while (it.hasNext()) { + if (it.next().equalsIgnoreCase(s)) + return true; + } + return false; + } + public String quoteColumn(String column) { return getStartQuote() + column + getEndQuote(); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index 8e513adfc..7c14807d2 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -24,6 +24,7 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -37,67 +38,75 @@ public class ExtendOutputFormat extends RetractJDBCOutputFormat { - @Override - public boolean isReplaceInsertQuery() throws SQLException { - fillRealIndexes(); - fillFullColumns(); + @Override + public boolean isReplaceInsertQuery() throws SQLException { + fillRealIndexes(); + fillFullColumns(); - if (!getRealIndexes().isEmpty()) { - for (List value : getRealIndexes().values()) { - for (String fieldName : getDbSink().getFieldNames()) { - if (value.contains(fieldName)) { - return true; - } - } - } - } - return false; - } + if (!getRealIndexes().isEmpty()) { + for (List value : getRealIndexes().values()) { + for (String fieldName : getDbSink().getFieldNames()) { + if (containsIgnoreCase(value, fieldName)) { + return true; + } + } + } + } + return false; + } - /** - * get db all index - * - * @throws SQLException - */ - public void fillRealIndexes() throws SQLException { - Map> map = Maps.newHashMap(); - ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, null, getTableName(), true, false); + /** + * get db all index + * + * @throws SQLException + */ + public void fillRealIndexes() throws SQLException { + Map> map = Maps.newHashMap(); + ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, null, getTableName(), true, false); - while (rs.next()) { - String indexName = rs.getString("INDEX_NAME"); - if (!map.containsKey(indexName)) { - map.put(indexName, new ArrayList<>()); - } - String column_name = rs.getString("COLUMN_NAME"); - if (StringUtils.isNotBlank(column_name)) { - column_name = column_name; - } - map.get(indexName).add(column_name); - } + while (rs.next()) { + String indexName = rs.getString("INDEX_NAME"); + if (!map.containsKey(indexName)) { + map.put(indexName, new ArrayList<>()); + } + String column_name = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(column_name)) { + column_name = column_name; + } + map.get(indexName).add(column_name); + } - for (Map.Entry> entry : map.entrySet()) { - String k = entry.getKey(); - List v = entry.getValue(); - if (v != null && v.size() != 0 && v.get(0) != null) { - getRealIndexes().put(k, v); - } - } - } + for (Map.Entry> entry : map.entrySet()) { + String k = entry.getKey(); + List v = entry.getValue(); + if (v != null && v.size() != 0 && v.get(0) != null) { + getRealIndexes().put(k, v); + } + } + } - /** - * get db all column name - * - * @throws SQLException - */ - public void fillFullColumns() throws SQLException { - ResultSet rs = getDbConn().getMetaData().getColumns(null, null, getTableName(), null); - while (rs.next()) { - String columnName = rs.getString("COLUMN_NAME"); - if (StringUtils.isNotBlank(columnName)) { - getFullField().add(columnName); - } - } - } + /** + * get db all column name + * + * @throws SQLException + */ + public void fillFullColumns() throws SQLException { + ResultSet rs = getDbConn().getMetaData().getColumns(null, null, getTableName(), null); + while (rs.next()) { + String columnName = rs.getString("COLUMN_NAME"); + if (StringUtils.isNotBlank(columnName)) { + getFullField().add(columnName); + } + } + } + public boolean containsIgnoreCase(List l, String s) { + Iterator it = l.iterator(); + while (it.hasNext()) { + if (it.next().equalsIgnoreCase(s)) + return true; + } + return false; + } } diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java index 2bb53064e..e07d49945 100644 --- a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java @@ -92,7 +92,7 @@ protected List keyColList(Map> updateKey) { for (Map.Entry> entry : updateKey.entrySet()) { List list = entry.getValue(); for (String col : list) { - if (!keyCols.contains(col)) { + if (!containsIgnoreCase(keyCols,col)) { keyCols.add(col); } } @@ -108,7 +108,7 @@ public String getUpdateSql(List column, List fullColumn, String if (keyCols == null || keyCols.size() == 0) { continue; } - if (fullColumn == null || column.contains(col)) { + if (fullColumn == null || containsIgnoreCase(column,col)) { list.add(prefixLeft + col + "=" + prefixRight + col); } else { list.add(prefixLeft + col + "=null"); @@ -154,6 +154,16 @@ public String makeValues(List column) { return sb.toString(); } + public boolean containsIgnoreCase(List l, String s) { + Iterator it = l.iterator(); + while (it.hasNext()) { + if (it.next().equalsIgnoreCase(s)) + return true; + } + return false; + } + + public String quoteColumn(String column) { return getStartQuote() + column + getEndQuote(); } From 805fc17068f3a8504c15923c6512eb88a2576ef9 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Wed, 12 Dec 2018 13:51:52 +0800 Subject: [PATCH 207/470] add tmp table to cache --- .../main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 080d3d48b..a38719ebc 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -67,6 +67,8 @@ public class SideSqlExec { private SideSQLParser sideSQLParser = new SideSQLParser(); + private Map localTableCache = Maps.newHashMap(); + public void exec(String sql, Map sideTableMap, StreamTableEnvironment tableEnv, Map tableCache) throws Exception { @@ -75,7 +77,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl throw new RuntimeException("need to set localSqlPluginPath"); } - Map localTableCache = Maps.newHashMap(tableCache); + localTableCache.putAll(tableCache); Queue exeQueue = sideSQLParser.getExeQueue(sql, sideTableMap.keySet()); Object pollObj = null; @@ -501,7 +503,7 @@ public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, throw new RuntimeException("need to set localSqlPluginPath"); } - Map localTableCache = Maps.newHashMap(tableCache); + localTableCache.putAll(tableCache); Queue exeQueue = sideSQLParser.getExeQueue(result.getExecSql(), sideTableMap.keySet()); Object pollObj = null; @@ -540,6 +542,7 @@ public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, throw new RuntimeException("Fields mismatch"); } } + localTableCache.put(result.getTableName(), table); } From 8930d32ce02b0dd919ac1e82ca821279060c2dfa Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 12 Dec 2018 17:42:53 +0800 Subject: [PATCH 208/470] case upper and lower --- .../main/java/com/dtstack/flink/sql/Main.java | 8 +++++++- .../flink/sql/parser/CreateTmpTableParser.java | 8 ++++++-- .../flink/sql/parser/InsertSqlParser.java | 7 ++++++- .../dtstack/flink/sql/side/SideSQLParser.java | 7 ++++++- .../flink/sql/sink/oracle/OracleSink.java | 16 +++++++++++++--- .../sql/sink/rdb/format/ExtendOutputFormat.java | 12 ++++++++++-- .../flink/sql/sink/sqlserver/SqlserverSink.java | 12 ++++++++++-- 7 files changed, 58 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 610abf21f..91e5b6e07 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -16,7 +16,7 @@ * limitations under the License. */ - + package com.dtstack.flink.sql; @@ -34,6 +34,7 @@ import com.dtstack.flink.sql.watermarker.WaterMarkerAssigner; import com.dtstack.flink.sql.util.FlinkUtil; import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.calcite.config.Lex; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlNode; import org.apache.commons.cli.CommandLine; @@ -185,6 +186,11 @@ public static void main(String[] args) throws Exception { if (sqlTree.getTmpTableMap().containsKey(tableName)) { CreateTmpTableParser.SqlParserResult tmp = sqlTree.getTmpTableMap().get(tableName); String realSql = DtStringUtil.replaceIgnoreQuota(result.getExecSql(), "`", ""); + + org.apache.calcite.sql.parser.SqlParser.Config config = org.apache.calcite.sql.parser.SqlParser + .configBuilder() + .setLex(Lex.MYSQL) + .build(); SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql).parseStmt(); String tmpSql = ((SqlInsert) sqlNode).getSource().toString(); tmp.setExecSql(tmpSql); diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java index 8da89b2be..43c607849 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.parser; import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.calcite.config.Lex; import org.apache.calcite.sql.*; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; @@ -65,8 +66,11 @@ public void parseSql(String sql, SqlTree sqlTree) { tableName = matcher.group(1).toUpperCase(); selectSql = "select " + matcher.group(2); } - - SqlParser sqlParser = SqlParser.create(selectSql); + SqlParser.Config config = SqlParser + .configBuilder() + .setLex(Lex.MYSQL) + .build(); + SqlParser sqlParser = SqlParser.create(selectSql,config); SqlNode sqlNode = null; try { sqlNode = sqlParser.parseStmt(); diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index e08540c92..52541385f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.parser; +import org.apache.calcite.config.Lex; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlJoin; @@ -57,7 +58,11 @@ public static InsertSqlParser newInstance(){ @Override public void parseSql(String sql, SqlTree sqlTree) { - SqlParser sqlParser = SqlParser.create(sql); + SqlParser.Config config = SqlParser + .configBuilder() + .setLex(Lex.MYSQL) + .build(); + SqlParser sqlParser = SqlParser.create(sql,config); SqlNode sqlNode = null; try { sqlNode = sqlParser.parseStmt(); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 7f165ac30..a14458d0f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.side; import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.calcite.config.Lex; import org.apache.calcite.sql.JoinType; import org.apache.calcite.sql.SqlAsOperator; import org.apache.calcite.sql.SqlBasicCall; @@ -56,7 +57,11 @@ public Queue getExeQueue(String exeSql, Set sideTableSet) throws System.out.println("---exeSql---"); System.out.println(exeSql); Queue queueInfo = Queues.newLinkedBlockingQueue(); - SqlParser sqlParser = SqlParser.create(exeSql); + SqlParser.Config config = SqlParser + .configBuilder() + .setLex(Lex.MYSQL) + .build(); + SqlParser sqlParser = SqlParser.create(exeSql,config); SqlNode sqlNode = sqlParser.parseStmt(); parseSql(sqlNode, sideTableSet, queueInfo); queueInfo.offer(sqlNode); diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index 4252f025c..2ca020c54 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -24,6 +24,7 @@ import org.apache.commons.lang3.StringUtils; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -94,7 +95,7 @@ protected List keyColList(Map> updateKey) { for (Map.Entry> entry : updateKey.entrySet()) { List list = entry.getValue(); for (String col : list) { - if (!keyCols.contains(col)) { + if (!containsIgnoreCase(keyCols,col)) { keyCols.add(col); } } @@ -107,10 +108,10 @@ public String getUpdateSql(List column, List fullColumn, String String prefixRight = StringUtils.isBlank(rightTable) ? "" : quoteTable(rightTable) + "."; List list = new ArrayList<>(); for (String col : fullColumn) { - if (keyCols == null || keyCols.size() == 0 || keyCols.contains(col)) { + if (keyCols == null || keyCols.size() == 0 || containsIgnoreCase(keyCols,col)) { continue; } - if (fullColumn == null || column.contains(col)) { + if (fullColumn == null ||containsIgnoreCase(column,col)) { list.add(prefixLeft + col + "=" + prefixRight + col); } else { list.add(prefixLeft + col + "=null"); @@ -157,6 +158,15 @@ public String makeValues(List column) { return sb.toString(); } + public boolean containsIgnoreCase(List l, String s) { + Iterator it = l.iterator(); + while (it.hasNext()) { + if (it.next().equalsIgnoreCase(s)) + return true; + } + return false; + } + public String quoteColumn(String column) { return getStartQuote() + column + getEndQuote(); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index 3aa5e6f99..f288e50c0 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -24,6 +24,7 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -45,7 +46,7 @@ public boolean isReplaceInsertQuery() throws SQLException { if (!getRealIndexes().isEmpty()) { for (List value : getRealIndexes().values()) { for (String fieldName : getDbSink().getFieldNames()) { - if (value.contains(fieldName)) { + if (containsIgnoreCase(value, fieldName)) { return true; } } @@ -99,5 +100,12 @@ public void fillFullColumns() throws SQLException { } } - + public boolean containsIgnoreCase(List l, String s) { + Iterator it = l.iterator(); + while (it.hasNext()) { + if (it.next().equalsIgnoreCase(s)) + return true; + } + return false; + } } diff --git a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java index 2bb53064e..904cb0895 100644 --- a/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java +++ b/sqlserver/sqlserver-sink/src/main/java/com/dtstack/flink/sql/sink/sqlserver/SqlserverSink.java @@ -92,7 +92,7 @@ protected List keyColList(Map> updateKey) { for (Map.Entry> entry : updateKey.entrySet()) { List list = entry.getValue(); for (String col : list) { - if (!keyCols.contains(col)) { + if (!containsIgnoreCase(keyCols,col)) { keyCols.add(col); } } @@ -108,7 +108,7 @@ public String getUpdateSql(List column, List fullColumn, String if (keyCols == null || keyCols.size() == 0) { continue; } - if (fullColumn == null || column.contains(col)) { + if (fullColumn == null || containsIgnoreCase(column,col)) { list.add(prefixLeft + col + "=" + prefixRight + col); } else { list.add(prefixLeft + col + "=null"); @@ -154,6 +154,14 @@ public String makeValues(List column) { return sb.toString(); } + public boolean containsIgnoreCase(List l, String s) { + Iterator it = l.iterator(); + while (it.hasNext()) { + if (it.next().equalsIgnoreCase(s)) + return true; + } + return false; + } public String quoteColumn(String column) { return getStartQuote() + column + getEndQuote(); } From a19132a506700b2081313b210fa0ca357d2f0831 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 12 Dec 2018 19:07:21 +0800 Subject: [PATCH 209/470] case upper and lower 2 --- core/src/main/java/com/dtstack/flink/sql/Main.java | 4 ++-- .../java/com/dtstack/flink/sql/parser/CreateTableParser.java | 2 +- .../com/dtstack/flink/sql/parser/CreateTmpTableParser.java | 4 ++-- .../main/java/com/dtstack/flink/sql/side/SideSQLParser.java | 1 - .../src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 2 +- .../main/java/com/dtstack/flink/sql/table/AbsTableParser.java | 4 ---- 6 files changed, 6 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 91e5b6e07..415b6e2a9 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -191,7 +191,7 @@ public static void main(String[] args) throws Exception { .configBuilder() .setLex(Lex.MYSQL) .build(); - SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql).parseStmt(); + SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql,config).parseStmt(); String tmpSql = ((SqlInsert) sqlNode).getSource().toString(); tmp.setExecSql(tmpSql); sideSqlExec.registerTmpTable(tmp, sideTableMap, tableEnv, registerTableCache); @@ -252,7 +252,7 @@ private static void registerUDF(SqlTree sqlTree, List jarURList, URLClassLo classLoader = FlinkUtil.loadExtraJar(jarURList, parentClassloader); } classLoader.loadClass(funcInfo.getClassName()); - FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName().toUpperCase(), + FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName(), tableEnv, classLoader); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java index d10d6825e..b5e4a4aa4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java @@ -54,7 +54,7 @@ public boolean verify(String sql) { public void parseSql(String sql, SqlTree sqlTree) { Matcher matcher = PATTERN.matcher(sql); if(matcher.find()){ - String tableName = matcher.group(1).toUpperCase(); + String tableName = matcher.group(1); String fieldsInfoStr = matcher.group(2); String propsStr = matcher.group(3); Map props = parseProp(propsStr); diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java index 43c607849..cb5620d0a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -63,7 +63,7 @@ public void parseSql(String sql, SqlTree sqlTree) { String tableName = null; String selectSql = null; if(matcher.find()) { - tableName = matcher.group(1).toUpperCase(); + tableName = matcher.group(1); selectSql = "select " + matcher.group(2); } SqlParser.Config config = SqlParser @@ -93,7 +93,7 @@ public void parseSql(String sql, SqlTree sqlTree) { String tableName = null; String fieldsInfoStr = null; if (matcher.find()){ - tableName = matcher.group(1).toUpperCase(); + tableName = matcher.group(1); fieldsInfoStr = matcher.group(2); } CreateTmpTableParser.SqlParserResult sqlParseResult = new CreateTmpTableParser.SqlParserResult(); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index a14458d0f..388bb5497 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -53,7 +53,6 @@ public class SideSQLParser { public Queue getExeQueue(String exeSql, Set sideTableSet) throws SqlParseException { - exeSql = DtStringUtil.replaceIgnoreQuota(exeSql, "`", ""); System.out.println("---exeSql---"); System.out.println(exeSql); Queue queueInfo = Queues.newLinkedBlockingQueue(); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index a38719ebc..3623ecfd4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -645,7 +645,7 @@ private boolean checkFieldsInfo(CreateTmpTableParser.SqlParserResult result, Tab String[] filedNameArr = new String[filed.length - 1]; System.arraycopy(filed, 0, filedNameArr, 0, filed.length - 1); String fieldName = String.join(" ", filedNameArr); - fieldNames.add(fieldName.toUpperCase()); + fieldNames.add(fieldName); String fieldType = filed[filed.length - 1 ].trim(); Class fieldClass = ClassUtil.stringConvertClass(fieldType); Class tableField = table.getSchema().getType(i).get().getTypeClass(); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index 2fc2d799b..a43769918 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -82,10 +82,6 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ String[] fieldRows = DtStringUtil.splitIgnoreQuotaBrackets(fieldsInfo, ","); for(String fieldRow : fieldRows){ fieldRow = fieldRow.trim(); - if(fieldNameNeedsUpperCase()) { - fieldRow = fieldRow.toUpperCase(); - } - boolean isMatcherKey = dealKeyPattern(fieldRow, tableInfo); if(isMatcherKey){ From 35869af7c79ae73589b50f6214d51fffce7bc9a1 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 12 Dec 2018 20:07:34 +0800 Subject: [PATCH 210/470] fix conflict --- .../java/com/dtstack/flink/sql/parser/InsertSqlParser.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index 61ee3f9eb..f62b09d04 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -21,12 +21,7 @@ package com.dtstack.flink.sql.parser; import org.apache.calcite.config.Lex; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlInsert; -import org.apache.calcite.sql.SqlJoin; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlSelect; +import org.apache.calcite.sql.*; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; import org.apache.commons.lang3.StringUtils; From 66e8d6f3a63216cb823fa2f04d166d3cb1f9c3aa Mon Sep 17 00:00:00 2001 From: Hongtao Zhang <553780043@qq.com> Date: Thu, 13 Dec 2018 10:48:56 +0800 Subject: [PATCH 211/470] fix kafka offset reset check --- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 1b60ec476..6c1aae76f 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -123,7 +123,7 @@ public boolean check() { Preconditions.checkNotNull(topic, "kafka of topic is required"); //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + || offsetReset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } From dd31001e4af2b9c248aa4c1e36278e055de97266 Mon Sep 17 00:00:00 2001 From: Hongtao Zhang <553780043@qq.com> Date: Thu, 13 Dec 2018 10:50:29 +0800 Subject: [PATCH 212/470] fix kafka offset check --- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index de79e0509..768b30360 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -122,7 +122,7 @@ public boolean check() { Preconditions.checkNotNull(topic, "kafka of topic is required"); //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + || offsetReset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } From 54baa284be2dea7825eaea4bc84cb538547e3719 Mon Sep 17 00:00:00 2001 From: Hongtao Zhang <553780043@qq.com> Date: Thu, 13 Dec 2018 10:51:13 +0800 Subject: [PATCH 213/470] fix kafka offset check --- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 131941467..e39a389e1 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -122,7 +122,7 @@ public boolean check() { Preconditions.checkNotNull(topic, "kafka of topic is required"); //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + || offsetReset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } From 28b377de0604fead3625b2f64c993d0f5c9c84e2 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 13 Dec 2018 15:16:59 +0800 Subject: [PATCH 214/470] support int --- .../com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java index 71607e918..962054f24 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java @@ -166,7 +166,7 @@ public void writeRecord(Tuple2 record) throws IOException { for (int i = 0; i < fieldNames.length; i++) { StringBuilder key = new StringBuilder(); key.append(tableName).append(":").append(perKey).append(":").append(fieldNames[i]); - jedis.set(key.toString(), (String) row.getField(i)); + jedis.set(key.toString(), row.getField(i).toString()); } outRecords.inc(); } From fafb9f266de51bada7f712a9c7da3b73c4d82d72 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 14 Dec 2018 16:40:22 +0800 Subject: [PATCH 215/470] Update README.md --- README.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 5dfa80efb..93edde5c3 100644 --- a/README.md +++ b/README.md @@ -14,10 +14,13 @@ * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra # 后续开发计划 - * 增加kafka结果表功能 * 增加SQL支持CEP * 维表快照 * sql优化(谓词下移等) + * serverSocket 源表 + * console 结果表 + * kafka avro格式 + * topN ## 1 快速起步 ### 1.1 运行模式 From 7951643f7c60dc2b85917b4e35de2fef8ef59600 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Mon, 17 Dec 2018 16:31:38 +0800 Subject: [PATCH 216/470] youhua --- .../flink/sql/launcher/LauncherMain.java | 56 ++++++++----------- .../flink/sql/launcher/LauncherOptions.java | 10 ++++ 2 files changed, 33 insertions(+), 33 deletions(-) diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 96eb6fb43..ec6a3e10c 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -62,50 +62,40 @@ public static void main(String[] args) throws Exception { if (args.length == 1 && args[0].endsWith(".json")){ args = parseJson(args); } - LauncherOptionParser optionParser = new LauncherOptionParser(args); LauncherOptions launcherOptions = optionParser.getLauncherOptions(); String mode = launcherOptions.getMode(); List argList = optionParser.getProgramExeArgList(); - if(mode.equals(ClusterMode.local.name())) { String[] localArgs = argList.toArray(new String[argList.size()]); Main.main(localArgs); - return; - } - - String pluginRoot = launcherOptions.getLocalSqlPluginPath(); - File jarFile = new File(getLocalCoreJarPath(pluginRoot)); - String[] remoteArgs = argList.toArray(new String[argList.size()]); - PackagedProgram program = new PackagedProgram(jarFile, Lists.newArrayList(), remoteArgs); - - if(StringUtils.isNotBlank(launcherOptions.getSavePointPath())){ - program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getSavePointPath(), BooleanUtils.toBoolean(launcherOptions.getAllowNonRestoredState()))); - } - - if(mode.equals(ClusterMode.yarnPer.name())){ - String flinkConfDir = launcherOptions.getFlinkconf(); - Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); - JobGraph jobGraph = PackagedProgramUtils.createJobGraph(program, config, 1); - PerJobSubmitter.submit(launcherOptions, jobGraph); - } else { - ClusterClient clusterClient = ClusterClientFactory.createClusterClient(launcherOptions); - pluginRoot = launcherOptions.getLocalSqlPluginPath(); - jarFile = new File(getLocalCoreJarPath(pluginRoot)); - remoteArgs = argList.toArray(new String[argList.size()]); - program = new PackagedProgram(jarFile, Lists.newArrayList(), remoteArgs); + }else{ + String pluginRoot = launcherOptions.getLocalSqlPluginPath(); + File jarFile = new File(getLocalCoreJarPath(pluginRoot)); + String[] remoteArgs = argList.toArray(new String[argList.size()]); + PackagedProgram program = new PackagedProgram(jarFile, Lists.newArrayList(), remoteArgs); if(StringUtils.isNotBlank(launcherOptions.getSavePointPath())){ program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getSavePointPath(), BooleanUtils.toBoolean(launcherOptions.getAllowNonRestoredState()))); } - //final JobGraph jobGraph; - //jobGraph = PackagedProgramUtils.createJobGraph(program, new Configuration(), 1); - //clusterClient.runDetached(jobGraph,null); - clusterClient.run(program, 1); - clusterClient.shutdown(); - - System.exit(0); + if(mode.equals(ClusterMode.yarnPer.name())){ + String flinkConfDir = launcherOptions.getFlinkconf(); + Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); + JobGraph jobGraph = PackagedProgramUtils.createJobGraph(program, config, launcherOptions.getDefaultParallelism()); + PerJobSubmitter.submit(launcherOptions, jobGraph); + } else { + ClusterClient clusterClient = ClusterClientFactory.createClusterClient(launcherOptions); + pluginRoot = launcherOptions.getLocalSqlPluginPath(); + jarFile = new File(getLocalCoreJarPath(pluginRoot)); + remoteArgs = argList.toArray(new String[argList.size()]); + program = new PackagedProgram(jarFile, Lists.newArrayList(), remoteArgs); + if(StringUtils.isNotBlank(launcherOptions.getSavePointPath())){ + program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getSavePointPath(), BooleanUtils.toBoolean(launcherOptions.getAllowNonRestoredState()))); + } + clusterClient.run(program, launcherOptions.getDefaultParallelism()); + clusterClient.shutdown(); + System.exit(0); + } } - System.out.println("---submit end----"); } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java index 5cc762ebd..d9c5bd1ed 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java @@ -53,6 +53,8 @@ public class LauncherOptions { private String queue; + private int defaultParallelism=1; + public String getMode() { return mode; } @@ -156,4 +158,12 @@ public String getQueue() { public void setQueue(String queue) { this.queue = queue; } + + public int getDefaultParallelism() { + return defaultParallelism; + } + + public void setDefaultParallelism(int defaultParallelism) { + this.defaultParallelism = defaultParallelism; + } } From bfc219b883c65715408efcbbf4dbba3a5efa0f38 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 18 Dec 2018 10:54:43 +0800 Subject: [PATCH 217/470] add kafka sink and read csv,txt format data --- .../flink/sql/sink/StreamSinkFactory.java | 7 +- .../dtstack/flink/sql/table/TableInfo.java | 50 ++++ .../dtstack/flink/sql/util/DtStringUtil.java | 27 +- docs/kafkaSource.md | 263 ++++++++++++++++-- kafka08/kafka08-sink/pom.xml | 94 +++++++ .../sink/kafka/CustomerCsvSerialization.java | 120 ++++++++ .../flink/sql/sink/kafka/KafkaSink.java | 133 +++++++++ .../sql/sink/kafka/table/KafkaSinkParser.java | 48 ++++ .../sink/kafka/table/KafkaSinkTableInfo.java | 62 +++++ kafka08/kafka08-source/pom.xml | 91 ++++++ .../flink/sql/source/kafka/KafkaSource.java | 136 +++++++++ .../consumer/CustomerCommonConsumer.java | 68 +++++ .../kafka/consumer/CustomerCsvConsumer.java | 74 +++++ .../kafka/consumer/CustomerJsonConsumer.java | 73 +++++ .../CustomerCommonDeserialization.java | 91 ++++++ .../CustomerCsvDeserialization.java | 119 ++++++++ .../CustomerJsonDeserialization.java | 141 ++++++++++ .../source/kafka/table/KafkaSourceParser.java | 58 ++++ .../kafka/table/KafkaSourceTableInfo.java | 83 ++++++ kafka08/pom.xml | 37 +++ kafka09/kafka09-sink/pom.xml | 90 ++++++ .../sink/kafka/CustomerCsvSerialization.java | 120 ++++++++ .../flink/sql/sink/kafka/KafkaSink.java | 133 +++++++++ .../sql/sink/kafka/table/KafkaSinkParser.java | 48 ++++ .../sink/kafka/table/KafkaSinkTableInfo.java | 61 ++++ kafka09/kafka09-source/pom.xml | 10 - .../flink/sql/source/kafka/KafkaSource.java | 72 +++-- .../consumer/CustomerCommonConsumer.java | 74 +++++ .../kafka/consumer/CustomerCsvConsumer.java | 80 ++++++ .../CustomerJsonConsumer.java} | 21 +- .../CustomerCommonDeserialization.java | 150 ++++++++++ .../CustomerCsvDeserialization.java | 181 ++++++++++++ .../CustomerJsonDeserialization.java | 3 +- .../source/kafka/table/KafkaSourceParser.java | 24 +- .../kafka/table/KafkaSourceTableInfo.java | 97 ++----- kafka09/pom.xml | 8 + kafka10/kafka10-sink/pom.xml | 89 ++++++ .../sink/kafka/CustomerCsvSerialization.java | 120 ++++++++ .../flink/sql/sink/kafka/KafkaSink.java | 133 +++++++++ .../sql/sink/kafka/table/KafkaSinkParser.java | 48 ++++ .../sink/kafka/table/KafkaSinkTableInfo.java | 62 +++++ kafka10/kafka10-source/pom.xml | 11 - .../flink/sql/source/kafka/KafkaSource.java | 75 +++-- .../consumer/CustomerCommonConsumer.java | 74 +++++ .../kafka/consumer/CustomerCsvConsumer.java | 80 ++++++ .../kafka/consumer/CustomerJsonConsumer.java | 78 ++++++ .../CustomerCommonDeserialization.java | 150 ++++++++++ .../CustomerCsvDeserialization.java | 181 ++++++++++++ .../CustomerJsonDeserialization.java | 5 +- .../source/kafka/table/KafkaSourceParser.java | 24 +- .../kafka/table/KafkaSourceTableInfo.java | 97 ++----- kafka10/pom.xml | 7 + kafka11/kafka11-sink/pom.xml | 89 ++++++ .../sink/kafka/CustomerCsvSerialization.java | 137 +++++++++ .../flink/sql/sink/kafka/KafkaSink.java | 134 +++++++++ .../sql/sink/kafka/table/KafkaSinkParser.java | 48 ++++ .../sink/kafka/table/KafkaSinkTableInfo.java | 62 +++++ kafka11/kafka11-source/pom.xml | 9 - .../flink/sql/source/kafka/KafkaSource.java | 69 +++-- .../consumer/CustomerCommonConsumer.java | 73 +++++ .../kafka/consumer/CustomerCsvConsumer.java | 33 ++- .../CustomerJsonConsumer.java} | 13 +- .../CustomerCommonDeserialization.java | 152 ++++++++++ .../CustomerCsvDeserialization.java | 183 ++++++++++++ .../CustomerJsonDeserialization.java | 3 +- .../source/kafka/table/KafkaSourceParser.java | 45 +-- .../kafka/table/KafkaSourceTableInfo.java | 96 ++----- kafka11/pom.xml | 9 + pom.xml | 1 + 69 files changed, 4948 insertions(+), 389 deletions(-) create mode 100644 kafka08/kafka08-sink/pom.xml create mode 100644 kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java create mode 100644 kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java create mode 100644 kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java create mode 100644 kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java create mode 100644 kafka08/kafka08-source/pom.xml create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java create mode 100644 kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java create mode 100644 kafka08/pom.xml create mode 100644 kafka09/kafka09-sink/pom.xml create mode 100644 kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java create mode 100644 kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java create mode 100644 kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java create mode 100644 kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java create mode 100644 kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java create mode 100644 kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java rename kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/{CustomerKafka09Consumer.java => consumer/CustomerJsonConsumer.java} (69%) create mode 100644 kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java create mode 100644 kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java rename {kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka => kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization}/CustomerJsonDeserialization.java (98%) create mode 100644 kafka10/kafka10-sink/pom.xml create mode 100644 kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java create mode 100644 kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java create mode 100644 kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java create mode 100644 kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java create mode 100644 kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java create mode 100644 kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java create mode 100644 kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java create mode 100644 kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java create mode 100644 kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java rename {kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka => kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization}/CustomerJsonDeserialization.java (98%) create mode 100644 kafka11/kafka11-sink/pom.xml create mode 100644 kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java create mode 100644 kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java create mode 100644 kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java create mode 100644 kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java create mode 100644 kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java rename kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka010Consumer.java => kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java (70%) rename kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/{CustomerKafka011Consumer.java => consumer/CustomerJsonConsumer.java} (86%) create mode 100644 kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java create mode 100644 kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java rename kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/{ => deserialization}/CustomerJsonDeserialization.java (98%) diff --git a/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java b/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java index 9ef30f97c..e072a2345 100644 --- a/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java @@ -51,9 +51,9 @@ public static AbsTableParser getSqlParser(String pluginType, String sqlRootDir) DtClassLoader dtClassLoader = (DtClassLoader) classLoader; String pluginJarPath = PluginUtil.getJarFileDirPath(String.format(DIR_NAME_FORMAT, pluginType), sqlRootDir); - PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); - String className = PluginUtil.getSqlParserClassName(pluginType, CURR_TYPE); + String typeNoVersion = DtStringUtil.getPluginTypeWithoutVersion(pluginType); + String className = PluginUtil.getSqlParserClassName(typeNoVersion, CURR_TYPE); Class targetParser = dtClassLoader.loadClass(className); if(!AbsTableParser.class.isAssignableFrom(targetParser)){ @@ -76,8 +76,9 @@ public static TableSink getTableSink(TargetTableInfo targetTableInfo, String loc String pluginJarDirPath = PluginUtil.getJarFileDirPath(String.format(DIR_NAME_FORMAT, pluginType), localSqlRootDir); PluginUtil.addPluginJar(pluginJarDirPath, dtClassLoader); + String typeNoVersion = DtStringUtil.getPluginTypeWithoutVersion(pluginType); - String className = PluginUtil.getGenerClassName(pluginType, CURR_TYPE); + String className = PluginUtil.getGenerClassName(typeNoVersion, CURR_TYPE); Class sinkClass = dtClassLoader.loadClass(className); if(!IStreamSinkGener.class.isAssignableFrom(sinkClass)){ diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java index b56b790ea..d69b5c2bf 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java @@ -36,6 +36,14 @@ public abstract class TableInfo implements Serializable { public static final String PARALLELISM_KEY = "parallelism"; + public static final String SOURCE_DATA_TYPE = "sourcedatatype"; + + public static final String SINK_DATA_TYPE = "sinkdatatype"; + + public static final String FIELD_DELINITER = "fielddelimiter"; + + public static final String LENGTH_CHECK_POLICY = "lengthcheckpolicy"; + private String name; private String type; @@ -56,6 +64,16 @@ public abstract class TableInfo implements Serializable { private Integer parallelism = 1; + private String sourceDataType = "json"; + + private String sinkDataType = "json"; + + private String fieldDelimiter; + + private String lengthCheckPolicy = "SKIP"; + + + public String[] getFieldTypes() { return fieldTypes; } @@ -146,6 +164,38 @@ public List getFieldClassList() { return fieldClassList; } + public String getSourceDataType() { + return sourceDataType; + } + + public void setSourceDataType(String sourceDataType) { + this.sourceDataType = sourceDataType; + } + + public String getSinkDataType() { + return sinkDataType; + } + + public void setSinkDataType(String sinkDataType) { + this.sinkDataType = sinkDataType; + } + + public String getFieldDelimiter() { + return fieldDelimiter; + } + + public void setFieldDelimiter(String fieldDelimiter) { + this.fieldDelimiter = fieldDelimiter; + } + + public String getLengthCheckPolicy() { + return lengthCheckPolicy; + } + + public void setLengthCheckPolicy(String lengthCheckPolicy) { + this.lengthCheckPolicy = lengthCheckPolicy; + } + public void finish(){ this.fields = fieldList.toArray(new String[fieldList.size()]); this.fieldClasses = fieldClassList.toArray(new Class[fieldClassList.size()]); diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index 959d4b13c..96026ca4b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -16,7 +16,7 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.util; @@ -25,7 +25,7 @@ import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; - +import java.sql.Timestamp; import java.math.BigDecimal; import java.util.ArrayList; import java.util.List; @@ -225,4 +225,27 @@ public static boolean isJosn(String str){ } return flag; } + + public static Object parse(String str,Class clazz){ + String fieldType = clazz.getName(); + Object object = null; + if(fieldType.equals(Integer.class.getName())){ + object = Integer.parseInt(str); + }else if(fieldType.equals(Long.class.getName())){ + object = Long.parseLong(str); + }else if(fieldType.equals(Byte.class.getName())){ + object = str.getBytes()[0]; + }else if(fieldType.equals(String.class.getName())){ + object = str; + }else if(fieldType.equals(Float.class.getName())){ + object = Float.parseFloat(str); + }else if(fieldType.equals(Double.class.getName())){ + object = Double.parseDouble(str); + }else if (fieldType.equals(Timestamp.class.getName())){ + object = Timestamp.valueOf(str); + }else{ + throw new RuntimeException("no support field type for sql. the input type:" + fieldType); + } + return object; + } } diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index 748948a7b..d6745c3d7 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -1,3 +1,4 @@ +# 一、json格式数据源 ## 1.格式: ``` 数据现在支持json格式{"xx":"bb","cc":"dd"} @@ -9,16 +10,17 @@ CREATE TABLE tableName( WATERMARK FOR colName AS withOffset( colName , delayTime ) )WITH( type ='kafka09', - bootstrapServers ='ip:port,ip:port...', - zookeeperQuorum ='ip:port,ip:port/zkparent', - offsetReset ='latest', - topic ='topicName', - parallelism ='parllNum' + kafka.bootstrap.servers ='ip:port,ip:port...', + kafka.zookeeper.quorum ='ip:port,ip:port/zkparent', + kafka.auto.offset.reset ='latest', + kafka.topic ='topicName', + parallelism ='parllNum', + sourcedatatype ='json' #可不设置 ); ``` ## 2.支持的版本 - kafka09,kafka10,kafka11 + kafka08,kafka09,kafka10,kafka11 ## 3.表结构定义 @@ -35,14 +37,16 @@ CREATE TABLE tableName( |参数名称|含义|是否必填|默认值| |----|---|---|---| |type | kafka09 | 是|| -|bootstrapServers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| -|zookeeperQuorum | kafka zk地址信息(多个之间用逗号分隔)|是|| -|topic | 需要读取的 topic 名称|是|| -|topicIsPattern | topic是否是正则表达式格式|否| false -|groupId | 需要读取的 groupId 名称|否|| -|offsetReset | 读取的topic 的offset初始位置[latest\|earliest\|指定offset值({"0":12312,"1":12321,"2":12312},{"partition_no":offset_value})]|否|latest| +|kafka.group.id | 需要读取的 groupId 名称|否|| +|kafka.bootstrap.servers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| +|kafka.zookeeper.quorum | kafka zk地址信息(多个之间用逗号分隔)|是|| +|kafka.topic | 需要读取的 topic 名称|是|| +|patterntopic | topic是否是正则表达式格式(true|false) |否| false +|kafka.auto.offset.reset | 读取的topic 的offset初始位置[latest\|earliest\|指定offset值({"0":12312,"1":12321,"2":12312},{"partition_no":offset_value})]|否|latest| |parallelism | 并行度设置|否|1| - +|sourcedatatype | 数据类型|否|json| +**kafka相关参数可以自定义,使用kafka.开头即可。** + ## 5.样例: ``` CREATE TABLE MyTable( @@ -53,13 +57,230 @@ CREATE TABLE MyTable( CHARACTER_LENGTH(channel) AS timeLeng )WITH( type ='kafka09', - bootstrapServers ='172.16.8.198:9092', - zookeeperQuorum ='172.16.8.198:2181/kafka', - offsetReset ='latest', - groupId='nbTest', - topic ='nbTest1,nbTest2,nbTest3', - --- topic ='mqTest.*', - ---topicIsPattern='true', - parallelism ='1' + kafka.bootstrap.servers ='172.16.8.198:9092', + kafka.zookeeper.quorum ='172.16.8.198:2181/kafka', + kafka.auto.offset.reset ='latest', + kafka.topic ='nbTest1,nbTest2,nbTest3', + --kafka.topic ='mqTest.*', + --patterntopic='true' + parallelism ='1', + sourcedatatype ='json' #可不设置 ); ``` +# 二、csv格式数据源 +根据字段分隔符进行数据分隔,按顺序匹配sql中配置的列。如数据分隔列数和sql中配置的列数相等直接匹配;如不同参照lengthcheckpolicy策略处理。 +## 1.参数: + +|参数名称|含义|是否必填|默认值| +|----|---|---|---| +|type | kafka09 | 是|| +|kafka.bootstrap.servers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| +|kafka.zookeeper.quorum | kafka zk地址信息(多个之间用逗号分隔)|是|| +|kafka.topic | 需要读取的 topic 名称|是|| +|kafka.auto.offset.reset | 读取的topic 的offset初始位置[latest\|earliest]|否|latest| +|parallelism | 并行度设置 |否|1| +|sourcedatatype | 数据类型|是 |csv| +|fielddelimiter | 字段分隔符|是 || +|lengthcheckpolicy | 单行字段条数检查策略 |否|可选,默认为SKIP,其它可选值为EXCEPTION、PAD。SKIP:字段数目不符合时跳过 。EXCEPTION:字段数目不符合时抛出异常。PAD:按顺序填充,不存在的置为null。| +**kafka相关参数可以自定义,使用kafka.开头即可。** + +## 2.样例: +``` +CREATE TABLE MyTable( + name varchar, + channel varchar, + pv INT, + xctime bigint, + CHARACTER_LENGTH(channel) AS timeLeng + )WITH( + type ='kafka09', + kafka.bootstrap.servers ='172.16.8.198:9092', + kafka.zookeeper.quorum ='172.16.8.198:2181/kafka', + kafka.auto.offset.reset ='latest', + kafka.topic ='nbTest1', + --kafka.topic ='mqTest.*', + --kafka.topicIsPattern='true' + parallelism ='1', + sourcedatatype ='csv', + fielddelimiter ='\|', + lengthcheckpolicy = 'PAD' + ); + ``` +# 三、text格式数据源UDF自定义拆分 +Kafka源表数据解析流程:Kafka Source Table -> UDTF ->Realtime Compute -> SINK。从Kakfa读入的数据,都是VARBINARY(二进制)格式,对读入的每条数据,都需要用UDTF将其解析成格式化数据。 + 与其他格式不同,本格式定义DDL必须与以下SQL一摸一样,表中的五个字段顺序务必保持一致: + +## 1. 定义源表,注意:kafka源表DDL字段必须与以下例子一模一样。WITH中参数可改。 +``` +create table kafka_stream( + _topic STRING, + _messageKey STRING, + _message STRING, + _partition INT, + _offset BIGINT, +) with ( + type ='kafka09', + kafka.bootstrap.servers ='172.16.8.198:9092', + kafka.zookeeper.quorum ='172.16.8.198:2181/kafka', + kafka.auto.offset.reset ='latest', + kafka.topic ='nbTest1', + parallelism ='1', + sourcedatatype='text' + ) +``` +## 2.参数: + +|参数名称|含义|是否必填|默认值| +|----|---|---|---| +|type | kafka09 | 是|| +|kafka.bootstrap.servers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| +|kafka.zookeeper.quorum | kafka zk地址信息(多个之间用逗号分隔)|是|| +|kafka.topic | 需要读取的 topic 名称|是|| +|kafka.auto.offset.reset | 读取的topic 的offset初始位置[latest\|earliest]|否|latest| +|parallelism | 并行度设置|否|1| +|sourcedatatype | 数据类型|否|text| +**kafka相关参数可以自定义,使用kafka.开头即可。** + +## 2.自定义: +从kafka读出的数据,需要进行窗口计算。 按照实时计算目前的设计,滚窗/滑窗等窗口操作,需要(且必须)在源表DDL上定义Watermark。Kafka源表比较特殊。如果要以kafka中message字段中的的Event Time进行窗口操作, +需要先从message字段,使用UDX解析出event time,才能定义watermark。 在kafka源表场景中,需要使用计算列。 假设,kafka中写入的数据如下: +2018-11-11 00:00:00|1|Anna|female整个计算流程为:Kafka SOURCE->UDTF->Realtime Compute->RDS SINK(单一分隔符可直接使用类csv格式模板,自定义适用于更复杂的数据类型,本说明只做参考) + +**SQL** +``` +-- 定义解析Kakfa message的UDTF + CREATE FUNCTION kafkapaser AS 'com.XXXX.kafkaUDTF'; + CREATE FUNCTION kafkaUDF AS 'com.XXXX.kafkaUDF'; + -- 定义源表,注意:kafka源表DDL字段必须与以下例子一模一样。WITH中参数可改。 + create table kafka_src ( + _topic STRING, + _messageKey STRING, + _message STRING, + _partition INT, + _offset BIGINT, + ctime AS TO_TIMESTAMP(kafkaUDF(_message)), -- 定义计算列,计算列可理解为占位符,源表中并没有这一列,其中的数据可经过下游计算得出。注意计算里的类型必须为timestamp才能在做watermark。 + watermark for ctime as withoffset(ctime,0) -- 在计算列上定义watermark + ) WITH ( + type = 'kafka010', -- Kafka Source类型,与Kafka版本强相关,目前支持的Kafka版本请参考本文档 + topic = 'test_kafka_topic', + ... + ); + create table rds_sink ( + name VARCHAR, + age INT, + grade VARCHAR, + updateTime TIMESTAMP + ) WITH( + type='mysql', + url='jdbc:mysql://localhost:3306/test', + tableName='test4', + userName='test', + password='XXXXXX' + ); + -- 使用UDTF,将二进制数据解析成格式化数据 + CREATE VIEW input_view ( + name, + age, + grade, + updateTime + ) AS + SELECT + COUNT(*) as cnt, + T.ctime, + T.order, + T.name, + T.sex + from + kafka_src as S, + LATERAL TABLE (kafkapaser _message)) as T ( + ctime, + order, + name, + sex + ) + Group BY T.sex, + TUMBLE(ctime, INTERVAL '1' MINUTE); + -- 对input_view中输出的数据做计算 + CREATE VIEW view2 ( + cnt, + sex + ) AS + SELECT + COUNT(*) as cnt, + T.sex + from + input_view + Group BY sex, TUMBLE(ctime, INTERVAL '1' MINUTE); + -- 使用解析出的格式化数据进行计算,并将结果输出到RDS中 + insert into rds_sink + SELECT + cnt,sex + from view2; + ``` +**UDF&UDTF** +``` +package com.XXXX; + import com.XXXX.fastjson.JSONObject; + import org.apache.flink.table.functions.TableFunction; + import org.apache.flink.table.types.DataType; + import org.apache.flink.table.types.DataTypes; + import org.apache.flink.types.Row; + import java.io.UnsupportedEncodingException; + /** + 以下例子解析输入Kafka中的JSON字符串,并将其格式化输出 + **/ + public class kafkaUDTF extends TableFunction { + public void eval(byte[] message) { + try { + // 读入一个二进制数据,并将其转换为String格式 + String msg = new String(message, "UTF-8"); + // 提取JSON Object中各字段 + String ctime = Timestamp.valueOf(data.split('\\|')[0]); + String order = data.split('\\|')[1]; + String name = data.split('\\|')[2]; + String sex = data.split('\\|')[3]; + // 将解析出的字段放到要输出的Row()对象 + Row row = new Row(4); + row.setField(0, ctime); + row.setField(1, age); + row.setField(2, grade); + row.setField(3, updateTime); + System.out.println("Kafka message str ==>" + row.toString()); + // 输出一行 + collect(row); + } catch (ClassCastException e) { + System.out.println("Input data format error. Input data " + msg + "is not json string"); + } + } catch (UnsupportedEncodingException e) { + e.printStackTrace(); + } + } + @Override + // 如果返回值是Row,就必须重载实现这个方法,显式地告诉系统返回的字段类型 + // 定义输出Row()对象的字段类型 + public DataType getResultType(Object[] arguments, Class[] argTypes) { + return DataTypes.createRowType(DataTypes.TIMESTAMP,DataTypes.STRING, DataTypes.Integer, DataTypes.STRING,DataTypes.STRING); + } + } + + package com.dp58; + package com.dp58.sql.udx; + import org.apache.flink.table.functions.FunctionContext; + import org.apache.flink.table.functions.ScalarFunction; + public class KafkaUDF extends ScalarFunction { + // 可选,open方法可以不写 + // 需要import org.apache.flink.table.functions.FunctionContext; + public String eval(byte[] message) { + // 读入一个二进制数据,并将其转换为String格式 + String msg = new String(message, "UTF-8"); + return msg.split('\\|')[0]; + } + public long eval(String b, String c) { + return eval(b) + eval(c); + } + //可选,close方法可以不写 + @Override + public void close() { + } + } + ``` diff --git a/kafka08/kafka08-sink/pom.xml b/kafka08/kafka08-sink/pom.xml new file mode 100644 index 000000000..47391d182 --- /dev/null +++ b/kafka08/kafka08-sink/pom.xml @@ -0,0 +1,94 @@ + + + + sql.kafka08 + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.kafka08 + jar + + kafka08-sink + http://maven.apache.org + + + + org.apache.flink + flink-connector-kafka-0.8_2.11 + ${flink.version} + + + org.apache.flink + flink-json + ${flink.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java new file mode 100644 index 000000000..355c79645 --- /dev/null +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -0,0 +1,120 @@ +package com.dtstack.flink.sql.sink.kafka; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.shaded.org.apache.commons.lang.StringEscapeUtils; +import org.apache.flink.types.Row; +import org.apache.flink.types.StringValue; + +import java.io.IOException; + +import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; + +@Internal +public final class CustomerCsvSerialization extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + private String fieldDelimiter = "\u0001"; + private TypeInformation[] fieldTypes; + private TypeSerializer[] fieldSerializers; + private static final Row EMPTY = null; + + public CustomerCsvSerialization(String fielddelimiter,TypeInformation[] fieldTypes) { + this.fieldDelimiter = fielddelimiter; + this.fieldTypes = fieldTypes; + this.fieldSerializers = (TypeSerializer[])createSerializer(new ExecutionConfig()); + } + + public TypeSerializer[] createSerializer(ExecutionConfig config) { + int len = fieldTypes.length; + TypeSerializer[] fieldSerializers = new TypeSerializer[len]; + for (int i = 0; i < len; i++) { + fieldSerializers[i] = fieldTypes[i].createSerializer(config); + } + return fieldSerializers; + } + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + public Row createInstance() { + return EMPTY; + } + + @Override + public Row copy(Row from) { + return null; + } + + @Override + public Row copy(Row from, Row reuse) { + return null; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(Row record, DataOutputView target) throws IOException { + int len = fieldSerializers.length; + + if (record.getArity() != len) { + throw new RuntimeException("Row arity of from does not match serializers."); + } + + // write a null mask + writeNullMask(len, record, target); + + // serialize non-null fields + StringBuffer stringBuffer = new StringBuffer(); + for (int i = 0; i < len; i++) { + Object o = record.getField(i); + if (o != null) { + //fieldSerializers[i].serialize(o, target); + stringBuffer.append(o); + } + if(i != len-1){ + stringBuffer.append(StringEscapeUtils.unescapeJava(fieldDelimiter)); + //fieldSerializers[i].serialize(fieldDelimiter, target); + } + } + StringValue.writeString(stringBuffer.toString(), target); + } + + @Override + public Row deserialize(DataInputView source) throws IOException { + return null; + } + + @Override + public Row deserialize(Row reuse, DataInputView source) throws IOException { + return null; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + StringValue.copyString(source, target); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof CustomerCsvSerialization; + } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(StringValue.class.getCanonicalName()); + } +} diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java new file mode 100644 index 000000000..41f597fe1 --- /dev/null +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -0,0 +1,133 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.connectors.kafka.Kafka08TableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.TableSchemaBuilder; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +import java.util.Optional; +import java.util.Properties; + +public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { + + protected String[] fieldNames; + + protected TypeInformation[] fieldTypes; + + /** The schema of the table. */ + private TableSchema schema; + + /** The Kafka topic to write to. */ + protected String topic; + + /** Properties for the Kafka producer. */ + protected Properties properties; + + /** Serialization schema for encoding records to Kafka. */ + protected SerializationSchema serializationSchema; + + /** Partitioner to select Kafka partition for each item. */ + protected Optional> partitioner; + + @Override + public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { + KafkaSinkTableInfo kafka08SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; + this.topic = kafka08SinkTableInfo.getKafkaParam("topic"); + + Properties props = new Properties(); + for (String key:kafka08SinkTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka08SinkTableInfo.getKafkaParam(key)); + } + this.properties = props; + this.partitioner = Optional.of(new FlinkFixedPartitioner<>()); + this.fieldNames = kafka08SinkTableInfo.getFields(); + TypeInformation[] types = new TypeInformation[kafka08SinkTableInfo.getFields().length]; + for(int i = 0; i< kafka08SinkTableInfo.getFieldClasses().length; i++){ + types[i] = TypeInformation.of(kafka08SinkTableInfo.getFieldClasses()[i]); + } + this.fieldTypes = types; + + TableSchemaBuilder schemaBuilder = TableSchema.builder(); + for (int i=0;i dataStream) { + KafkaTableSink kafkaTableSink = new Kafka08TableSink( + schema, + topic, + properties, + partitioner, + serializationSchema + ); + + kafkaTableSink.emitDataStream(dataStream); + } + + @Override + public TypeInformation getOutputType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + @Override + public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + +} diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java new file mode 100644 index 000000000..838f59541 --- /dev/null +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -0,0 +1,48 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka.table; + +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +public class KafkaSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + KafkaSinkTableInfo kafka11SinkTableInfo = new KafkaSinkTableInfo(); + kafka11SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka11SinkTableInfo); + kafka11SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); + if (props.get(KafkaSinkTableInfo.SINK_DATA_TYPE) != null) { + kafka11SinkTableInfo.setSourceDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); + } + if (props.get(KafkaSinkTableInfo.FIELD_DELINITER) != null) { + kafka11SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); + } + + for (String key:props.keySet()) { + if (!key.isEmpty() && key.startsWith("kafka.")) { + kafka11SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + } + } + return kafka11SinkTableInfo; + } +} diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java new file mode 100644 index 000000000..67100dc05 --- /dev/null +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.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 com.dtstack.flink.sql.sink.kafka.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +public class KafkaSinkTableInfo extends TargetTableInfo { + //version + private static final String CURR_TYPE = "kafka11"; + + public KafkaSinkTableInfo(){ + super.setType(CURR_TYPE); + } + + public static Map kafkaParam = new HashMap(); + + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); + } + + public String getKafkaParam(String key){ + return kafkaParam.get(key); + } + + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); + } + + @Override + public boolean check() { + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + // Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); + return false; + } + + @Override + public String getType() { + return super.getType(); + } +} diff --git a/kafka08/kafka08-source/pom.xml b/kafka08/kafka08-source/pom.xml new file mode 100644 index 000000000..df98c2603 --- /dev/null +++ b/kafka08/kafka08-source/pom.xml @@ -0,0 +1,91 @@ + + + + sql.kafka08 + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.source.kafka08 + jar + + kafka08-source + http://maven.apache.org + + + + org.apache.flink + flink-connector-kafka-0.8_2.11 + 1.6.0 + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + org.slf4j + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java new file mode 100644 index 000000000..4ec6832a2 --- /dev/null +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -0,0 +1,136 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka; + +import com.dtstack.flink.sql.source.IStreamSourceGener; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerCommonConsumer; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerCsvConsumer; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerJsonConsumer; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; +import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; +import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; +import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.types.Row; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +public class KafkaSource implements IStreamSourceGener
{ + + private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; + + /** + * Get kafka data source, you need to provide the data field names, data types + * If you do not specify auto.offset.reset, the default use groupoffset + * + * @param sourceTableInfo + * @return + */ + @SuppressWarnings("rawtypes") + @Override + public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { + KafkaSourceTableInfo kafka08SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; + String topicName = kafka08SourceTableInfo.getKafkaParam("topic"); + String offsetReset = kafka08SourceTableInfo.getKafkaParam("auto.offset.reset"); + Boolean topicIsPattern = kafka08SourceTableInfo.getPatternTopic(); + + Properties props = new Properties(); + for (String key : kafka08SourceTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka08SourceTableInfo.getKafkaParam(key)); + } + + TypeInformation[] types = new TypeInformation[kafka08SourceTableInfo.getFields().length]; + for (int i = 0; i < kafka08SourceTableInfo.getFieldClasses().length; i++) { + types[i] = TypeInformation.of(kafka08SourceTableInfo.getFieldClasses()[i]); + } + + TypeInformation typeInformation = new RowTypeInfo(types, kafka08SourceTableInfo.getFields()); + + FlinkKafkaConsumer08 kafkaSrc; + String fields = StringUtils.join(kafka08SourceTableInfo.getFields(), ","); + + if ("json".equalsIgnoreCase(kafka08SourceTableInfo.getSourceDataType())) { + if (topicIsPattern) { + kafkaSrc = new CustomerJsonConsumer(Pattern.compile(topicName), + new com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization(typeInformation), props); + } else { + kafkaSrc = new CustomerJsonConsumer(topicName, + new CustomerJsonDeserialization(typeInformation), props); + } + } else if ("csv".equalsIgnoreCase(kafka08SourceTableInfo.getSourceDataType())) { + if (topicIsPattern) { + kafkaSrc = new CustomerCsvConsumer(topicName, + new com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization(typeInformation, + kafka08SourceTableInfo.getFieldDelimiter(), kafka08SourceTableInfo.getLengthCheckPolicy()), props); + } else { + kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), + new CustomerCsvDeserialization(typeInformation, + kafka08SourceTableInfo.getFieldDelimiter(), kafka08SourceTableInfo.getLengthCheckPolicy()), props); + } + } else { + if (topicIsPattern) { + kafkaSrc = new CustomerCommonConsumer(topicName, new com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization(), props); + } else { + kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); + } + + } + + //earliest,latest + if ("earliest".equalsIgnoreCase(offsetReset)) { + kafkaSrc.setStartFromEarliest(); + } else if (DtStringUtil.isJosn(offsetReset)) {// {"0":12312,"1":12321,"2":12312} + try { + Properties properties = PluginUtil.jsonStrToObject(offsetReset, Properties.class); + Map offsetMap = PluginUtil.ObjectToMap(properties); + Map specificStartupOffsets = new HashMap<>(); + for (Map.Entry entry : offsetMap.entrySet()) { + specificStartupOffsets.put(new KafkaTopicPartition(topicName, Integer.valueOf(entry.getKey())), Long.valueOf(entry.getValue().toString())); + } + kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); + } catch (Exception e) { + throw new RuntimeException("not support offsetReset type:" + offsetReset); + } + } else { + kafkaSrc.setStartFromLatest(); + } + String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); + DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); + Integer parallelism = kafka08SourceTableInfo.getParallelism(); + if (parallelism != null) { + kafkaSource.setParallelism(parallelism); + } + return tableEnv.fromDataStream(kafkaSource, fields); + } +} diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java new file mode 100644 index 000000000..b1e6f24c7 --- /dev/null +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java @@ -0,0 +1,68 @@ +/** + * 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 com.dtstack.flink.sql.source.kafka.consumer; + +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/12/17 + * Company: www.dtstack.com + * + * @author maqi + */ +public class CustomerCommonConsumer extends FlinkKafkaConsumer08 { + + private CustomerCommonDeserialization customerCommonDeserialization; + + + public CustomerCommonConsumer(String topic, KeyedDeserializationSchema deserializer, Properties props) { + super(topic, deserializer, props); + this.customerCommonDeserialization= (CustomerCommonDeserialization) deserializer; + } + + public CustomerCommonConsumer(Pattern subscriptionPattern, KeyedDeserializationSchema deserializer, Properties props) { + super(subscriptionPattern, deserializer, props); + this.customerCommonDeserialization= (CustomerCommonDeserialization) deserializer; + } + + + @Override + public void run(SourceFunction.SourceContext sourceContext) throws Exception { + customerCommonDeserialization.setRuntimeContext(getRuntimeContext()); + customerCommonDeserialization.initMetric(); + super.run(sourceContext); + } + +} diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java new file mode 100644 index 000000000..7457dd9ff --- /dev/null +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java @@ -0,0 +1,74 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.consumer; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/10/19 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class CustomerCsvConsumer extends FlinkKafkaConsumer08 { + + private static final long serialVersionUID = -2265366268827807739L; + + private CustomerCsvDeserialization customerCsvDeserialization; + + public CustomerCsvConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + super(Arrays.asList(topic.split(",")), valueDeserializer, props); + this.customerCsvDeserialization = (CustomerCsvDeserialization) valueDeserializer; + } + + public CustomerCsvConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); + this.customerCsvDeserialization = (CustomerCsvDeserialization) valueDeserializer; + } + + + + + @Override + public void run(SourceFunction.SourceContext sourceContext) throws Exception { + customerCsvDeserialization.setRuntimeContext(getRuntimeContext()); + customerCsvDeserialization.initMetric(); + super.run(sourceContext); + } + +} diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java new file mode 100644 index 000000000..0b775a4dc --- /dev/null +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java @@ -0,0 +1,73 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.consumer; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/10/19 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class CustomerJsonConsumer extends FlinkKafkaConsumer08 { + + private static final long serialVersionUID = -2265366268827807739L; + + private CustomerJsonDeserialization customerJsonDeserialization; + + public CustomerJsonConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + super(Arrays.asList(topic.split(",")), valueDeserializer, props); + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + } + + public CustomerJsonConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + } + + + @Override + public void run(SourceFunction.SourceContext sourceContext) throws Exception { + customerJsonDeserialization.setRuntimeContext(getRuntimeContext()); + customerJsonDeserialization.initMetric(); + super.run(sourceContext); + } + + +} diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java new file mode 100644 index 000000000..0d08dd092 --- /dev/null +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -0,0 +1,91 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.deserialization; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import static java.nio.charset.StandardCharsets.UTF_8; + +/** + * Date: 2017/5/28 + * + * @author DocLi + */ +public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { + private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); + + public static final String[] KAFKA_COLUMNS = new String[]{"_TOPIC", "_MESSAGEKEY", "_MESSAGE", "_PARTITION", "_OFFSET"}; + + private boolean firstMsg = true; + + @Override + public Row deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) { + + numInRecord.inc(); + numInBytes.inc(message.length); + numInBytes.inc(messageKey.length); + + try { + Row row = Row.of( + topic, //topic + messageKey == null ? null : new String(messageKey, UTF_8), //key + new String(message, UTF_8), //message + partition, + offset + ); + return row; + } catch (Throwable t) { + LOG.error(t.getMessage()); + dirtyDataCounter.inc(); + return null; + } + } + + @Override + public Row deserialize(byte[] message) throws IOException { + return null; + } + + + @Override + public boolean isEndOfStream(Row nextElement) { + return false; + } + + public TypeInformation getProducedType() { + TypeInformation[] types = new TypeInformation[]{ + TypeExtractor.createTypeInfo(String.class), + TypeExtractor.createTypeInfo(String.class), //createTypeInformation[String] + TypeExtractor.createTypeInfo(String.class), + Types.INT, + Types.LONG + }; + return new RowTypeInfo(types, KAFKA_COLUMNS); + } + +} diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java new file mode 100644 index 000000000..caa0ce7c6 --- /dev/null +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -0,0 +1,119 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.deserialization; + + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/** + * json string parsing custom + * Date: 2017/5/28 + * Company: www.dtstack.com + * @author DocLi + */ + +public class CustomerCsvDeserialization extends AbsDeserialization { + + private static final Logger LOG = LoggerFactory.getLogger(CustomerCsvDeserialization.class); + + private static final long serialVersionUID = -2706012724306826506L; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Type information describing the result type. */ + private final TypeInformation typeInfo; + + /** Field names to parse. Indices match fieldTypes indices. */ + private final String[] fieldNames; + + /** Types to parse fields as. Indices match fieldNames indices. */ + private final TypeInformation[] fieldTypes; + + /** Flag indicating whether to fail on a missing field. */ + private boolean failOnMissingField; + + private String fieldDelimiter; + + private String lengthCheckPolicy; + + public CustomerCsvDeserialization(TypeInformation typeInfo, String fieldDelimiter, String lengthCheckPolicy){ + this.typeInfo = typeInfo; + + this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); + + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + + this.fieldDelimiter = fieldDelimiter; + + this.lengthCheckPolicy = lengthCheckPolicy; + } + + @Override + public Row deserialize(byte[] message) throws IOException { + + try { + numInRecord.inc(); + numInBytes.inc(message.length); + String[] fieldsList = null; + if (message != null && message.length > 0){ + fieldsList = new String(message).split(fieldDelimiter); + } + if (fieldsList == null || fieldsList.length != fieldNames.length){//exception condition + if (lengthCheckPolicy.equalsIgnoreCase("SKIP")) { + return null; + }else if (lengthCheckPolicy.equalsIgnoreCase("EXCEPTION")) { + throw new RuntimeException("lengthCheckPolicy Error,message have "+fieldsList.length+" fields,sql have "+fieldNames.length); + } + } + + Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + if (i { + + private static final Logger LOG = LoggerFactory.getLogger(CustomerJsonDeserialization.class); + + private static final long serialVersionUID = 2385115520960444192L; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Type information describing the result type. */ + private final TypeInformation typeInfo; + + /** Field names to parse. Indices match fieldTypes indices. */ + private final String[] fieldNames; + + /** Types to parse fields as. Indices match fieldNames indices. */ + private final TypeInformation[] fieldTypes; + + /** Flag indicating whether to fail on a missing field. */ + private boolean failOnMissingField; + + private AbstractFetcher fetcher; + + public CustomerJsonDeserialization(TypeInformation typeInfo){ + this.typeInfo = typeInfo; + + this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); + + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + } + + @Override + public Row deserialize(byte[] message) throws IOException { + + try { + numInRecord.inc(); + numInBytes.inc(message.length); + + JsonNode root = objectMapper.readTree(message); + Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + JsonNode node = getIgnoreCase(root, fieldNames[i]); + + if (node == null) { + if (failOnMissingField) { + throw new IllegalStateException("Failed to find field with name '" + + fieldNames[i] + "'."); + } else { + row.setField(i, null); + } + } else { + // Read the value as specified type + Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + row.setField(i, value); + } + } + + numInResolveRecord.inc(); + return row; + } catch (Throwable t) { + //add metric of dirty data + LOG.error(t.getMessage()); + dirtyDataCounter.inc(); + return null; + } + } + + public void setFailOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; + } + + public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { + + Iterator iter = jsonNode.fieldNames(); + while (iter.hasNext()) { + String key1 = iter.next(); + if (key1.equalsIgnoreCase(key)) { + return jsonNode.get(key1); + } + } + + return null; + + } + + public void setFetcher(AbstractFetcher fetcher) { + this.fetcher = fetcher; + } + + + private static String partitionLagMetricName(TopicPartition tp) { + return tp + ".records-lag"; + } +} diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java new file mode 100644 index 000000000..668cbcfd9 --- /dev/null +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -0,0 +1,58 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.table; + +import com.dtstack.flink.sql.table.AbsSourceParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + + +public class KafkaSourceParser extends AbsSourceParser { + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + + KafkaSourceTableInfo kafka08SourceTableInfo = new KafkaSourceTableInfo(); + kafka08SourceTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka08SourceTableInfo); + kafka08SourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSourceTableInfo.PARALLELISM_KEY.toLowerCase()))); + + kafka08SourceTableInfo.setPatternTopic(MathUtil.getBoolean(props.get(KafkaSourceTableInfo.PATTERNTOPIC_KEY.toLowerCase()))); + + if (props.get(KafkaSourceTableInfo.SOURCE_DATA_TYPE) != null) { + kafka08SourceTableInfo.setSourceDataType(props.get(KafkaSourceTableInfo.SOURCE_DATA_TYPE).toString()); + } + if (props.get(KafkaSourceTableInfo.FIELD_DELINITER) != null) { + kafka08SourceTableInfo.setFieldDelimiter(props.get(KafkaSourceTableInfo.FIELD_DELINITER).toString()); + } + if (props.get(KafkaSourceTableInfo.LENGTH_CHECK_POLICY) != null) { + kafka08SourceTableInfo.setLengthCheckPolicy(props.get(KafkaSourceTableInfo.LENGTH_CHECK_POLICY).toString()); + } + for (String key:props.keySet()) { + if (!key.isEmpty() && key.startsWith("kafka.")) { + kafka08SourceTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + } + } + return kafka08SourceTableInfo; + } +} diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java new file mode 100644 index 000000000..4fb59f7d2 --- /dev/null +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -0,0 +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 com.dtstack.flink.sql.source.kafka.table; + +import com.dtstack.flink.sql.table.SourceTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + + +public class KafkaSourceTableInfo extends SourceTableInfo { + + //version + private static final String CURR_TYPE = "kafka08"; + + public static final String PATTERNTOPIC_KEY = "patterntopic"; + + private Boolean patternTopic=false; + + public Boolean getPatternTopic() { + return patternTopic; + } + + public void setPatternTopic(Boolean patternTopic) { + if (patternTopic==null){ + return; + } + this.patternTopic = patternTopic; + } + + public KafkaSourceTableInfo(){ + super.setType(CURR_TYPE); + } + + public static Map kafkaParam = new HashMap(); + + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); + } + + public String getKafkaParam(String key){ + return kafkaParam.get(key); + } + + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); + } + + @Override + public boolean check() { + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + Preconditions.checkState(kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("earliest") + || kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + + return false; + } + + @Override + public String getType() { + return super.getType(); + } +} diff --git a/kafka08/pom.xml b/kafka08/pom.xml new file mode 100644 index 000000000..ced039c1c --- /dev/null +++ b/kafka08/pom.xml @@ -0,0 +1,37 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.kafka08 + pom + + + kafka08-source + kafka08-sink + + + + + junit + junit + 3.8.1 + test + + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + + + \ No newline at end of file diff --git a/kafka09/kafka09-sink/pom.xml b/kafka09/kafka09-sink/pom.xml new file mode 100644 index 000000000..5d11d9df8 --- /dev/null +++ b/kafka09/kafka09-sink/pom.xml @@ -0,0 +1,90 @@ + + + + sql.kafka09 + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.kafka09 + jar + + kafka09-sink + http://maven.apache.org + + + + org.apache.flink + flink-json + ${flink.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java new file mode 100644 index 000000000..355c79645 --- /dev/null +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -0,0 +1,120 @@ +package com.dtstack.flink.sql.sink.kafka; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.shaded.org.apache.commons.lang.StringEscapeUtils; +import org.apache.flink.types.Row; +import org.apache.flink.types.StringValue; + +import java.io.IOException; + +import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; + +@Internal +public final class CustomerCsvSerialization extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + private String fieldDelimiter = "\u0001"; + private TypeInformation[] fieldTypes; + private TypeSerializer[] fieldSerializers; + private static final Row EMPTY = null; + + public CustomerCsvSerialization(String fielddelimiter,TypeInformation[] fieldTypes) { + this.fieldDelimiter = fielddelimiter; + this.fieldTypes = fieldTypes; + this.fieldSerializers = (TypeSerializer[])createSerializer(new ExecutionConfig()); + } + + public TypeSerializer[] createSerializer(ExecutionConfig config) { + int len = fieldTypes.length; + TypeSerializer[] fieldSerializers = new TypeSerializer[len]; + for (int i = 0; i < len; i++) { + fieldSerializers[i] = fieldTypes[i].createSerializer(config); + } + return fieldSerializers; + } + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + public Row createInstance() { + return EMPTY; + } + + @Override + public Row copy(Row from) { + return null; + } + + @Override + public Row copy(Row from, Row reuse) { + return null; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(Row record, DataOutputView target) throws IOException { + int len = fieldSerializers.length; + + if (record.getArity() != len) { + throw new RuntimeException("Row arity of from does not match serializers."); + } + + // write a null mask + writeNullMask(len, record, target); + + // serialize non-null fields + StringBuffer stringBuffer = new StringBuffer(); + for (int i = 0; i < len; i++) { + Object o = record.getField(i); + if (o != null) { + //fieldSerializers[i].serialize(o, target); + stringBuffer.append(o); + } + if(i != len-1){ + stringBuffer.append(StringEscapeUtils.unescapeJava(fieldDelimiter)); + //fieldSerializers[i].serialize(fieldDelimiter, target); + } + } + StringValue.writeString(stringBuffer.toString(), target); + } + + @Override + public Row deserialize(DataInputView source) throws IOException { + return null; + } + + @Override + public Row deserialize(Row reuse, DataInputView source) throws IOException { + return null; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + StringValue.copyString(source, target); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof CustomerCsvSerialization; + } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(StringValue.class.getCanonicalName()); + } +} diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java new file mode 100644 index 000000000..2500bad39 --- /dev/null +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -0,0 +1,133 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.connectors.kafka.Kafka09TableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.TableSchemaBuilder; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +import java.util.Optional; +import java.util.Properties; + +public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { + + protected String[] fieldNames; + + protected TypeInformation[] fieldTypes; + + /** The schema of the table. */ + private TableSchema schema; + + /** The Kafka topic to write to. */ + protected String topic; + + /** Properties for the Kafka producer. */ + protected Properties properties; + + /** Serialization schema for encoding records to Kafka. */ + protected SerializationSchema serializationSchema; + + /** Partitioner to select Kafka partition for each item. */ + protected Optional> partitioner; + + @Override + public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { + KafkaSinkTableInfo kafka09SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; + this.topic = kafka09SinkTableInfo.getKafkaParam("topic"); + + Properties props = new Properties(); + for (String key:kafka09SinkTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka09SinkTableInfo.getKafkaParam(key)); + } + this.properties = props; + this.partitioner = Optional.of(new FlinkFixedPartitioner<>()); + this.fieldNames = kafka09SinkTableInfo.getFields(); + TypeInformation[] types = new TypeInformation[kafka09SinkTableInfo.getFields().length]; + for(int i = 0; i< kafka09SinkTableInfo.getFieldClasses().length; i++){ + types[i] = TypeInformation.of(kafka09SinkTableInfo.getFieldClasses()[i]); + } + this.fieldTypes = types; + + TableSchemaBuilder schemaBuilder = TableSchema.builder(); + for (int i=0;i dataStream) { + KafkaTableSink kafkaTableSink = new Kafka09TableSink( + schema, + topic, + properties, + partitioner, + serializationSchema + ); + + kafkaTableSink.emitDataStream(dataStream); + } + + @Override + public TypeInformation getOutputType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + @Override + public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + +} diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java new file mode 100644 index 000000000..838f59541 --- /dev/null +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -0,0 +1,48 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka.table; + +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +public class KafkaSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + KafkaSinkTableInfo kafka11SinkTableInfo = new KafkaSinkTableInfo(); + kafka11SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka11SinkTableInfo); + kafka11SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); + if (props.get(KafkaSinkTableInfo.SINK_DATA_TYPE) != null) { + kafka11SinkTableInfo.setSourceDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); + } + if (props.get(KafkaSinkTableInfo.FIELD_DELINITER) != null) { + kafka11SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); + } + + for (String key:props.keySet()) { + if (!key.isEmpty() && key.startsWith("kafka.")) { + kafka11SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + } + } + return kafka11SinkTableInfo; + } +} diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java new file mode 100644 index 000000000..6643204a9 --- /dev/null +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.flink.sql.sink.kafka.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +public class KafkaSinkTableInfo extends TargetTableInfo { + //version + private static final String CURR_TYPE = "kafka11"; + + public KafkaSinkTableInfo(){ + super.setType(CURR_TYPE); + } + + public static Map kafkaParam = new HashMap(); + + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); + } + + public String getKafkaParam(String key){ + return kafkaParam.get(key); + } + + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); + } + + @Override + public boolean check() { + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + return false; + } + + @Override + public String getType() { + return super.getType(); + } +} diff --git a/kafka09/kafka09-source/pom.xml b/kafka09/kafka09-source/pom.xml index 59e05c020..6f26b62a0 100644 --- a/kafka09/kafka09-source/pom.xml +++ b/kafka09/kafka09-source/pom.xml @@ -14,16 +14,6 @@ kafka09-source http://maven.apache.org - - - - org.apache.flink - flink-connector-kafka-0.9_2.11 - ${flink.version} - - - - diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index b08485e0b..c37da6791 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -21,15 +21,20 @@ package com.dtstack.flink.sql.source.kafka; import com.dtstack.flink.sql.source.IStreamSourceGener; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerCommonConsumer; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerCsvConsumer; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerJsonConsumer; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; -import org.apache.commons.lang3.BooleanUtils; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; @@ -64,16 +69,15 @@ public class KafkaSource implements IStreamSourceGener

{ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { KafkaSourceTableInfo kafka09SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; - String topicName = kafka09SourceTableInfo.getTopic(); + String topicName = kafka09SourceTableInfo.getKafkaParam("topic"); + String offsetReset = kafka09SourceTableInfo.getKafkaParam("auto.offset.reset"); + Boolean topicIsPattern = kafka09SourceTableInfo.getPatternTopic(); Properties props = new Properties(); - props.setProperty("bootstrap.servers", kafka09SourceTableInfo.getBootstrapServers()); - props.setProperty("auto.offset.reset", kafka09SourceTableInfo.getOffsetReset()); - if (StringUtils.isNotBlank(kafka09SourceTableInfo.getGroupId())){ - props.setProperty("group.id", kafka09SourceTableInfo.getGroupId()); + + for (String key:kafka09SourceTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka09SourceTableInfo.getKafkaParam(key)); } - // only required for Kafka 0.8 - //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) TypeInformation[] types = new TypeInformation[kafka09SourceTableInfo.getFields().length]; for(int i = 0; i< kafka09SourceTableInfo.getFieldClasses().length; i++){ @@ -81,21 +85,44 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } TypeInformation typeInformation = new RowTypeInfo(types, kafka09SourceTableInfo.getFields()); + + FlinkKafkaConsumer09 kafkaSrc; - if (BooleanUtils.isTrue(kafka09SourceTableInfo.getTopicIsPattern())) { - kafkaSrc = new CustomerKafka09Consumer(Pattern.compile(topicName), - new CustomerJsonDeserialization(typeInformation), props); + String fields=StringUtils.join(kafka09SourceTableInfo.getFields(), ","); + + if ("json".equalsIgnoreCase(kafka09SourceTableInfo.getSourceDataType())) { + if (topicIsPattern) { + kafkaSrc = new CustomerJsonConsumer(Pattern.compile(topicName), + new com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization(typeInformation), props); + } else { + kafkaSrc = new CustomerJsonConsumer(topicName, + new CustomerJsonDeserialization(typeInformation), props); + } + } else if ("csv".equalsIgnoreCase(kafka09SourceTableInfo.getSourceDataType())) { + if (topicIsPattern) { + kafkaSrc = new CustomerCsvConsumer(topicName, + new CustomerCsvDeserialization(typeInformation, + kafka09SourceTableInfo.getFieldDelimiter(), kafka09SourceTableInfo.getLengthCheckPolicy()), props); + } else { + kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), + new CustomerCsvDeserialization(typeInformation, + kafka09SourceTableInfo.getFieldDelimiter(), kafka09SourceTableInfo.getLengthCheckPolicy()), props); + } } else { - kafkaSrc = new CustomerKafka09Consumer(topicName, - new CustomerJsonDeserialization(typeInformation), props); + if (topicIsPattern) { + kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); + } else { + kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); + } + } //earliest,latest - if("earliest".equalsIgnoreCase(kafka09SourceTableInfo.getOffsetReset())){ + if("earliest".equalsIgnoreCase(offsetReset)){ kafkaSrc.setStartFromEarliest(); - }else if(DtStringUtil.isJosn(kafka09SourceTableInfo.getOffsetReset())){// {"0":12312,"1":12321,"2":12312} + }else if(DtStringUtil.isJosn(offsetReset)){// {"0":12312,"1":12321,"2":12312} try { - Properties properties = PluginUtil.jsonStrToObject(kafka09SourceTableInfo.getOffsetReset(), Properties.class); + Properties properties = PluginUtil.jsonStrToObject(offsetReset, Properties.class); Map offsetMap = PluginUtil.ObjectToMap(properties); Map specificStartupOffsets = new HashMap<>(); for(Map.Entry entry:offsetMap.entrySet()){ @@ -103,14 +130,17 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); } catch (Exception e) { - throw new RuntimeException("not support offsetReset type:" + kafka09SourceTableInfo.getOffsetReset()); + throw new RuntimeException("not support offsetReset type:" + offsetReset); } }else { kafkaSrc.setStartFromLatest(); } - - String fields = StringUtils.join(kafka09SourceTableInfo.getFields(), ","); String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); - return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); + DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); + Integer parallelism = kafka09SourceTableInfo.getParallelism(); + if(parallelism != null){ + kafkaSource.setParallelism(parallelism); + } + return tableEnv.fromDataStream(kafkaSource, fields); } } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java new file mode 100644 index 000000000..46272b72c --- /dev/null +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java @@ -0,0 +1,74 @@ +/** + * 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 com.dtstack.flink.sql.source.kafka.consumer; + +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/12/17 + * Company: www.dtstack.com + * + * @author maqi + */ +public class CustomerCommonConsumer extends FlinkKafkaConsumer09 { + + private CustomerCommonDeserialization customerCommonDeserialization; + + + public CustomerCommonConsumer(String topic, KeyedDeserializationSchema deserializer, Properties props) { + super(topic, deserializer, props); + this.customerCommonDeserialization= (CustomerCommonDeserialization) deserializer; + } + + public CustomerCommonConsumer(Pattern subscriptionPattern, KeyedDeserializationSchema deserializer, Properties props) { + super(subscriptionPattern, deserializer, props); + this.customerCommonDeserialization= (CustomerCommonDeserialization) deserializer; + } + + + @Override + public void run(SourceFunction.SourceContext sourceContext) throws Exception { + customerCommonDeserialization.setRuntimeContext(getRuntimeContext()); + customerCommonDeserialization.initMetric(); + super.run(sourceContext); + } + + @Override + protected AbstractFetcher createFetcher(SourceFunction.SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); + customerCommonDeserialization.setFetcher(fetcher); + return fetcher; + } +} diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java new file mode 100644 index 000000000..28b374c43 --- /dev/null +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java @@ -0,0 +1,80 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.consumer; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/10/19 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class CustomerCsvConsumer extends FlinkKafkaConsumer09 { + + private static final long serialVersionUID = -2265366268827807739L; + + private CustomerCsvDeserialization customerCsvDeserialization; + + public CustomerCsvConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + super(Arrays.asList(topic.split(",")), valueDeserializer, props); + this.customerCsvDeserialization = (CustomerCsvDeserialization) valueDeserializer; + } + + public CustomerCsvConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); + this.customerCsvDeserialization = (CustomerCsvDeserialization) valueDeserializer; + } + + + + + @Override + public void run(SourceFunction.SourceContext sourceContext) throws Exception { + customerCsvDeserialization.setRuntimeContext(getRuntimeContext()); + customerCsvDeserialization.initMetric(); + super.run(sourceContext); + } + + @Override + protected AbstractFetcher createFetcher(SourceFunction.SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); + customerCsvDeserialization.setFetcher(fetcher); + return fetcher; + } +} diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java similarity index 69% rename from kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java rename to kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java index 09e684c75..55b1ba851 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka09Consumer.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java @@ -16,12 +16,14 @@ * limitations under the License. */ -package com.dtstack.flink.sql.source.kafka; +package com.dtstack.flink.sql.source.kafka.consumer; import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer09; import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; @@ -37,35 +39,38 @@ /** * Reason: - * Date: 2018/10/12 + * Date: 2018/10/19 * Company: www.dtstack.com + * * @author xuchao */ -public class CustomerKafka09Consumer extends FlinkKafkaConsumer09 { +public class CustomerJsonConsumer extends FlinkKafkaConsumer09 { - private static final long serialVersionUID = 4451177393982291909L; + private static final long serialVersionUID = -2265366268827807739L; private CustomerJsonDeserialization customerJsonDeserialization; - public CustomerKafka09Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + public CustomerJsonConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { super(Arrays.asList(topic.split(",")), valueDeserializer, props); this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } - public CustomerKafka09Consumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + + public CustomerJsonConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { super(subscriptionPattern, valueDeserializer, props); this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } + @Override - public void run(SourceContext sourceContext) throws Exception { + public void run(SourceFunction.SourceContext sourceContext) throws Exception { customerJsonDeserialization.setRuntimeContext(getRuntimeContext()); customerJsonDeserialization.initMetric(); super.run(sourceContext); } @Override - protected AbstractFetcher createFetcher(SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + protected AbstractFetcher createFetcher(SourceFunction.SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); customerJsonDeserialization.setFetcher(fetcher); return fetcher; diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java new file mode 100644 index 000000000..cc23e69cf --- /dev/null +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -0,0 +1,150 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.deserialization; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.Set; + +import static com.dtstack.flink.sql.metric.MetricConstant.*; +import static java.nio.charset.StandardCharsets.UTF_8; + +/** + * Date: 2017/5/28 + * + * @author DocLi + */ +public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { + private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); + + public static final String[] KAFKA_COLUMNS = new String[]{"_TOPIC", "_MESSAGEKEY", "_MESSAGE", "_PARTITION", "_OFFSET"}; + + private AbstractFetcher fetcher; + + private boolean firstMsg = true; + + @Override + public Row deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) { + if (firstMsg) { + try { + registerPtMetric(fetcher); + } catch (Exception e) { + LOG.error("register topic partition metric error.", e); + } + firstMsg = false; + } + + numInRecord.inc(); + numInBytes.inc(message.length); + numInBytes.inc(messageKey.length); + + try { + Row row = Row.of( + topic, //topic + messageKey == null ? null : new String(messageKey, UTF_8), //key + new String(message, UTF_8), //message + partition, + offset + ); + return row; + } catch (Throwable t) { + LOG.error(t.getMessage()); + dirtyDataCounter.inc(); + return null; + } + } + + @Override + public Row deserialize(byte[] message) throws IOException { + return null; + } + + public void setFetcher(AbstractFetcher fetcher) { + this.fetcher = fetcher; + } + + + @Override + public boolean isEndOfStream(Row nextElement) { + return false; + } + + public TypeInformation getProducedType() { + TypeInformation[] types = new TypeInformation[]{ + TypeExtractor.createTypeInfo(String.class), + TypeExtractor.createTypeInfo(String.class), //createTypeInformation[String] + TypeExtractor.createTypeInfo(String.class), + Types.INT, + Types.LONG + }; + return new RowTypeInfo(types, KAFKA_COLUMNS); + } + + protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + + Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + consumerThreadField.setAccessible(true); + KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); + + Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); + hasAssignedPartitionsField.setAccessible(true); + + //wait until assignedPartitions + + boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); + + if (!hasAssignedPartitions) { + throw new RuntimeException("wait 50 secs, but not assignedPartitions"); + } + + Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); + consumerField.setAccessible(true); + + KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); + Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); + subscriptionStateField.setAccessible(true); + + //topic partitions lag + SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); + Set assignedPartitions = subscriptionState.assignedPartitions(); + for (TopicPartition topicPartition : assignedPartitions) { + MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) + .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); + metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); + } + + } +} diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java new file mode 100644 index 000000000..38b874ed9 --- /dev/null +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -0,0 +1,181 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.deserialization; + + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.Set; + +import static com.dtstack.flink.sql.metric.MetricConstant.*; + +/** + * json string parsing custom + * Date: 2017/5/28 + * Company: www.dtstack.com + * @author DocLi + */ + +public class CustomerCsvDeserialization extends AbsDeserialization { + + private static final Logger LOG = LoggerFactory.getLogger(CustomerCsvDeserialization.class); + + private static final long serialVersionUID = -2706012724306826506L; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Type information describing the result type. */ + private final TypeInformation typeInfo; + + /** Field names to parse. Indices match fieldTypes indices. */ + private final String[] fieldNames; + + /** Types to parse fields as. Indices match fieldNames indices. */ + private final TypeInformation[] fieldTypes; + + /** Flag indicating whether to fail on a missing field. */ + private boolean failOnMissingField; + + private AbstractFetcher fetcher; + + private boolean firstMsg = true; + + private String fieldDelimiter; + + private String lengthCheckPolicy; + + public CustomerCsvDeserialization(TypeInformation typeInfo, String fieldDelimiter, String lengthCheckPolicy){ + this.typeInfo = typeInfo; + + this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); + + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + + this.fieldDelimiter = fieldDelimiter; + + this.lengthCheckPolicy = lengthCheckPolicy; + } + + @Override + public Row deserialize(byte[] message) throws IOException { + if(firstMsg){ + try { + registerPtMetric(fetcher); + } catch (Exception e) { + LOG.error("register topic partition metric error.", e); + } + + firstMsg = false; + } + + try { + numInRecord.inc(); + numInBytes.inc(message.length); + String[] fieldsList = null; + if (message != null && message.length > 0){ + fieldsList = new String(message).split(fieldDelimiter); + } + if (fieldsList == null || fieldsList.length != fieldNames.length){//exception condition + if (lengthCheckPolicy.equalsIgnoreCase("SKIP")) { + return null; + }else if (lengthCheckPolicy.equalsIgnoreCase("EXCEPTION")) { + throw new RuntimeException("lengthCheckPolicy Error,message have "+fieldsList.length+" fields,sql have "+fieldNames.length); + } + } + + Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + if (i fetcher) { + this.fetcher = fetcher; + } + + protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + + Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + consumerThreadField.setAccessible(true); + KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); + + Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); + hasAssignedPartitionsField.setAccessible(true); + + //wait until assignedPartitions + + boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); + + if(!hasAssignedPartitions){ + throw new RuntimeException("wait 50 secs, but not assignedPartitions"); + } + + Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); + consumerField.setAccessible(true); + + KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); + Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); + subscriptionStateField.setAccessible(true); + + //topic partitions lag + SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); + Set assignedPartitions = subscriptionState.assignedPartitions(); + for(TopicPartition topicPartition : assignedPartitions){ + MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) + .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); + metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); + } + + } + + public void setFailOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; + } + +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java similarity index 98% rename from kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java rename to kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java index bcb54159a..24b607ee1 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java @@ -18,7 +18,7 @@ -package com.dtstack.flink.sql.source.kafka; +package com.dtstack.flink.sql.source.kafka.deserialization; import com.dtstack.flink.sql.source.AbsDeserialization; @@ -123,6 +123,7 @@ public Row deserialize(byte[] message) throws IOException { return row; } catch (Throwable t) { //add metric of dirty data + LOG.error(t.getMessage()); dirtyDataCounter.inc(); return null; } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 8800dc18e..0b7edcdaf 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -41,12 +41,24 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map kafkaParam = new HashMap(); - public void setBootstrapServers(String bootstrapServers) { - this.bootstrapServers = bootstrapServers; - } - public String getTopic() { - return topic; + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); } - public void setTopic(String topic) { - this.topic = topic; + public String getKafkaParam(String key){ + return kafkaParam.get(key); } - public String getGroupId() { - return groupId; + public Boolean getKafkaBooleanParam(String key){ + return Boolean.valueOf(kafkaParam.getOrDefault(key,"false").toLowerCase()); } - - public void setGroupId(String groupId) { - this.groupId = groupId; - } - - public String getOffsetReset() { - return offsetReset; - } - - public void setOffsetReset(String offsetReset) { - if (offsetReset == null) { - return; - } - - this.offsetReset = offsetReset; - } - - public String getOffset() { - return offset; - } - - public void setOffset(String offset) { - this.offset = offset; + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); } @Override public boolean check() { - Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); - Preconditions.checkNotNull(topic, "kafka of topic is required"); - //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); - Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); - + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + //Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); + Preconditions.checkState(kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("earliest") + || kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); return false; } @Override public String getType() { -// return super.getType() + SOURCE_SUFFIX; return super.getType(); } } diff --git a/kafka09/pom.xml b/kafka09/pom.xml index 457a8d387..1ae761e9b 100644 --- a/kafka09/pom.xml +++ b/kafka09/pom.xml @@ -15,9 +15,17 @@ kafka09-source + kafka09-sink + + + org.apache.flink + flink-connector-kafka-0.9_2.11 + ${flink.version} + + junit junit diff --git a/kafka10/kafka10-sink/pom.xml b/kafka10/kafka10-sink/pom.xml new file mode 100644 index 000000000..3a8b7592e --- /dev/null +++ b/kafka10/kafka10-sink/pom.xml @@ -0,0 +1,89 @@ + + + + sql.kafka10 + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.kafka10 + jar + + kafka10-sink + http://maven.apache.org + + + + org.apache.flink + flink-json + ${flink.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java new file mode 100644 index 000000000..355c79645 --- /dev/null +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -0,0 +1,120 @@ +package com.dtstack.flink.sql.sink.kafka; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.shaded.org.apache.commons.lang.StringEscapeUtils; +import org.apache.flink.types.Row; +import org.apache.flink.types.StringValue; + +import java.io.IOException; + +import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; + +@Internal +public final class CustomerCsvSerialization extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + private String fieldDelimiter = "\u0001"; + private TypeInformation[] fieldTypes; + private TypeSerializer[] fieldSerializers; + private static final Row EMPTY = null; + + public CustomerCsvSerialization(String fielddelimiter,TypeInformation[] fieldTypes) { + this.fieldDelimiter = fielddelimiter; + this.fieldTypes = fieldTypes; + this.fieldSerializers = (TypeSerializer[])createSerializer(new ExecutionConfig()); + } + + public TypeSerializer[] createSerializer(ExecutionConfig config) { + int len = fieldTypes.length; + TypeSerializer[] fieldSerializers = new TypeSerializer[len]; + for (int i = 0; i < len; i++) { + fieldSerializers[i] = fieldTypes[i].createSerializer(config); + } + return fieldSerializers; + } + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + public Row createInstance() { + return EMPTY; + } + + @Override + public Row copy(Row from) { + return null; + } + + @Override + public Row copy(Row from, Row reuse) { + return null; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(Row record, DataOutputView target) throws IOException { + int len = fieldSerializers.length; + + if (record.getArity() != len) { + throw new RuntimeException("Row arity of from does not match serializers."); + } + + // write a null mask + writeNullMask(len, record, target); + + // serialize non-null fields + StringBuffer stringBuffer = new StringBuffer(); + for (int i = 0; i < len; i++) { + Object o = record.getField(i); + if (o != null) { + //fieldSerializers[i].serialize(o, target); + stringBuffer.append(o); + } + if(i != len-1){ + stringBuffer.append(StringEscapeUtils.unescapeJava(fieldDelimiter)); + //fieldSerializers[i].serialize(fieldDelimiter, target); + } + } + StringValue.writeString(stringBuffer.toString(), target); + } + + @Override + public Row deserialize(DataInputView source) throws IOException { + return null; + } + + @Override + public Row deserialize(Row reuse, DataInputView source) throws IOException { + return null; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + StringValue.copyString(source, target); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof CustomerCsvSerialization; + } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(StringValue.class.getCanonicalName()); + } +} diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java new file mode 100644 index 000000000..5f0927960 --- /dev/null +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -0,0 +1,133 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.connectors.kafka.Kafka010TableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.TableSchemaBuilder; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +import java.util.Optional; +import java.util.Properties; + +public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { + + protected String[] fieldNames; + + protected TypeInformation[] fieldTypes; + + /** The schema of the table. */ + private TableSchema schema; + + /** The Kafka topic to write to. */ + protected String topic; + + /** Properties for the Kafka producer. */ + protected Properties properties; + + /** Serialization schema for encoding records to Kafka. */ + protected SerializationSchema serializationSchema; + + /** Partitioner to select Kafka partition for each item. */ + protected Optional> partitioner; + + @Override + public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { + KafkaSinkTableInfo kafka010SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; + this.topic = kafka010SinkTableInfo.getKafkaParam("topic"); + + Properties props = new Properties(); + for (String key:kafka010SinkTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka010SinkTableInfo.getKafkaParam(key)); + } + this.properties = props; + this.partitioner = Optional.of(new FlinkFixedPartitioner<>()); + this.fieldNames = kafka010SinkTableInfo.getFields(); + TypeInformation[] types = new TypeInformation[kafka010SinkTableInfo.getFields().length]; + for(int i = 0; i< kafka010SinkTableInfo.getFieldClasses().length; i++){ + types[i] = TypeInformation.of(kafka010SinkTableInfo.getFieldClasses()[i]); + } + this.fieldTypes = types; + + TableSchemaBuilder schemaBuilder = TableSchema.builder(); + for (int i=0;i dataStream) { + KafkaTableSink kafkaTableSink = new Kafka010TableSink( + schema, + topic, + properties, + partitioner, + serializationSchema + ); + + kafkaTableSink.emitDataStream(dataStream); + } + + @Override + public TypeInformation getOutputType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + @Override + public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + +} diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java new file mode 100644 index 000000000..838f59541 --- /dev/null +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -0,0 +1,48 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka.table; + +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +public class KafkaSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + KafkaSinkTableInfo kafka11SinkTableInfo = new KafkaSinkTableInfo(); + kafka11SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka11SinkTableInfo); + kafka11SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); + if (props.get(KafkaSinkTableInfo.SINK_DATA_TYPE) != null) { + kafka11SinkTableInfo.setSourceDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); + } + if (props.get(KafkaSinkTableInfo.FIELD_DELINITER) != null) { + kafka11SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); + } + + for (String key:props.keySet()) { + if (!key.isEmpty() && key.startsWith("kafka.")) { + kafka11SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + } + } + return kafka11SinkTableInfo; + } +} diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java new file mode 100644 index 000000000..15a29cde3 --- /dev/null +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.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 com.dtstack.flink.sql.sink.kafka.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +public class KafkaSinkTableInfo extends TargetTableInfo { + //version + private static final String CURR_TYPE = "kafka11"; + + public KafkaSinkTableInfo(){ + super.setType(CURR_TYPE); + } + + public static Map kafkaParam = new HashMap(); + + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); + } + + public String getKafkaParam(String key){ + return kafkaParam.get(key); + } + + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); + } + + @Override + public boolean check() { + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + //Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); + return false; + } + + @Override + public String getType() { + return super.getType(); + } +} diff --git a/kafka10/kafka10-source/pom.xml b/kafka10/kafka10-source/pom.xml index 365f98048..0f0c55c36 100644 --- a/kafka10/kafka10-source/pom.xml +++ b/kafka10/kafka10-source/pom.xml @@ -13,17 +13,6 @@ kafka10-source http://maven.apache.org - - - - org.apache.flink - flink-connector-kafka-0.10_2.11 - ${flink.version} - - - - - diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 523eb25dc..36c1e5e6e 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -20,14 +20,20 @@ package com.dtstack.flink.sql.source.kafka; import com.dtstack.flink.sql.source.IStreamSourceGener; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerCommonConsumer; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerCsvConsumer; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerJsonConsumer; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; -import org.apache.commons.lang3.BooleanUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; @@ -63,40 +69,63 @@ public class KafkaSource implements IStreamSourceGener

{ @Override public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { + KafkaSourceTableInfo kafka010SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; - String topicName = kafka010SourceTableInfo.getTopic(); + String topicName = kafka010SourceTableInfo.getKafkaParam("topic"); + String offsetReset = kafka010SourceTableInfo.getKafkaParam("auto.offset.reset"); + Boolean topicIsPattern = kafka010SourceTableInfo.getPatternTopic(); Properties props = new Properties(); - props.setProperty("bootstrap.servers", kafka010SourceTableInfo.getBootstrapServers()); - props.setProperty("auto.offset.reset", kafka010SourceTableInfo.getOffsetReset()); - if (StringUtils.isNotBlank(kafka010SourceTableInfo.getGroupId())){ - props.setProperty("group.id", kafka010SourceTableInfo.getGroupId()); + for (String key:kafka010SourceTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka010SourceTableInfo.getKafkaParam(key)); } - // only required for Kafka 0.8 - //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) TypeInformation[] types = new TypeInformation[kafka010SourceTableInfo.getFields().length]; - for (int i = 0; i < kafka010SourceTableInfo.getFieldClasses().length; i++) { + + for(int i = 0; i< kafka010SourceTableInfo.getFieldClasses().length; i++){ types[i] = TypeInformation.of(kafka010SourceTableInfo.getFieldClasses()[i]); } TypeInformation typeInformation = new RowTypeInfo(types, kafka010SourceTableInfo.getFields()); FlinkKafkaConsumer010 kafkaSrc; - if (BooleanUtils.isTrue(kafka010SourceTableInfo.getTopicIsPattern())) { - kafkaSrc = new CustomerKafka010Consumer(Pattern.compile(topicName), - new CustomerJsonDeserialization(typeInformation), props); + + String fields= StringUtils.join(kafka010SourceTableInfo.getFields(), ","); + + + if ("json".equalsIgnoreCase(kafka010SourceTableInfo.getSourceDataType())) { + if (topicIsPattern) { + kafkaSrc = new CustomerJsonConsumer(Pattern.compile(topicName), + new CustomerJsonDeserialization(typeInformation), props); + } else { + kafkaSrc = new CustomerJsonConsumer(topicName, + new CustomerJsonDeserialization(typeInformation), props); + } + } else if ("csv".equalsIgnoreCase(kafka010SourceTableInfo.getSourceDataType())) { + if (topicIsPattern) { + kafkaSrc = new CustomerCsvConsumer(topicName, + new CustomerCsvDeserialization(typeInformation, + kafka010SourceTableInfo.getFieldDelimiter(), kafka010SourceTableInfo.getLengthCheckPolicy()), props); + } else { + kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), + new CustomerCsvDeserialization(typeInformation, + kafka010SourceTableInfo.getFieldDelimiter(), kafka010SourceTableInfo.getLengthCheckPolicy()), props); + } } else { - kafkaSrc = new CustomerKafka010Consumer(topicName, - new CustomerJsonDeserialization(typeInformation), props); + if (topicIsPattern) { + kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); + } else { + kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); + } + } //earliest,latest - if ("earliest".equalsIgnoreCase(kafka010SourceTableInfo.getOffsetReset())) { + if("earliest".equalsIgnoreCase(offsetReset)){ kafkaSrc.setStartFromEarliest(); - } else if (DtStringUtil.isJosn(kafka010SourceTableInfo.getOffsetReset())) {// {"0":12312,"1":12321,"2":12312} + } else if (DtStringUtil.isJosn(offsetReset)) {// {"0":12312,"1":12321,"2":12312} try { - Properties properties = PluginUtil.jsonStrToObject(kafka010SourceTableInfo.getOffsetReset(), Properties.class); + Properties properties = PluginUtil.jsonStrToObject(offsetReset, Properties.class); Map offsetMap = PluginUtil.ObjectToMap(properties); Map specificStartupOffsets = new HashMap<>(); for (Map.Entry entry : offsetMap.entrySet()) { @@ -104,14 +133,18 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); } catch (Exception e) { - throw new RuntimeException("not support offsetReset type:" + kafka010SourceTableInfo.getOffsetReset()); + throw new RuntimeException("not support offsetReset type:" + offsetReset); } - } else { + }else{ kafkaSrc.setStartFromLatest(); } - String fields = StringUtils.join(kafka010SourceTableInfo.getFields(), ","); String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); - return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); + DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); + Integer parallelism = kafka010SourceTableInfo.getParallelism(); + if(parallelism != null){ + kafkaSource.setParallelism(parallelism); + } + return tableEnv.fromDataStream(kafkaSource, fields); } } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java new file mode 100644 index 000000000..6faf68b1b --- /dev/null +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java @@ -0,0 +1,74 @@ +/** + * 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 com.dtstack.flink.sql.source.kafka.consumer; + +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/12/17 + * Company: www.dtstack.com + * + * @author maqi + */ +public class CustomerCommonConsumer extends FlinkKafkaConsumer010 { + + private CustomerCommonDeserialization customerCommonDeserialization; + + + public CustomerCommonConsumer(String topic, KeyedDeserializationSchema deserializer, Properties props) { + super(topic, deserializer, props); + this.customerCommonDeserialization= (CustomerCommonDeserialization) deserializer; + } + + public CustomerCommonConsumer(Pattern subscriptionPattern, KeyedDeserializationSchema deserializer, Properties props) { + super(subscriptionPattern, deserializer, props); + this.customerCommonDeserialization= (CustomerCommonDeserialization) deserializer; + } + + + @Override + public void run(SourceFunction.SourceContext sourceContext) throws Exception { + customerCommonDeserialization.setRuntimeContext(getRuntimeContext()); + customerCommonDeserialization.initMetric(); + super.run(sourceContext); + } + + @Override + protected AbstractFetcher createFetcher(SourceFunction.SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); + customerCommonDeserialization.setFetcher(fetcher); + return fetcher; + } +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java new file mode 100644 index 000000000..cc5916849 --- /dev/null +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java @@ -0,0 +1,80 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.consumer; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/10/19 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class CustomerCsvConsumer extends FlinkKafkaConsumer010 { + + private static final long serialVersionUID = -2265366268827807739L; + + private CustomerCsvDeserialization customerCsvDeserialization; + + public CustomerCsvConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + super(Arrays.asList(topic.split(",")), valueDeserializer, props); + this.customerCsvDeserialization = (CustomerCsvDeserialization) valueDeserializer; + } + + public CustomerCsvConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); + this.customerCsvDeserialization = (CustomerCsvDeserialization) valueDeserializer; + } + + + + + @Override + public void run(SourceFunction.SourceContext sourceContext) throws Exception { + customerCsvDeserialization.setRuntimeContext(getRuntimeContext()); + customerCsvDeserialization.initMetric(); + super.run(sourceContext); + } + + @Override + protected AbstractFetcher createFetcher(SourceFunction.SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); + customerCsvDeserialization.setFetcher(fetcher); + return fetcher; + } +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java new file mode 100644 index 000000000..e41337885 --- /dev/null +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java @@ -0,0 +1,78 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.consumer; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/10/19 + * Company: www.dtstack.com + * + * @author xuchao + */ + +public class CustomerJsonConsumer extends FlinkKafkaConsumer010 { + + private static final long serialVersionUID = -2265366268827807739L; + + private CustomerJsonDeserialization customerJsonDeserialization; + + public CustomerJsonConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + super(Arrays.asList(topic.split(",")), valueDeserializer, props); + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + } + + public CustomerJsonConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + } + + + @Override + public void run(SourceFunction.SourceContext sourceContext) throws Exception { + customerJsonDeserialization.setRuntimeContext(getRuntimeContext()); + customerJsonDeserialization.initMetric(); + super.run(sourceContext); + } + + @Override + protected AbstractFetcher createFetcher(SourceFunction.SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); + customerJsonDeserialization.setFetcher(fetcher); + return fetcher; + } +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java new file mode 100644 index 000000000..cc23e69cf --- /dev/null +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -0,0 +1,150 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.deserialization; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.Set; + +import static com.dtstack.flink.sql.metric.MetricConstant.*; +import static java.nio.charset.StandardCharsets.UTF_8; + +/** + * Date: 2017/5/28 + * + * @author DocLi + */ +public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { + private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); + + public static final String[] KAFKA_COLUMNS = new String[]{"_TOPIC", "_MESSAGEKEY", "_MESSAGE", "_PARTITION", "_OFFSET"}; + + private AbstractFetcher fetcher; + + private boolean firstMsg = true; + + @Override + public Row deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) { + if (firstMsg) { + try { + registerPtMetric(fetcher); + } catch (Exception e) { + LOG.error("register topic partition metric error.", e); + } + firstMsg = false; + } + + numInRecord.inc(); + numInBytes.inc(message.length); + numInBytes.inc(messageKey.length); + + try { + Row row = Row.of( + topic, //topic + messageKey == null ? null : new String(messageKey, UTF_8), //key + new String(message, UTF_8), //message + partition, + offset + ); + return row; + } catch (Throwable t) { + LOG.error(t.getMessage()); + dirtyDataCounter.inc(); + return null; + } + } + + @Override + public Row deserialize(byte[] message) throws IOException { + return null; + } + + public void setFetcher(AbstractFetcher fetcher) { + this.fetcher = fetcher; + } + + + @Override + public boolean isEndOfStream(Row nextElement) { + return false; + } + + public TypeInformation getProducedType() { + TypeInformation[] types = new TypeInformation[]{ + TypeExtractor.createTypeInfo(String.class), + TypeExtractor.createTypeInfo(String.class), //createTypeInformation[String] + TypeExtractor.createTypeInfo(String.class), + Types.INT, + Types.LONG + }; + return new RowTypeInfo(types, KAFKA_COLUMNS); + } + + protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + + Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + consumerThreadField.setAccessible(true); + KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); + + Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); + hasAssignedPartitionsField.setAccessible(true); + + //wait until assignedPartitions + + boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); + + if (!hasAssignedPartitions) { + throw new RuntimeException("wait 50 secs, but not assignedPartitions"); + } + + Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); + consumerField.setAccessible(true); + + KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); + Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); + subscriptionStateField.setAccessible(true); + + //topic partitions lag + SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); + Set assignedPartitions = subscriptionState.assignedPartitions(); + for (TopicPartition topicPartition : assignedPartitions) { + MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) + .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); + metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); + } + + } +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java new file mode 100644 index 000000000..38b874ed9 --- /dev/null +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -0,0 +1,181 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.deserialization; + + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.Set; + +import static com.dtstack.flink.sql.metric.MetricConstant.*; + +/** + * json string parsing custom + * Date: 2017/5/28 + * Company: www.dtstack.com + * @author DocLi + */ + +public class CustomerCsvDeserialization extends AbsDeserialization { + + private static final Logger LOG = LoggerFactory.getLogger(CustomerCsvDeserialization.class); + + private static final long serialVersionUID = -2706012724306826506L; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Type information describing the result type. */ + private final TypeInformation typeInfo; + + /** Field names to parse. Indices match fieldTypes indices. */ + private final String[] fieldNames; + + /** Types to parse fields as. Indices match fieldNames indices. */ + private final TypeInformation[] fieldTypes; + + /** Flag indicating whether to fail on a missing field. */ + private boolean failOnMissingField; + + private AbstractFetcher fetcher; + + private boolean firstMsg = true; + + private String fieldDelimiter; + + private String lengthCheckPolicy; + + public CustomerCsvDeserialization(TypeInformation typeInfo, String fieldDelimiter, String lengthCheckPolicy){ + this.typeInfo = typeInfo; + + this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); + + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + + this.fieldDelimiter = fieldDelimiter; + + this.lengthCheckPolicy = lengthCheckPolicy; + } + + @Override + public Row deserialize(byte[] message) throws IOException { + if(firstMsg){ + try { + registerPtMetric(fetcher); + } catch (Exception e) { + LOG.error("register topic partition metric error.", e); + } + + firstMsg = false; + } + + try { + numInRecord.inc(); + numInBytes.inc(message.length); + String[] fieldsList = null; + if (message != null && message.length > 0){ + fieldsList = new String(message).split(fieldDelimiter); + } + if (fieldsList == null || fieldsList.length != fieldNames.length){//exception condition + if (lengthCheckPolicy.equalsIgnoreCase("SKIP")) { + return null; + }else if (lengthCheckPolicy.equalsIgnoreCase("EXCEPTION")) { + throw new RuntimeException("lengthCheckPolicy Error,message have "+fieldsList.length+" fields,sql have "+fieldNames.length); + } + } + + Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + if (i fetcher) { + this.fetcher = fetcher; + } + + protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + + Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + consumerThreadField.setAccessible(true); + KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); + + Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); + hasAssignedPartitionsField.setAccessible(true); + + //wait until assignedPartitions + + boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); + + if(!hasAssignedPartitions){ + throw new RuntimeException("wait 50 secs, but not assignedPartitions"); + } + + Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); + consumerField.setAccessible(true); + + KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); + Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); + subscriptionStateField.setAccessible(true); + + //topic partitions lag + SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); + Set assignedPartitions = subscriptionState.assignedPartitions(); + for(TopicPartition topicPartition : assignedPartitions){ + MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) + .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); + metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); + } + + } + + public void setFailOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; + } + +} diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java similarity index 98% rename from kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java rename to kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java index ec369b28f..24b607ee1 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java @@ -16,9 +16,9 @@ * limitations under the License. */ + - -package com.dtstack.flink.sql.source.kafka; +package com.dtstack.flink.sql.source.kafka.deserialization; import com.dtstack.flink.sql.source.AbsDeserialization; @@ -123,6 +123,7 @@ public Row deserialize(byte[] message) throws IOException { return row; } catch (Throwable t) { //add metric of dirty data + LOG.error(t.getMessage()); dirtyDataCounter.inc(); return null; } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index ed89d8ce4..acc2f795d 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -41,12 +41,24 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map kafkaParam = new HashMap(); - public void setGroupId(String groupId) { - this.groupId = groupId; - } - public String getOffsetReset() { - return offsetReset; + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); } - public void setOffsetReset(String offsetReset) { - if(offsetReset == null){ - return; - } - this.offsetReset = offsetReset; + public String getKafkaParam(String key){ + return kafkaParam.get(key); } - public String getOffset() { - return offset; + public Boolean getKafkaBooleanParam(String key){ + return Boolean.valueOf(kafkaParam.getOrDefault(key,"false").toLowerCase()); } - public void setOffset(String offset) { - if (offsetReset == null) { - return; - } - this.offset = offset; + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); } - public Boolean getTopicIsPattern() { - return topicIsPattern; - } - public void setTopicIsPattern(Boolean topicIsPattern) { - this.topicIsPattern = topicIsPattern; - } @Override public boolean check() { - Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); - Preconditions.checkNotNull(topic, "kafka of topic is required"); - //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); - Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + //Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); + Preconditions.checkState(kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("earliest") + || kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); return false; } @Override public String getType() { -// return super.getType() + SOURCE_SUFFIX; return super.getType(); } } diff --git a/kafka10/pom.xml b/kafka10/pom.xml index 9c8c78bba..2f85f5c3b 100644 --- a/kafka10/pom.xml +++ b/kafka10/pom.xml @@ -14,9 +14,16 @@ kafka10-source + kafka10-sink + + org.apache.flink + flink-connector-kafka-0.10_2.11 + ${flink.version} + + junit junit diff --git a/kafka11/kafka11-sink/pom.xml b/kafka11/kafka11-sink/pom.xml new file mode 100644 index 000000000..4c748c90b --- /dev/null +++ b/kafka11/kafka11-sink/pom.xml @@ -0,0 +1,89 @@ + + + + sql.kafka11 + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.kafka11 + 1.0-SNAPSHOT + kafka11-sink + jar + + + + + org.apache.flink + flink-json + ${flink.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java new file mode 100644 index 000000000..4f3715923 --- /dev/null +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -0,0 +1,137 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.shaded.org.apache.commons.lang.StringEscapeUtils; +import org.apache.flink.types.Row; +import org.apache.flink.types.StringValue; + +import java.io.IOException; + +import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; + +@Internal +public final class CustomerCsvSerialization extends TypeSerializerSingleton { + + private static final long serialVersionUID = 1L; + + private String fieldDelimiter = "\u0001"; + private TypeInformation[] fieldTypes; + private TypeSerializer[] fieldSerializers; + private static final Row EMPTY = null; + + public CustomerCsvSerialization(String fielddelimiter,TypeInformation[] fieldTypes) { + this.fieldDelimiter = fielddelimiter; + this.fieldTypes = fieldTypes; + this.fieldSerializers = (TypeSerializer[])createSerializer(new ExecutionConfig()); + } + + public TypeSerializer[] createSerializer(ExecutionConfig config) { + int len = fieldTypes.length; + TypeSerializer[] fieldSerializers = new TypeSerializer[len]; + for (int i = 0; i < len; i++) { + fieldSerializers[i] = fieldTypes[i].createSerializer(config); + } + return fieldSerializers; + } + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + public Row createInstance() { + return EMPTY; + } + + @Override + public Row copy(Row from) { + return null; + } + + @Override + public Row copy(Row from, Row reuse) { + return null; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(Row record, DataOutputView target) throws IOException { + int len = fieldSerializers.length; + + if (record.getArity() != len) { + throw new RuntimeException("Row arity of from does not match serializers."); + } + + // write a null mask + writeNullMask(len, record, target); + + // serialize non-null fields + StringBuffer stringBuffer = new StringBuffer(); + for (int i = 0; i < len; i++) { + Object o = record.getField(i); + if (o != null) { + //fieldSerializers[i].serialize(o, target); + stringBuffer.append(o); + } + if(i != len-1){ + stringBuffer.append(StringEscapeUtils.unescapeJava(fieldDelimiter)); + //fieldSerializers[i].serialize(fieldDelimiter, target); + } + } + StringValue.writeString(stringBuffer.toString(), target); + } + + @Override + public Row deserialize(DataInputView source) throws IOException { + return null; + } + + @Override + public Row deserialize(Row reuse, DataInputView source) throws IOException { + return null; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + StringValue.copyString(source, target); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof CustomerCsvSerialization; + } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(StringValue.class.getCanonicalName()); + } +} diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java new file mode 100644 index 000000000..cc13ac79e --- /dev/null +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -0,0 +1,134 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; + +import org.apache.flink.streaming.connectors.kafka.Kafka011TableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.TableSchemaBuilder; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +import java.util.Optional; +import java.util.Properties; + +public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { + + protected String[] fieldNames; + + protected TypeInformation[] fieldTypes; + + /** The schema of the table. */ + private TableSchema schema; + + /** The Kafka topic to write to. */ + protected String topic; + + /** Properties for the Kafka producer. */ + protected Properties properties; + + /** Serialization schema for encoding records to Kafka. */ + protected SerializationSchema serializationSchema; + + /** Partitioner to select Kafka partition for each item. */ + protected Optional> partitioner; + + @Override + public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { + KafkaSinkTableInfo kafka011SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; + this.topic = kafka011SinkTableInfo.getKafkaParam("topic"); + + Properties props = new Properties(); + for (String key:kafka011SinkTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka011SinkTableInfo.getKafkaParam(key)); + } + this.properties = props; + this.partitioner = Optional.of(new FlinkFixedPartitioner<>()); + this.fieldNames = kafka011SinkTableInfo.getFields(); + TypeInformation[] types = new TypeInformation[kafka011SinkTableInfo.getFields().length]; + for(int i = 0; i< kafka011SinkTableInfo.getFieldClasses().length; i++){ + types[i] = TypeInformation.of(kafka011SinkTableInfo.getFieldClasses()[i]); + } + this.fieldTypes = types; + + TableSchemaBuilder schemaBuilder = TableSchema.builder(); + for (int i=0;i dataStream) { + KafkaTableSink kafkaTableSink = new Kafka011TableSink( + schema, + topic, + properties, + partitioner, + serializationSchema + ); + + kafkaTableSink.emitDataStream(dataStream); + } + + @Override + public TypeInformation getOutputType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + @Override + public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + +} diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java new file mode 100644 index 000000000..838f59541 --- /dev/null +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -0,0 +1,48 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka.table; + +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +public class KafkaSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + KafkaSinkTableInfo kafka11SinkTableInfo = new KafkaSinkTableInfo(); + kafka11SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka11SinkTableInfo); + kafka11SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); + if (props.get(KafkaSinkTableInfo.SINK_DATA_TYPE) != null) { + kafka11SinkTableInfo.setSourceDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); + } + if (props.get(KafkaSinkTableInfo.FIELD_DELINITER) != null) { + kafka11SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); + } + + for (String key:props.keySet()) { + if (!key.isEmpty() && key.startsWith("kafka.")) { + kafka11SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + } + } + return kafka11SinkTableInfo; + } +} diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java new file mode 100644 index 000000000..15a29cde3 --- /dev/null +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.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 com.dtstack.flink.sql.sink.kafka.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +public class KafkaSinkTableInfo extends TargetTableInfo { + //version + private static final String CURR_TYPE = "kafka11"; + + public KafkaSinkTableInfo(){ + super.setType(CURR_TYPE); + } + + public static Map kafkaParam = new HashMap(); + + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); + } + + public String getKafkaParam(String key){ + return kafkaParam.get(key); + } + + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); + } + + @Override + public boolean check() { + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + //Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); + return false; + } + + @Override + public String getType() { + return super.getType(); + } +} diff --git a/kafka11/kafka11-source/pom.xml b/kafka11/kafka11-source/pom.xml index 98182aa64..ae6997f7e 100644 --- a/kafka11/kafka11-source/pom.xml +++ b/kafka11/kafka11-source/pom.xml @@ -13,15 +13,6 @@ kafka11-source http://maven.apache.org - - - - org.apache.flink - flink-connector-kafka-0.11_2.11 - ${flink.version} - - - diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 5c9f5eb49..f1ac74b01 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -20,14 +20,20 @@ package com.dtstack.flink.sql.source.kafka; import com.dtstack.flink.sql.source.IStreamSourceGener; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerCommonConsumer; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerCsvConsumer; +import com.dtstack.flink.sql.source.kafka.consumer.CustomerJsonConsumer; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; -import org.apache.commons.lang3.BooleanUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; @@ -64,16 +70,14 @@ public class KafkaSource implements IStreamSourceGener

{ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { KafkaSourceTableInfo kafka011SourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; - String topicName = kafka011SourceTableInfo.getTopic(); + String topicName = kafka011SourceTableInfo.getKafkaParam("topic"); + String offsetReset = kafka011SourceTableInfo.getKafkaParam("auto.offset.reset"); + Boolean topicIsPattern = kafka011SourceTableInfo.getPatternTopic(); Properties props = new Properties(); - props.setProperty("bootstrap.servers", kafka011SourceTableInfo.getBootstrapServers()); - props.setProperty("auto.offset.reset", kafka011SourceTableInfo.getOffsetReset()); - if (StringUtils.isNotBlank(kafka011SourceTableInfo.getGroupId())){ - props.setProperty("group.id", kafka011SourceTableInfo.getGroupId()); + for (String key : kafka011SourceTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka011SourceTableInfo.getKafkaParam(key)); } - // only required for Kafka 0.8 - //TODO props.setProperty("zookeeper.connect", kafka09SourceTableInfo.) TypeInformation[] types = new TypeInformation[kafka011SourceTableInfo.getFields().length]; for (int i = 0; i < kafka011SourceTableInfo.getFieldClasses().length; i++) { @@ -83,21 +87,41 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv TypeInformation typeInformation = new RowTypeInfo(types, kafka011SourceTableInfo.getFields()); FlinkKafkaConsumer011 kafkaSrc; - if (BooleanUtils.isTrue(kafka011SourceTableInfo.getTopicIsPattern())) { - kafkaSrc = new CustomerKafka011Consumer(Pattern.compile(topicName), - new CustomerJsonDeserialization(typeInformation), props); + String fields = StringUtils.join(kafka011SourceTableInfo.getFields(), ","); + + if ("json".equalsIgnoreCase(kafka011SourceTableInfo.getSourceDataType())) { + if (topicIsPattern) { + kafkaSrc = new CustomerJsonConsumer(Pattern.compile(topicName), + new CustomerJsonDeserialization(typeInformation), props); + } else { + kafkaSrc = new CustomerJsonConsumer(topicName, + new CustomerJsonDeserialization(typeInformation), props); + } + } else if ("csv".equalsIgnoreCase(kafka011SourceTableInfo.getSourceDataType())) { + if (topicIsPattern) { + kafkaSrc = new CustomerCsvConsumer(topicName, + new CustomerCsvDeserialization(typeInformation, + kafka011SourceTableInfo.getFieldDelimiter(), kafka011SourceTableInfo.getLengthCheckPolicy()), props); + } else { + kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), + new CustomerCsvDeserialization(typeInformation, + kafka011SourceTableInfo.getFieldDelimiter(), kafka011SourceTableInfo.getLengthCheckPolicy()), props); + } } else { - kafkaSrc = new CustomerKafka011Consumer(topicName, - new CustomerJsonDeserialization(typeInformation), props); - } + if (topicIsPattern) { + kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); + } else { + kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); + } + } //earliest,latest - if ("earliest".equalsIgnoreCase(kafka011SourceTableInfo.getOffsetReset())) { + if ("earliest".equalsIgnoreCase(offsetReset)) { kafkaSrc.setStartFromEarliest(); - } else if (DtStringUtil.isJosn(kafka011SourceTableInfo.getOffsetReset())) {// {"0":12312,"1":12321,"2":12312} + } else if (DtStringUtil.isJosn(offsetReset)) {// {"0":12312,"1":12321,"2":12312} try { - Properties properties = PluginUtil.jsonStrToObject(kafka011SourceTableInfo.getOffsetReset(), Properties.class); + Properties properties = PluginUtil.jsonStrToObject(offsetReset, Properties.class); Map offsetMap = PluginUtil.ObjectToMap(properties); Map specificStartupOffsets = new HashMap<>(); for (Map.Entry entry : offsetMap.entrySet()) { @@ -105,14 +129,17 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); } catch (Exception e) { - throw new RuntimeException("not support offsetReset type:" + kafka011SourceTableInfo.getOffsetReset()); + throw new RuntimeException("not support offsetReset type:" + offsetReset); } } else { kafkaSrc.setStartFromLatest(); } - - String fields = StringUtils.join(kafka011SourceTableInfo.getFields(), ","); String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); - return tableEnv.fromDataStream(env.addSource(kafkaSrc, sourceOperatorName, typeInformation), fields); + DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); + Integer parallelism = kafka011SourceTableInfo.getParallelism(); + if (parallelism != null) { + kafkaSource.setParallelism(parallelism); + } + return tableEnv.fromDataStream(kafkaSource, fields); } } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java new file mode 100644 index 000000000..58191142f --- /dev/null +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java @@ -0,0 +1,73 @@ +/** + * 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 com.dtstack.flink.sql.source.kafka.consumer; + +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/12/17 + * Company: www.dtstack.com + * + * @author maqi + */ +public class CustomerCommonConsumer extends FlinkKafkaConsumer011 { + + private CustomerCommonDeserialization customerCommonDeserialization; + + + public CustomerCommonConsumer(String topic, KeyedDeserializationSchema deserializer, Properties props) { + super(topic, deserializer, props); + this.customerCommonDeserialization= (CustomerCommonDeserialization) deserializer; + } + + public CustomerCommonConsumer(Pattern subscriptionPattern, KeyedDeserializationSchema deserializer, Properties props) { + super(subscriptionPattern, deserializer, props); + this.customerCommonDeserialization= (CustomerCommonDeserialization) deserializer; + } + + + @Override + public void run(SourceContext sourceContext) throws Exception { + customerCommonDeserialization.setRuntimeContext(getRuntimeContext()); + customerCommonDeserialization.initMetric(); + super.run(sourceContext); + } + + @Override + protected AbstractFetcher createFetcher(SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); + customerCommonDeserialization.setFetcher(fetcher); + return fetcher; + } +} diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka010Consumer.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java similarity index 70% rename from kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka010Consumer.java rename to kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java index f6878473b..e704d1f26 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka010Consumer.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java @@ -16,17 +16,20 @@ * limitations under the License. */ -package com.dtstack.flink.sql.source.kafka; +package com.dtstack.flink.sql.source.kafka.consumer; import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.types.Row; import org.apache.flink.util.SerializedValue; @@ -39,36 +42,40 @@ * Reason: * Date: 2018/10/19 * Company: www.dtstack.com + * * @author xuchao */ -public class CustomerKafka010Consumer extends FlinkKafkaConsumer010 { +public class CustomerCsvConsumer extends FlinkKafkaConsumer011 { - private static final long serialVersionUID = 4873757508981691375L; + private static final long serialVersionUID = -2265366268827807739L; - private CustomerJsonDeserialization customerJsonDeserialization; + private CustomerCsvDeserialization customerCsvDeserialization; - public CustomerKafka010Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + public CustomerCsvConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { super(Arrays.asList(topic.split(",")), valueDeserializer, props); - this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + this.customerCsvDeserialization = (CustomerCsvDeserialization) valueDeserializer; } - public CustomerKafka010Consumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + public CustomerCsvConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { super(subscriptionPattern, valueDeserializer, props); - this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + this.customerCsvDeserialization = (CustomerCsvDeserialization) valueDeserializer; } + + + + @Override public void run(SourceContext sourceContext) throws Exception { - customerJsonDeserialization.setRuntimeContext(getRuntimeContext()); - customerJsonDeserialization.initMetric(); + customerCsvDeserialization.setRuntimeContext(getRuntimeContext()); + customerCsvDeserialization.initMetric(); super.run(sourceContext); } @Override protected AbstractFetcher createFetcher(SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); - customerJsonDeserialization.setFetcher(fetcher); + customerCsvDeserialization.setFetcher(fetcher); return fetcher; } - } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka011Consumer.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java similarity index 86% rename from kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka011Consumer.java rename to kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java index 7cc1f6f40..568ef54c0 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafka011Consumer.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java @@ -16,24 +16,24 @@ * limitations under the License. */ -package com.dtstack.flink.sql.source.kafka; +package com.dtstack.flink.sql.source.kafka.consumer; import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; -import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.types.Row; import org.apache.flink.util.SerializedValue; import java.util.Arrays; -import java.util.Collections; import java.util.Map; import java.util.Properties; import java.util.regex.Pattern; @@ -46,22 +46,23 @@ * @author xuchao */ -public class CustomerKafka011Consumer extends FlinkKafkaConsumer011 { +public class CustomerJsonConsumer extends FlinkKafkaConsumer011 { private static final long serialVersionUID = -2265366268827807739L; private CustomerJsonDeserialization customerJsonDeserialization; - public CustomerKafka011Consumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + public CustomerJsonConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { super(Arrays.asList(topic.split(",")), valueDeserializer, props); this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } - public CustomerKafka011Consumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + public CustomerJsonConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { super(subscriptionPattern, valueDeserializer, props); this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; } + @Override public void run(SourceContext sourceContext) throws Exception { customerJsonDeserialization.setRuntimeContext(getRuntimeContext()); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java new file mode 100644 index 000000000..e15d3892a --- /dev/null +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -0,0 +1,152 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.deserialization; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; +import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.Set; + +import static com.dtstack.flink.sql.metric.MetricConstant.DT_PARTITION_GROUP; +import static com.dtstack.flink.sql.metric.MetricConstant.DT_TOPIC_GROUP; +import static com.dtstack.flink.sql.metric.MetricConstant.DT_TOPIC_PARTITION_LAG_GAUGE; +import static java.nio.charset.StandardCharsets.UTF_8; + +/** + * Date: 2017/5/28 + * + * @author DocLi + */ +public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { + private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); + + public static final String[] KAFKA_COLUMNS = new String[]{"_TOPIC", "_MESSAGEKEY", "_MESSAGE", "_PARTITION", "_OFFSET"}; + + private AbstractFetcher fetcher; + + private boolean firstMsg = true; + + @Override + public Row deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) { + if (firstMsg) { + try { + registerPtMetric(fetcher); + } catch (Exception e) { + LOG.error("register topic partition metric error.", e); + } + firstMsg = false; + } + + numInRecord.inc(); + numInBytes.inc(message.length); + numInBytes.inc(messageKey.length); + + try { + Row row = Row.of( + topic, //topic + messageKey == null ? null : new String(messageKey, UTF_8), //key + new String(message, UTF_8), //message + partition, + offset + ); + return row; + } catch (Throwable t) { + LOG.error(t.getMessage()); + dirtyDataCounter.inc(); + return null; + } + } + + @Override + public Row deserialize(byte[] message) throws IOException { + return null; + } + + public void setFetcher(AbstractFetcher fetcher) { + this.fetcher = fetcher; + } + + + @Override + public boolean isEndOfStream(Row nextElement) { + return false; + } + + public TypeInformation getProducedType() { + TypeInformation[] types = new TypeInformation[]{ + TypeExtractor.createTypeInfo(String.class), + TypeExtractor.createTypeInfo(String.class), //createTypeInformation[String] + TypeExtractor.createTypeInfo(String.class), + Types.INT, + Types.LONG + }; + return new RowTypeInfo(types, KAFKA_COLUMNS); + } + + protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + + Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + consumerThreadField.setAccessible(true); + KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); + + Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); + hasAssignedPartitionsField.setAccessible(true); + + //wait until assignedPartitions + + boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); + + if (!hasAssignedPartitions) { + throw new RuntimeException("wait 50 secs, but not assignedPartitions"); + } + + Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); + consumerField.setAccessible(true); + + KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); + Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); + subscriptionStateField.setAccessible(true); + + //topic partitions lag + SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); + Set assignedPartitions = subscriptionState.assignedPartitions(); + for (TopicPartition topicPartition : assignedPartitions) { + MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) + .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); + metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); + } + + } +} diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java new file mode 100644 index 000000000..cf15b9904 --- /dev/null +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -0,0 +1,183 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.deserialization; + + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.Set; + +import static com.dtstack.flink.sql.metric.MetricConstant.DT_PARTITION_GROUP; +import static com.dtstack.flink.sql.metric.MetricConstant.DT_TOPIC_GROUP; +import static com.dtstack.flink.sql.metric.MetricConstant.DT_TOPIC_PARTITION_LAG_GAUGE; + +/** + * json string parsing custom + * Date: 2017/5/28 + * Company: www.dtstack.com + * @author DocLi + */ + +public class CustomerCsvDeserialization extends AbsDeserialization { + + private static final Logger LOG = LoggerFactory.getLogger(CustomerCsvDeserialization.class); + + private static final long serialVersionUID = -2706012724306826506L; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Type information describing the result type. */ + private final TypeInformation typeInfo; + + /** Field names to parse. Indices match fieldTypes indices. */ + private final String[] fieldNames; + + /** Types to parse fields as. Indices match fieldNames indices. */ + private final TypeInformation[] fieldTypes; + + /** Flag indicating whether to fail on a missing field. */ + private boolean failOnMissingField; + + private AbstractFetcher fetcher; + + private boolean firstMsg = true; + + private String fieldDelimiter; + + private String lengthCheckPolicy; + + public CustomerCsvDeserialization(TypeInformation typeInfo, String fieldDelimiter, String lengthCheckPolicy){ + this.typeInfo = typeInfo; + + this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); + + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + + this.fieldDelimiter = fieldDelimiter; + + this.lengthCheckPolicy = lengthCheckPolicy; + } + + @Override + public Row deserialize(byte[] message) throws IOException { + if(firstMsg){ + try { + registerPtMetric(fetcher); + } catch (Exception e) { + LOG.error("register topic partition metric error.", e); + } + + firstMsg = false; + } + + try { + numInRecord.inc(); + numInBytes.inc(message.length); + String[] fieldsList = null; + if (message != null && message.length > 0){ + fieldsList = new String(message).split(fieldDelimiter); + } + if (fieldsList == null || fieldsList.length != fieldNames.length){//exception condition + if (lengthCheckPolicy.equalsIgnoreCase("SKIP")) { + return null; + }else if (lengthCheckPolicy.equalsIgnoreCase("EXCEPTION")) { + throw new RuntimeException("lengthCheckPolicy Error,message have "+fieldsList.length+" fields,sql have "+fieldNames.length); + } + } + + Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + if (i fetcher) { + this.fetcher = fetcher; + } + + protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + + Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + consumerThreadField.setAccessible(true); + KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); + + Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); + hasAssignedPartitionsField.setAccessible(true); + + //wait until assignedPartitions + + boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); + + if(!hasAssignedPartitions){ + throw new RuntimeException("wait 50 secs, but not assignedPartitions"); + } + + Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); + consumerField.setAccessible(true); + + KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); + Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); + subscriptionStateField.setAccessible(true); + + //topic partitions lag + SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); + Set assignedPartitions = subscriptionState.assignedPartitions(); + for(TopicPartition topicPartition : assignedPartitions){ + MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) + .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); + metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); + } + + } + + public void setFailOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; + } + +} diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java similarity index 98% rename from kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java rename to kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java index 78f1c17ff..e72fd0303 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java @@ -18,7 +18,7 @@ -package com.dtstack.flink.sql.source.kafka; +package com.dtstack.flink.sql.source.kafka.deserialization; import com.dtstack.flink.sql.source.AbsDeserialization; @@ -125,6 +125,7 @@ public Row deserialize(byte[] message) throws IOException { return row; } catch (Throwable t) { //add metric of dirty data + LOG.error(t.getMessage()); dirtyDataCounter.inc(); return null; } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 47d453adc..f5431abbc 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -16,7 +16,6 @@ * limitations under the License. */ - package com.dtstack.flink.sql.source.kafka.table; @@ -30,25 +29,37 @@ * Reason: * Date: 2018/09/18 * Company: www.dtstack.com + * * @author sishu.yss */ public class KafkaSourceParser extends AbsSourceParser { - @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - - KafkaSourceTableInfo kafka11SourceTableInfo = new KafkaSourceTableInfo(); - kafka11SourceTableInfo.setName(tableName); - parseFieldsInfo(fieldsInfo, kafka11SourceTableInfo); - - kafka11SourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSourceTableInfo.PARALLELISM_KEY.toLowerCase()))); - kafka11SourceTableInfo.setBootstrapServers(MathUtil.getString(props.get(KafkaSourceTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase()))); - kafka11SourceTableInfo.setGroupId(MathUtil.getString(props.get(KafkaSourceTableInfo.GROUPID_KEY.toLowerCase()))); - kafka11SourceTableInfo.setTopic(MathUtil.getString(props.get(KafkaSourceTableInfo.TOPIC_KEY.toLowerCase()))); - kafka11SourceTableInfo.setOffsetReset(MathUtil.getString(props.get(KafkaSourceTableInfo.OFFSETRESET_KEY.toLowerCase()))); - kafka11SourceTableInfo.setTopicIsPattern(MathUtil.getBoolean(props.get(KafkaSourceTableInfo.TOPICISPATTERN_KEY.toLowerCase()))); - kafka11SourceTableInfo.check(); - return kafka11SourceTableInfo; - } + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + + KafkaSourceTableInfo kafka11SourceTableInfo = new KafkaSourceTableInfo(); + kafka11SourceTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka11SourceTableInfo); + kafka11SourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSourceTableInfo.PARALLELISM_KEY.toLowerCase()))); + + kafka11SourceTableInfo.setPatternTopic(MathUtil.getBoolean(props.get(KafkaSourceTableInfo.PATTERNTOPIC_KEY.toLowerCase()))); + + if (props.get(KafkaSourceTableInfo.SOURCE_DATA_TYPE) != null) { + kafka11SourceTableInfo.setSourceDataType(props.get(KafkaSourceTableInfo.SOURCE_DATA_TYPE).toString()); + } + if (props.get(KafkaSourceTableInfo.FIELD_DELINITER) != null) { + kafka11SourceTableInfo.setFieldDelimiter(props.get(KafkaSourceTableInfo.FIELD_DELINITER).toString()); + } + if (props.get(KafkaSourceTableInfo.LENGTH_CHECK_POLICY) != null) { + kafka11SourceTableInfo.setLengthCheckPolicy(props.get(KafkaSourceTableInfo.LENGTH_CHECK_POLICY).toString()); + } + for (String key : props.keySet()) { + if (!key.isEmpty() && key.startsWith("kafka.")) { + kafka11SourceTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + } + } + return kafka11SourceTableInfo; + } } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 1b60ec476..ffc270486 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -21,7 +21,9 @@ import com.dtstack.flink.sql.table.SourceTableInfo; import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; -import org.apache.flink.table.shaded.org.apache.commons.lang.BooleanUtils; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; /** * Reason: @@ -32,105 +34,53 @@ */ public class KafkaSourceTableInfo extends SourceTableInfo { - //version private static final String CURR_TYPE = "kafka11"; - public static final String BOOTSTRAPSERVERS_KEY = "bootstrapServers"; - - public static final String TOPIC_KEY = "topic"; - - public static final String GROUPID_KEY = "groupId"; - - public static final String OFFSETRESET_KEY = "offsetReset"; + public static final String PATTERNTOPIC_KEY = "patterntopic"; - public static final String TOPICISPATTERN_KEY = "topicIsPattern"; + private Boolean patternTopic = false; - private String topic; - - private String groupId; - - private Boolean topicIsPattern = false; - - private String bootstrapServers; - public Boolean getTopicIsPattern() { - return topicIsPattern; + public Boolean getPatternTopic() { + return patternTopic; } - public void setTopicIsPattern(Boolean topicIsPattern) { - if (topicIsPattern == null) return; - - this.topicIsPattern = topicIsPattern; + public void setPatternTopic(Boolean patternTopic) { + if (patternTopic == null) { + return; + } + this.patternTopic = patternTopic; } - //latest, earliest - private String offsetReset = "latest"; - - private String offset; - public KafkaSourceTableInfo() { super.setType(CURR_TYPE); } + public static Map kafkaParam = new HashMap(); - public String getBootstrapServers() { - return bootstrapServers; - } - - public void setBootstrapServers(String bootstrapServers) { - this.bootstrapServers = bootstrapServers; - } - - public String getTopic() { - return topic; - } - - public void setTopic(String topic) { - this.topic = topic; - } - - public String getGroupId() { - return groupId; + public void addKafkaParam(String key, String value) { + kafkaParam.put(key, value); } - public void setGroupId(String groupId) { - this.groupId = groupId; + public String getKafkaParam(String key) { + return kafkaParam.get(key); } - public String getOffsetReset() { - return offsetReset; - } - - public void setOffsetReset(String offsetReset) { - if (offsetReset == null) { - return; - } - - this.offsetReset = offsetReset; - } - - public String getOffset() { - return offset; - } - - public void setOffset(String offset) { - this.offset = offset; + public Set getKafkaParamKeys() { + return kafkaParam.keySet(); } @Override public boolean check() { - Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); - Preconditions.checkNotNull(topic, "kafka of topic is required"); - //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); - Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); - + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + Preconditions.checkState(kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("latest") + || kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } @Override public String getType() { -// return super.getType() + SOURCE_SUFFIX; return super.getType(); } } diff --git a/kafka11/pom.xml b/kafka11/pom.xml index 841e20be9..f1f57aa03 100644 --- a/kafka11/pom.xml +++ b/kafka11/pom.xml @@ -14,9 +14,17 @@ kafka11-source + kafka11-sink + + org.apache.flink + flink-connector-kafka-0.11_2.11 + ${flink.version} + + + junit junit @@ -33,4 +41,5 @@ + \ No newline at end of file diff --git a/pom.xml b/pom.xml index df1a54646..58cf34f03 100644 --- a/pom.xml +++ b/pom.xml @@ -23,6 +23,7 @@ sqlserver oracle cassandra + kafka08 From c6a0ed583c76775a7bfe36407c1157ae7a2ced22 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 18 Dec 2018 12:04:56 +0800 Subject: [PATCH 218/470] add class description --- .../dtstack/flink/sql/side/SideSqlExec.java | 1 + .../flink/sql/table/SourceTableInfo.java | 12 +++++++++ .../dtstack/flink/sql/table/TableInfo.java | 24 ----------------- .../flink/sql/table/TargetTableInfo.java | 12 +++++++++ docs/kafkaSource.md | 3 ++- .../sink/kafka/CustomerCsvSerialization.java | 8 ++++++ .../flink/sql/sink/kafka/KafkaSink.java | 12 ++++++--- .../sql/sink/kafka/table/KafkaSinkParser.java | 26 +++++++++++++------ .../sink/kafka/table/KafkaSinkTableInfo.java | 11 +++++++- .../flink/sql/source/kafka/KafkaSource.java | 1 - .../consumer/CustomerCommonConsumer.java | 16 +++--------- .../kafka/consumer/CustomerCsvConsumer.java | 6 ++--- .../CustomerCommonDeserialization.java | 6 +++-- .../CustomerCsvDeserialization.java | 5 ++-- .../source/kafka/table/KafkaSourceParser.java | 9 ++++++- .../sink/kafka/CustomerCsvSerialization.java | 9 ++++++- .../flink/sql/sink/kafka/KafkaSink.java | 13 +++++++--- .../sql/sink/kafka/table/KafkaSinkParser.java | 24 +++++++++++------ .../sink/kafka/table/KafkaSinkTableInfo.java | 11 ++++++-- .../flink/sql/source/kafka/KafkaSource.java | 1 - .../consumer/CustomerCommonConsumer.java | 7 ++--- .../kafka/consumer/CustomerCsvConsumer.java | 9 +++---- .../kafka/consumer/CustomerJsonConsumer.java | 7 ++--- .../CustomerCommonDeserialization.java | 7 +++-- .../CustomerCsvDeserialization.java | 7 ++--- .../source/kafka/table/KafkaSourceParser.java | 7 +++-- .../sink/kafka/CustomerCsvSerialization.java | 11 +++++++- .../flink/sql/sink/kafka/KafkaSink.java | 15 ++++++++--- .../sql/sink/kafka/table/KafkaSinkParser.java | 24 +++++++++++------ .../sink/kafka/table/KafkaSinkTableInfo.java | 13 ++++++++-- .../flink/sql/source/kafka/KafkaSource.java | 1 - .../consumer/CustomerCommonConsumer.java | 9 ++++--- .../kafka/consumer/CustomerCsvConsumer.java | 9 ++++--- .../CustomerCommonDeserialization.java | 7 ++++- .../CustomerCsvDeserialization.java | 8 ++++-- .../sink/kafka/CustomerCsvSerialization.java | 10 ++++++- .../flink/sql/sink/kafka/KafkaSink.java | 14 ++++++++-- .../sql/sink/kafka/table/KafkaSinkParser.java | 12 +++++++-- .../sink/kafka/table/KafkaSinkTableInfo.java | 10 ++++++- .../flink/sql/source/kafka/KafkaSource.java | 1 - .../consumer/CustomerCommonConsumer.java | 8 +++--- .../kafka/consumer/CustomerCsvConsumer.java | 8 +++--- .../CustomerCommonDeserialization.java | 6 ++++- .../CustomerCsvDeserialization.java | 8 +++--- 44 files changed, 296 insertions(+), 132 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 3623ecfd4..5e5534438 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -391,6 +391,7 @@ private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable getFieldClassList() { return fieldClassList; } - public String getSourceDataType() { - return sourceDataType; - } - - public void setSourceDataType(String sourceDataType) { - this.sourceDataType = sourceDataType; - } - public String getSinkDataType() { - return sinkDataType; - } - - public void setSinkDataType(String sinkDataType) { - this.sinkDataType = sinkDataType; - } public String getFieldDelimiter() { return fieldDelimiter; diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TargetTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/TargetTableInfo.java index a9a846707..70b625b5c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TargetTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TargetTableInfo.java @@ -30,4 +30,16 @@ public abstract class TargetTableInfo extends TableInfo { public static final String TARGET_SUFFIX = "Sink"; + + public static final String SINK_DATA_TYPE = "sinkdatatype"; + + private String sinkDataType = "json"; + + public String getSinkDataType() { + return sinkDataType; + } + + public void setSinkDataType(String sinkDataType) { + this.sinkDataType = sinkDataType; + } } diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index d6745c3d7..2f6ce202c 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -20,7 +20,8 @@ CREATE TABLE tableName( ``` ## 2.支持的版本 - kafka08,kafka09,kafka10,kafka11 + kafka08,kafka09,kafka10,kafka11 + **kafka读取和写入的版本必须一致,否则会有兼容性错误。** ## 3.表结构定义 diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 355c79645..53febb240 100644 --- a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -15,6 +15,14 @@ import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + */ @Internal public final class CustomerCsvSerialization extends TypeSerializerSingleton { diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 41f597fe1..4aa7f49fa 100644 --- a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -39,7 +39,13 @@ import java.util.Optional; import java.util.Properties; - +/** + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + */ public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { protected String[] fieldNames; @@ -86,9 +92,9 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { this.schema = schemaBuilder.build(); //this.serializationSchema = Optional.of(JsonRowSerializationSchema.class); - if ("json".equalsIgnoreCase(kafka08SinkTableInfo.getSourceDataType())) { + if ("json".equalsIgnoreCase(kafka08SinkTableInfo.getSinkDataType())) { this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); - } else if ("csv".equalsIgnoreCase(kafka08SinkTableInfo.getSourceDataType())){ + } else if ("csv".equalsIgnoreCase(kafka08SinkTableInfo.getSinkDataType())){ this.serializationSchema = new TypeInformationSerializationSchema(TypeInformation.of(Row.class), new CustomerCsvSerialization(kafka08SinkTableInfo.getFieldDelimiter(),fieldTypes)); } diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index 838f59541..2b6c50512 100644 --- a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -24,25 +24,35 @@ import java.util.Map; +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSinkParser extends AbsTableParser { @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - KafkaSinkTableInfo kafka11SinkTableInfo = new KafkaSinkTableInfo(); - kafka11SinkTableInfo.setName(tableName); - parseFieldsInfo(fieldsInfo, kafka11SinkTableInfo); - kafka11SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); + KafkaSinkTableInfo kafka08SinkTableInfo = new KafkaSinkTableInfo(); + kafka08SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka08SinkTableInfo); + kafka08SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); if (props.get(KafkaSinkTableInfo.SINK_DATA_TYPE) != null) { - kafka11SinkTableInfo.setSourceDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); + kafka08SinkTableInfo.setSinkDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); } if (props.get(KafkaSinkTableInfo.FIELD_DELINITER) != null) { - kafka11SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); + kafka08SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); } for (String key:props.keySet()) { if (!key.isEmpty() && key.startsWith("kafka.")) { - kafka11SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + kafka08SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); } } - return kafka11SinkTableInfo; + return kafka08SinkTableInfo; } } diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index 67100dc05..20e228d30 100644 --- a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -25,9 +25,18 @@ import java.util.Map; import java.util.Set; +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSinkTableInfo extends TargetTableInfo { //version - private static final String CURR_TYPE = "kafka11"; + private static final String CURR_TYPE = "kafka08"; public KafkaSinkTableInfo(){ super.setType(CURR_TYPE); diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 4ec6832a2..2e7136666 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -104,7 +104,6 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } else { kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); } - } //earliest,latest diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java index b1e6f24c7..34b349e2c 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java @@ -18,29 +18,21 @@ package com.dtstack.flink.sql.source.kafka.consumer; import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; -import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; -import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; -import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; import org.apache.flink.types.Row; -import org.apache.flink.util.SerializedValue; -import java.util.Map; import java.util.Properties; import java.util.regex.Pattern; /** - * Reason: - * Date: 2018/12/17 + * + * Date: 2018/12/18 * Company: www.dtstack.com + * @author DocLi * - * @author maqi + * @modifyer maqi */ public class CustomerCommonConsumer extends FlinkKafkaConsumer08 { diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java index 7457dd9ff..ad859e485 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java @@ -38,11 +38,11 @@ import java.util.regex.Pattern; /** - * Reason: - * Date: 2018/10/19 + * Date: 2018/12/18 * Company: www.dtstack.com + * @author DocLi * - * @author xuchao + * @modifyer maqi */ public class CustomerCsvConsumer extends FlinkKafkaConsumer08 { diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java index 0d08dd092..f35bacce7 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -32,9 +32,11 @@ import static java.nio.charset.StandardCharsets.UTF_8; /** - * Date: 2017/5/28 - * + * Date: 2018/12/18 + * Company: www.dtstack.com * @author DocLi + * + * @modifyer maqi */ public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java index caa0ce7c6..e67d79400 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -33,10 +33,11 @@ import java.io.IOException; /** - * json string parsing custom - * Date: 2017/5/28 + * Date: 2018/12/18 * Company: www.dtstack.com * @author DocLi + * + * @modifyer maqi */ public class CustomerCsvDeserialization extends AbsDeserialization { diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 668cbcfd9..9af3c88ca 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -25,7 +25,14 @@ import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; - +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author xuchao + * + * @modifyer DocLi + */ public class KafkaSourceParser extends AbsSourceParser { diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 355c79645..88a10a293 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -14,7 +14,14 @@ import java.io.IOException; import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; - +/** + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ @Internal public final class CustomerCsvSerialization extends TypeSerializerSingleton { diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 2500bad39..106c3ab5c 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -39,7 +39,14 @@ import java.util.Optional; import java.util.Properties; - +/** + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { protected String[] fieldNames; @@ -86,9 +93,9 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { this.schema = schemaBuilder.build(); //this.serializationSchema = Optional.of(JsonRowSerializationSchema.class); - if ("json".equalsIgnoreCase(kafka09SinkTableInfo.getSourceDataType())) { + if ("json".equalsIgnoreCase(kafka09SinkTableInfo.getSinkDataType())) { this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); - } else if ("csv".equalsIgnoreCase(kafka09SinkTableInfo.getSourceDataType())){ + } else if ("csv".equalsIgnoreCase(kafka09SinkTableInfo.getSinkDataType())){ this.serializationSchema = new TypeInformationSerializationSchema(TypeInformation.of(Row.class), new CustomerCsvSerialization(kafka09SinkTableInfo.getFieldDelimiter(),fieldTypes)); } diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index 838f59541..b51b9ea01 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -24,25 +24,33 @@ import java.util.Map; +/** + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSinkParser extends AbsTableParser { @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - KafkaSinkTableInfo kafka11SinkTableInfo = new KafkaSinkTableInfo(); - kafka11SinkTableInfo.setName(tableName); - parseFieldsInfo(fieldsInfo, kafka11SinkTableInfo); - kafka11SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); + KafkaSinkTableInfo kafka09SinkTableInfo = new KafkaSinkTableInfo(); + kafka09SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka09SinkTableInfo); + kafka09SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); if (props.get(KafkaSinkTableInfo.SINK_DATA_TYPE) != null) { - kafka11SinkTableInfo.setSourceDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); + kafka09SinkTableInfo.setSinkDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); } if (props.get(KafkaSinkTableInfo.FIELD_DELINITER) != null) { - kafka11SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); + kafka09SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); } for (String key:props.keySet()) { if (!key.isEmpty() && key.startsWith("kafka.")) { - kafka11SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + kafka09SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); } } - return kafka11SinkTableInfo; + return kafka09SinkTableInfo; } } diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index 6643204a9..bad84231d 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -24,10 +24,17 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; - +/** + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSinkTableInfo extends TargetTableInfo { //version - private static final String CURR_TYPE = "kafka11"; + private static final String CURR_TYPE = "kafka09"; public KafkaSinkTableInfo(){ super.setType(CURR_TYPE); diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index c37da6791..c01366e4c 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -114,7 +114,6 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } else { kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); } - } //earliest,latest diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java index 46272b72c..f6e0eb670 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java @@ -36,11 +36,12 @@ import java.util.regex.Pattern; /** - * Reason: - * Date: 2018/12/17 + * Date: 2018/12/18 * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi * - * @author maqi */ public class CustomerCommonConsumer extends FlinkKafkaConsumer09 { diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java index 28b374c43..70fd50b7b 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java @@ -38,11 +38,12 @@ import java.util.regex.Pattern; /** - * Reason: - * Date: 2018/10/19 + * Date: 2018/12/18 * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi * - * @author xuchao */ public class CustomerCsvConsumer extends FlinkKafkaConsumer09 { @@ -62,8 +63,6 @@ public CustomerCsvConsumer(Pattern subscriptionPattern, AbsDeserialization } - - @Override public void run(SourceFunction.SourceContext sourceContext) throws Exception { customerCsvDeserialization.setRuntimeContext(getRuntimeContext()); diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java index 55b1ba851..e3e9e2d8a 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java @@ -38,11 +38,12 @@ import java.util.regex.Pattern; /** - * Reason: - * Date: 2018/10/19 + * Date: 2018/12/18 * Company: www.dtstack.com - * * @author xuchao + * + * @modifyer maqi + * */ public class CustomerJsonConsumer extends FlinkKafkaConsumer09 { diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java index cc23e69cf..4a235e296 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -43,9 +43,12 @@ import static java.nio.charset.StandardCharsets.UTF_8; /** - * Date: 2017/5/28 - * + * Date: 2018/12/18 + * Company: www.dtstack.com * @author DocLi + * + * @modifyer maqi + * */ public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java index 38b874ed9..20829bf61 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -44,12 +44,13 @@ import static com.dtstack.flink.sql.metric.MetricConstant.*; /** - * json string parsing custom - * Date: 2017/5/28 + * Date: 2018/12/18 * Company: www.dtstack.com * @author DocLi + * + * @modifyer maqi + * */ - public class CustomerCsvDeserialization extends AbsDeserialization { private static final Logger LOG = LoggerFactory.getLogger(CustomerCsvDeserialization.class); diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 0b7edcdaf..3b0d3cc9a 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -27,10 +27,13 @@ import java.util.Map; /** - * Reason: - * Date: 2018/7/4 + /** + * Date: 2018/12/18 * Company: www.dtstack.com * @author xuchao + * + * @modifyer Docli + * */ public class KafkaSourceParser extends AbsSourceParser { diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 355c79645..825703332 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -14,7 +14,16 @@ import java.io.IOException; import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; - +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author DocLi + * + * @modifyer maqi + * + */ @Internal public final class CustomerCsvSerialization extends TypeSerializerSingleton { diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 5f0927960..7d3748441 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -39,7 +39,16 @@ import java.util.Optional; import java.util.Properties; - +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { protected String[] fieldNames; @@ -86,9 +95,9 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { this.schema = schemaBuilder.build(); //this.serializationSchema = Optional.of(JsonRowSerializationSchema.class); - if ("json".equalsIgnoreCase(kafka010SinkTableInfo.getSourceDataType())) { + if ("json".equalsIgnoreCase(kafka010SinkTableInfo.getSinkDataType())) { this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); - } else if ("csv".equalsIgnoreCase(kafka010SinkTableInfo.getSourceDataType())){ + } else if ("csv".equalsIgnoreCase(kafka010SinkTableInfo.getSinkDataType())){ this.serializationSchema = new TypeInformationSerializationSchema(TypeInformation.of(Row.class), new CustomerCsvSerialization(kafka010SinkTableInfo.getFieldDelimiter(),fieldTypes)); } diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index 838f59541..c0818f34c 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -24,25 +24,33 @@ import java.util.Map; +/** + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSinkParser extends AbsTableParser { @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - KafkaSinkTableInfo kafka11SinkTableInfo = new KafkaSinkTableInfo(); - kafka11SinkTableInfo.setName(tableName); - parseFieldsInfo(fieldsInfo, kafka11SinkTableInfo); - kafka11SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); + KafkaSinkTableInfo kafka10SinkTableInfo = new KafkaSinkTableInfo(); + kafka10SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka10SinkTableInfo); + kafka10SinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); if (props.get(KafkaSinkTableInfo.SINK_DATA_TYPE) != null) { - kafka11SinkTableInfo.setSourceDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); + kafka10SinkTableInfo.setSinkDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); } if (props.get(KafkaSinkTableInfo.FIELD_DELINITER) != null) { - kafka11SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); + kafka10SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); } for (String key:props.keySet()) { if (!key.isEmpty() && key.startsWith("kafka.")) { - kafka11SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + kafka10SinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); } } - return kafka11SinkTableInfo; + return kafka10SinkTableInfo; } } diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index 15a29cde3..6557355d9 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -24,10 +24,19 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; - +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSinkTableInfo extends TargetTableInfo { //version - private static final String CURR_TYPE = "kafka11"; + private static final String CURR_TYPE = "kafka10"; public KafkaSinkTableInfo(){ super.setType(CURR_TYPE); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 36c1e5e6e..586f046af 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -117,7 +117,6 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } else { kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); } - } //earliest,latest diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java index 6faf68b1b..bbf91997a 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCommonConsumer.java @@ -36,11 +36,14 @@ import java.util.regex.Pattern; /** - * Reason: - * Date: 2018/12/17 + * + * Date: 2018/12/18 * Company: www.dtstack.com * - * @author maqi + * @author DocLi + * + * @modifyer maqi + * */ public class CustomerCommonConsumer extends FlinkKafkaConsumer010 { diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java index cc5916849..c576a445c 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java @@ -38,11 +38,14 @@ import java.util.regex.Pattern; /** - * Reason: - * Date: 2018/10/19 + * + * Date: 2018/12/18 * Company: www.dtstack.com * - * @author xuchao + * @author DocLi + * + * @modifyer maqi + * */ public class CustomerCsvConsumer extends FlinkKafkaConsumer010 { diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java index cc23e69cf..3502c959c 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -43,9 +43,14 @@ import static java.nio.charset.StandardCharsets.UTF_8; /** - * Date: 2017/5/28 + * + * Date: 2018/12/18 + * Company: www.dtstack.com * * @author DocLi + * + * @modifyer maqi + * */ public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java index 38b874ed9..50a8c5e12 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -44,10 +44,14 @@ import static com.dtstack.flink.sql.metric.MetricConstant.*; /** - * json string parsing custom - * Date: 2017/5/28 + * + * Date: 2018/12/18 * Company: www.dtstack.com + * * @author DocLi + * + * @modifyer maqi + * */ public class CustomerCsvDeserialization extends AbsDeserialization { diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 4f3715923..5184eba4f 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -31,7 +31,15 @@ import java.io.IOException; import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; - +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ @Internal public final class CustomerCsvSerialization extends TypeSerializerSingleton { diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index cc13ac79e..44383ec9b 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -41,6 +41,16 @@ import java.util.Optional; import java.util.Properties; +/** + * kafka result table + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { protected String[] fieldNames; @@ -87,9 +97,9 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { this.schema = schemaBuilder.build(); //this.serializationSchema = Optional.of(JsonRowSerializationSchema.class); - if ("json".equalsIgnoreCase(kafka011SinkTableInfo.getSourceDataType())) { + if ("json".equalsIgnoreCase(kafka011SinkTableInfo.getSinkDataType())) { this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); - } else if ("csv".equalsIgnoreCase(kafka011SinkTableInfo.getSourceDataType())){ + } else if ("csv".equalsIgnoreCase(kafka011SinkTableInfo.getSinkDataType())){ this.serializationSchema = new TypeInformationSerializationSchema(TypeInformation.of(Row.class), new CustomerCsvSerialization(kafka011SinkTableInfo.getFieldDelimiter(),fieldTypes)); } diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index 838f59541..9bf83dccc 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -23,7 +23,15 @@ import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; - +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ public class KafkaSinkParser extends AbsTableParser { @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { @@ -32,7 +40,7 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map { diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java index e704d1f26..66a83bdd1 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java @@ -39,11 +39,13 @@ import java.util.regex.Pattern; /** - * Reason: - * Date: 2018/10/19 + * + * Date: 2018/12/18 * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi * - * @author xuchao */ public class CustomerCsvConsumer extends FlinkKafkaConsumer011 { diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java index e15d3892a..7fc891638 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -45,9 +45,13 @@ import static java.nio.charset.StandardCharsets.UTF_8; /** - * Date: 2017/5/28 * + * Date: 2018/12/18 + * Company: www.dtstack.com * @author DocLi + * + * @modifyer maqi + * */ public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java index cf15b9904..cd0eb6be7 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -46,12 +46,14 @@ import static com.dtstack.flink.sql.metric.MetricConstant.DT_TOPIC_PARTITION_LAG_GAUGE; /** - * json string parsing custom - * Date: 2017/5/28 + * + * Date: 2018/12/18 * Company: www.dtstack.com * @author DocLi + * + * @modifyer maqi + * */ - public class CustomerCsvDeserialization extends AbsDeserialization { private static final Logger LOG = LoggerFactory.getLogger(CustomerCsvDeserialization.class); From 6108d0df1694d9d426a954e5ef521e39143dd973 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 18 Dec 2018 14:22:19 +0800 Subject: [PATCH 219/470] add defaultParallelism to options --- core/src/main/java/com/dtstack/flink/sql/Main.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 415b6e2a9..54ba2cde2 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -111,6 +111,8 @@ public static void main(String[] args) throws Exception { options.addOption("savePointPath", true, "Savepoint restore path"); options.addOption("allowNonRestoredState", true, "Flag indicating whether non restored state is allowed if the savepoint"); + options.addOption("defaultParallelism",false,"defaultParallelism"); + CommandLineParser parser = new DefaultParser(); CommandLine cl = parser.parse(options, args); From 66fd1b3f86ad766f7e6d8c4dc148abc1aabb8c5c Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 18 Dec 2018 19:45:34 +0800 Subject: [PATCH 220/470] add serversocket source --- docs/serversocketSource.md | 73 +++++++ .../flink/sql/source/kafka/KafkaSource.java | 1 + pom.xml | 1 + serversocket/pom.xml | 40 ++++ serversocket/serversocket-source/pom.xml | 83 ++++++++ .../CustomerSocketTextStreamFunction.java | 179 ++++++++++++++++++ .../serversocket/ServersocketSource.java | 62 ++++++ .../table/ServersocketSourceParser.java | 49 +++++ .../table/ServersocketSourceTableInfo.java | 101 ++++++++++ 9 files changed, 589 insertions(+) create mode 100644 docs/serversocketSource.md create mode 100644 serversocket/pom.xml create mode 100644 serversocket/serversocket-source/pom.xml create mode 100644 serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java create mode 100644 serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/ServersocketSource.java create mode 100644 serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceParser.java create mode 100644 serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceTableInfo.java diff --git a/docs/serversocketSource.md b/docs/serversocketSource.md new file mode 100644 index 000000000..cffdc8dd2 --- /dev/null +++ b/docs/serversocketSource.md @@ -0,0 +1,73 @@ + +## 1.数据格式: +``` +数据现在只支持json格式 {"xx":"bb","cc":"dd"} + +CREATE TABLE MyTable( + channel varchar, + pv int, + xctime date, + xtime date + + )WITH( + type='serversocket', + host='127.0.0.1', + port='8888', + delimiter=';', + maxNumRetries='100' + ); +``` + + +## 2.参数: + +|参数名称|含义|是否必填|默认值| +|----|---|---|---| +|type | serversocket | 是|| +|host | server host|是|| +|port | server port|是|| +|delimiter| 每条json数据的分割符(比如:;)|是|| +|maxNumRetries| 最大重连次数 (大于0)|是|| + + +## 3.Server端样例: +``` +String str = "{\"CHANNEL\":\"xc3\",\"pv\":1234567,\"xdate\":\"2018-12-07\",\"xtime\":\"2018-12-15\"};"; + + +public class TimeServerHandler implements Runnable { + Socket socket; + + String str = "{\"CHANNEL\":\"xc3\",\"pv\":1234567,\"xdate\":\"2018-12-07\",\"xtime\":\"2018-12-15\"};"; + + public TimeServerHandler(Socket socket) { + this.socket = socket; + } + + public void run() { + PrintWriter out = null; + try { + out = new PrintWriter(this.socket.getOutputStream(), true); + while (true) { + Thread.sleep(3000); + out.println(str); + } + } catch (IOException e) { + e.printStackTrace(); + + if (out != null) { + out.close(); + } + if (socket != null) { + try { + socket.close(); + } catch (IOException e1) { + e1.printStackTrace(); + } + } + } catch (InterruptedException e) { + e.printStackTrace(); + } + } +} +``` diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 129802fbc..02a6bd0b4 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -35,6 +35,7 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SocketTextStreamFunction; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.table.api.Table; diff --git a/pom.xml b/pom.xml index 58cf34f03..de0f1f7d6 100644 --- a/pom.xml +++ b/pom.xml @@ -24,6 +24,7 @@ oracle cassandra kafka08 + serversocket diff --git a/serversocket/pom.xml b/serversocket/pom.xml new file mode 100644 index 000000000..dd76df619 --- /dev/null +++ b/serversocket/pom.xml @@ -0,0 +1,40 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.serversocket + 1.0-SNAPSHOT + + serversocket-source + + pom + + + 3.8.1 + 1.0-SNAPSHOT + + + + + junit + junit + ${junit.version} + test + + + + com.dtstack.flink + sql.core + ${sql.core.version} + provided + + + \ No newline at end of file diff --git a/serversocket/serversocket-source/pom.xml b/serversocket/serversocket-source/pom.xml new file mode 100644 index 000000000..4df2c3849 --- /dev/null +++ b/serversocket/serversocket-source/pom.xml @@ -0,0 +1,83 @@ + + + + sql.serversocket + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.source.serversocket + 1.0-SNAPSHOT + jar + + serversocket-source + http://maven.apache.org + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + org.slf4j + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java new file mode 100644 index 000000000..8e6004e90 --- /dev/null +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java @@ -0,0 +1,179 @@ +/** + * 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 com.dtstack.flink.sql.source.serversocket; + +import com.dtstack.flink.sql.source.serversocket.table.ServersocketSourceTableInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.types.Row; +import org.apache.flink.util.IOUtils; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.util.Iterator; + + +/** + * Reason: + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author maqi + */ +public class CustomerSocketTextStreamFunction implements SourceFunction { + + /** + * Default delay between successive connection attempts. + */ + private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 500; + + /** + * Default connection timeout when connecting to the server socket (infinite). + */ + private static final int CONNECTION_TIMEOUT_TIME = 0; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** + * Type information describing the result type. + */ + private final TypeInformation typeInfo; + + /** + * Field names to parse. Indices match fieldTypes indices. + */ + private final String[] fieldNames; + + /** + * Types to parse fields as. Indices match fieldNames indices. + */ + private final TypeInformation[] fieldTypes; + + private volatile boolean isRunning = true; + + private transient Socket currentSocket; + + ServersocketSourceTableInfo tableInfo; + + public CustomerSocketTextStreamFunction(ServersocketSourceTableInfo tableInfo, TypeInformation typeInfo) { + this.typeInfo = typeInfo; + + this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); + + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + + this.tableInfo = tableInfo; + } + + @Override + public void run(SourceContext ctx) throws Exception { + final StringBuilder buffer = new StringBuilder(); + long attempt = 0; + + while (isRunning) { + + try (Socket socket = new Socket()) { + currentSocket = socket; + + socket.connect(new InetSocketAddress(tableInfo.getHostname(), tableInfo.getPort()), CONNECTION_TIMEOUT_TIME); + try (BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream()))) { + + char[] cbuf = new char[8192]; + int bytesRead; + while (isRunning && (bytesRead = reader.read(cbuf)) != -1) { + buffer.append(cbuf, 0, bytesRead); + int delimPos; + String delimiter = tableInfo.getDelimiter(); + while (buffer.length() >= delimiter.length() && (delimPos = buffer.indexOf(delimiter)) != -1) { + String record = buffer.substring(0, delimPos); + // truncate trailing carriage return + if (delimiter.equals("\n") && record.endsWith("\r")) { + record = record.substring(0, record.length() - 1); + } + ctx.collect(convertToRow(record)); + buffer.delete(0, delimPos + delimiter.length()); + } + } + } + } + + // if we dropped out of this loop due to an EOF, sleep and retry + if (isRunning) { + attempt++; + if (tableInfo.getMaxNumRetries() == -1 || attempt < tableInfo.getMaxNumRetries()) { + Thread.sleep(DEFAULT_CONNECTION_RETRY_SLEEP); + } else { + // this should probably be here, but some examples expect simple exists of the stream source + // throw new EOFException("Reached end of stream and reconnects are not enabled."); + break; + } + } + } + + // collect trailing data + if (buffer.length() > 0) { + ctx.collect(convertToRow(buffer.toString())); + } + } + + public Row convertToRow(String record) throws IOException { + JsonNode root = objectMapper.readTree(record); + Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + JsonNode node = getIgnoreCase(root, fieldNames[i]); + if (node == null) { + row.setField(i, null); + } else { + // Read the value as specified type + Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + row.setField(i, value); + } + } + return row; + } + + + @Override + public void cancel() { + isRunning = false; + + // we need to close the socket as well, because the Thread.interrupt() function will + // not wake the thread in the socketStream.read() method when blocked. + Socket theSocket = this.currentSocket; + if (theSocket != null) { + IOUtils.closeSocket(theSocket); + } + } + + public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { + Iterator iter = jsonNode.fieldNames(); + while (iter.hasNext()) { + String key1 = iter.next(); + if (key1.equalsIgnoreCase(key)) { + return jsonNode.get(key1); + } + } + return null; + } +} diff --git a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/ServersocketSource.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/ServersocketSource.java new file mode 100644 index 000000000..3a67d2c98 --- /dev/null +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/ServersocketSource.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 com.dtstack.flink.sql.source.serversocket; + +import com.dtstack.flink.sql.source.IStreamSourceGener; +import com.dtstack.flink.sql.source.serversocket.table.ServersocketSourceTableInfo; +import com.dtstack.flink.sql.table.SourceTableInfo; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SocketTextStreamFunction; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.types.Row; + +/** + * Reason: + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author maqi + */ +public class ServersocketSource implements IStreamSourceGener

{ + @Override + public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { + ServersocketSourceTableInfo serversocketSourceTableInfo = (ServersocketSourceTableInfo) sourceTableInfo; + + String tableName = serversocketSourceTableInfo.getName(); + + TypeInformation[] types = new TypeInformation[serversocketSourceTableInfo.getFields().length]; + for (int i = 0; i < serversocketSourceTableInfo.getFieldClasses().length; i++) { + types[i] = TypeInformation.of(serversocketSourceTableInfo.getFieldClasses()[i]); + } + + TypeInformation typeInformation = new RowTypeInfo(types, serversocketSourceTableInfo.getFields()); + + String fields = StringUtils.join(serversocketSourceTableInfo.getFields(), ","); + + CustomerSocketTextStreamFunction customerSocketTextStreamFunction = new CustomerSocketTextStreamFunction(serversocketSourceTableInfo, typeInformation); + + DataStreamSource serversocketSource = env.addSource(customerSocketTextStreamFunction, tableName, typeInformation); + + return tableEnv.fromDataStream(serversocketSource, fields); + } +} diff --git a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceParser.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceParser.java new file mode 100644 index 000000000..b52e38e59 --- /dev/null +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceParser.java @@ -0,0 +1,49 @@ +/** + * 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 com.dtstack.flink.sql.source.serversocket.table; + +import com.dtstack.flink.sql.table.AbsSourceParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +/** + * Reason: + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author maqi + */ +public class ServersocketSourceParser extends AbsSourceParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + ServersocketSourceTableInfo serversocketSourceTableInfo = new ServersocketSourceTableInfo(); + serversocketSourceTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, serversocketSourceTableInfo); + + serversocketSourceTableInfo.setHostname(MathUtil.getString(props.get(ServersocketSourceTableInfo.HOSTNAME_KEY.toLowerCase()))); + serversocketSourceTableInfo.setPort(MathUtil.getIntegerVal(props.get(ServersocketSourceTableInfo.PORT_KEY.toLowerCase()))); + serversocketSourceTableInfo.setDelimiter(MathUtil.getString(props.get(ServersocketSourceTableInfo.DELIMITER_KEY.toLowerCase()))); + serversocketSourceTableInfo.setMaxNumRetries(MathUtil.getLongVal(props.get(ServersocketSourceTableInfo.MAXNUMRETRIES_KEY.toLowerCase()))); + + serversocketSourceTableInfo.check(); + + return serversocketSourceTableInfo; + } +} diff --git a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceTableInfo.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceTableInfo.java new file mode 100644 index 000000000..2ff9a6739 --- /dev/null +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceTableInfo.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 com.dtstack.flink.sql.source.serversocket.table; + +import com.dtstack.flink.sql.table.SourceTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * Reason: + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author maqi + */ +public class ServersocketSourceTableInfo extends SourceTableInfo { + //version + private static final String CURR_TYPE = "serversocket"; + + public static final String HOSTNAME_KEY = "host"; + + public static final String PORT_KEY = "port"; + + public static final String DELIMITER_KEY = "delimiter"; + + public static final String MAXNUMRETRIES_KEY = "maxNumRetries"; + + + public ServersocketSourceTableInfo() { + super.setType(CURR_TYPE); + } + + private String hostname; + + private int port; + + private String delimiter; + + private long maxNumRetries; + + public String getHostname() { + return hostname; + } + + public void setHostname(String hostname) { + this.hostname = hostname; + } + + public int getPort() { + return port; + } + + public void setPort(int port) { + this.port = port; + } + + public String getDelimiter() { + return delimiter; + } + + public void setDelimiter(String delimiter) { + this.delimiter = delimiter; + } + + public long getMaxNumRetries() { + return maxNumRetries; + } + + public void setMaxNumRetries(long maxNumRetries) { + this.maxNumRetries = maxNumRetries; + } + + + @Override + public boolean check() { + Preconditions.checkNotNull(hostname,"host name not null"); + Preconditions.checkNotNull(port,"port not null"); + Preconditions.checkNotNull(delimiter,"delimiter name not null"); + Preconditions.checkNotNull(maxNumRetries,"maxNumRetries name not null"); + + Preconditions.checkArgument(port > 0 && port < 65536, "port is out of range"); + Preconditions.checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), or -1 (infinite retries)"); + return false; + } + + +} From ee191e0d3349052f8e122b2f96c277f3c4f7f25a Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 19 Dec 2018 11:29:49 +0800 Subject: [PATCH 221/470] code optimize --- .../main/java/com/dtstack/flink/sql/Main.java | 70 ++++++++----------- .../MyLocalStreamEnvironment.java | 2 +- .../sql/options}/LauncherOptionParser.java | 11 +-- .../flink/sql/options}/LauncherOptions.java | 7 +- .../sql/launcher/ClusterClientFactory.java | 3 +- .../flink/sql/launcher/LauncherMain.java | 2 +- .../sql/launcher/perjob/PerJobSubmitter.java | 2 +- 7 files changed, 39 insertions(+), 58 deletions(-) rename core/src/main/java/com/dtstack/flink/sql/{ => environment}/MyLocalStreamEnvironment.java (99%) rename {launcher/src/main/java/com/dtstack/flink/sql/launcher => core/src/main/java/com/dtstack/flink/sql/options}/LauncherOptionParser.java (96%) rename {launcher/src/main/java/com/dtstack/flink/sql/launcher => core/src/main/java/com/dtstack/flink/sql/options}/LauncherOptions.java (94%) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 54ba2cde2..5ff2c6450 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -22,6 +22,9 @@ import com.dtstack.flink.sql.classloader.DtClassLoader; import com.dtstack.flink.sql.enums.ECacheType; +import com.dtstack.flink.sql.environment.MyLocalStreamEnvironment; +import com.dtstack.flink.sql.options.LauncherOptionParser; +import com.dtstack.flink.sql.options.LauncherOptions; import com.dtstack.flink.sql.parser.*; import com.dtstack.flink.sql.side.SideSqlExec; import com.dtstack.flink.sql.side.SideTableInfo; @@ -42,6 +45,7 @@ import org.apache.commons.cli.DefaultParser; import org.apache.commons.cli.Options; import org.apache.commons.io.Charsets; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -59,7 +63,6 @@ import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -99,44 +102,22 @@ public class Main { private static final int delayInterval = 10; //sec public static void main(String[] args) throws Exception { - - Options options = new Options(); - options.addOption("sql", true, "sql config"); - options.addOption("name", true, "job name"); - options.addOption("addjar", true, "add jar"); - options.addOption("localSqlPluginPath", true, "local sql plugin path"); - options.addOption("remoteSqlPluginPath", true, "remote sql plugin path"); - options.addOption("confProp", true, "env properties"); - options.addOption("mode", true, "deploy mode"); - - options.addOption("savePointPath", true, "Savepoint restore path"); - options.addOption("allowNonRestoredState", true, "Flag indicating whether non restored state is allowed if the savepoint"); - options.addOption("defaultParallelism",false,"defaultParallelism"); - - - CommandLineParser parser = new DefaultParser(); - CommandLine cl = parser.parse(options, args); - String sql = cl.getOptionValue("sql"); - String name = cl.getOptionValue("name"); - String addJarListStr = cl.getOptionValue("addjar"); - String localSqlPluginPath = cl.getOptionValue("localSqlPluginPath"); - String remoteSqlPluginPath = cl.getOptionValue("remoteSqlPluginPath"); - String deployMode = cl.getOptionValue("mode"); - String confProp = cl.getOptionValue("confProp"); - - Preconditions.checkNotNull(sql, "parameters of sql is required"); - Preconditions.checkNotNull(name, "parameters of name is required"); - Preconditions.checkNotNull(localSqlPluginPath, "parameters of localSqlPluginPath is required"); - + LauncherOptionParser optionParser = new LauncherOptionParser(args); + LauncherOptions launcherOptions = optionParser.getLauncherOptions(); + String sql = launcherOptions.getSql(); + String name =launcherOptions.getName(); + String addJarListStr = launcherOptions.getAddjar(); + String localSqlPluginPath = launcherOptions.getLocalSqlPluginPath(); + String remoteSqlPluginPath = launcherOptions.getRemoteSqlPluginPath(); + String deployMode = launcherOptions.getMode(); + String confProp = launcherOptions.getConfProp(); sql = URLDecoder.decode(sql, Charsets.UTF_8.name()); SqlParser.setLocalSqlPluginRoot(localSqlPluginPath); - List addJarFileList = Lists.newArrayList(); if(!Strings.isNullOrEmpty(addJarListStr)){ addJarListStr = URLDecoder.decode(addJarListStr, Charsets.UTF_8.name()); addJarFileList = objMapper.readValue(addJarListStr, List.class); } - ClassLoader threadClassLoader = Thread.currentThread().getContextClassLoader(); DtClassLoader dtClassLoader = new DtClassLoader(new URL[]{}, threadClassLoader); Thread.currentThread().setContextClassLoader(dtClassLoader); @@ -240,6 +221,12 @@ private static void addEnvClassPath(StreamExecutionEnvironment env, Set cla contextEnvironment.getClasspaths().add(url); } } + int i = 0; + for(URL url : classPathSet){ + String classFileName = String.format(CLASS_FILE_NAME_FMT, i); + env.registerCachedFile(url.getPath(), classFileName, true); + i++; + } } private static void registerUDF(SqlTree sqlTree, List jarURList, URLClassLoader parentClassloader, @@ -291,18 +278,23 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en Table regTable = tableEnv.fromDataStream(adaptStream, fields); tableEnv.registerTable(tableInfo.getName(), regTable); registerTableCache.put(tableInfo.getName(), regTable); - classPathSet.add(PluginUtil.getRemoteJarFilePath(tableInfo.getType(), SourceTableInfo.SOURCE_SUFFIX, remoteSqlPluginPath)); + if(StringUtils.isNotBlank(remoteSqlPluginPath)){ + classPathSet.add(PluginUtil.getRemoteJarFilePath(tableInfo.getType(), SourceTableInfo.SOURCE_SUFFIX, remoteSqlPluginPath)); + } } else if (tableInfo instanceof TargetTableInfo) { TableSink tableSink = StreamSinkFactory.getTableSink((TargetTableInfo) tableInfo, localSqlPluginPath); TypeInformation[] flinkTypes = FlinkUtil.transformTypes(tableInfo.getFieldClasses()); tableEnv.registerTableSink(tableInfo.getName(), tableInfo.getFields(), flinkTypes, tableSink); - classPathSet.add( PluginUtil.getRemoteJarFilePath(tableInfo.getType(), TargetTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); + if(StringUtils.isNotBlank(remoteSqlPluginPath)){ + classPathSet.add( PluginUtil.getRemoteJarFilePath(tableInfo.getType(), TargetTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); + } } else if(tableInfo instanceof SideTableInfo){ - String sideOperator = ECacheType.ALL.name().equals(((SideTableInfo) tableInfo).getCacheType()) ? "all" : "async"; sideTableMap.put(tableInfo.getName(), (SideTableInfo) tableInfo); - classPathSet.add(PluginUtil.getRemoteSideJarFilePath(tableInfo.getType(), sideOperator, SideTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); + if(StringUtils.isNotBlank(remoteSqlPluginPath)){ + classPathSet.add(PluginUtil.getRemoteSideJarFilePath(tableInfo.getType(), sideOperator, SideTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); + } }else { throw new RuntimeException("not support table type:" + tableInfo.getType()); } @@ -310,12 +302,6 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en //The plug-in information corresponding to the table is loaded into the classPath env addEnvClassPath(env, classPathSet); - int i = 0; - for(URL url : classPathSet){ - String classFileName = String.format(CLASS_FILE_NAME_FMT, i); - env.registerCachedFile(url.getPath(), classFileName, true); - i++; - } } private static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws IOException { diff --git a/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java b/core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java similarity index 99% rename from core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java rename to core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java index acd28e3f3..c1cea1e14 100644 --- a/core/src/main/java/com/dtstack/flink/sql/MyLocalStreamEnvironment.java +++ b/core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.dtstack.flink.sql; +package com.dtstack.flink.sql.environment; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/core/src/main/java/com/dtstack/flink/sql/options/LauncherOptionParser.java similarity index 96% rename from launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java rename to core/src/main/java/com/dtstack/flink/sql/options/LauncherOptionParser.java index a801b193e..d4b4a19f2 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/options/LauncherOptionParser.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package com.dtstack.flink.sql.launcher; +package com.dtstack.flink.sql.options; import avro.shaded.com.google.common.collect.Lists; import org.apache.commons.cli.BasicParser; @@ -73,7 +73,7 @@ public class LauncherOptionParser { private LauncherOptions properties = new LauncherOptions(); - public LauncherOptionParser(String[] args) { + public LauncherOptionParser(String[] args) throws Exception { options.addOption(OPTION_MODE, true, "Running mode"); options.addOption(OPTION_SQL, true, "Job sql file"); options.addOption(OPTION_NAME, true, "Job name"); @@ -87,8 +87,6 @@ public LauncherOptionParser(String[] args) { options.addOption(OPTION_SAVE_POINT_PATH, true, "Savepoint restore path"); options.addOption(OPTION_ALLOW_NON_RESTORED_STATE, true, "Flag indicating whether non restored state is allowed if the savepoint"); options.addOption(OPTION_FLINK_JAR_PATH, true, "flink jar path for submit of perjob mode"); - - try { CommandLine cl = parser.parse(options, args); String mode = cl.getOptionValue(OPTION_MODE, ClusterMode.local.name()); //check mode @@ -109,7 +107,6 @@ public LauncherOptionParser(String[] args) { properties.setLocalSqlPluginPath(localPlugin); String remotePlugin = cl.getOptionValue(OPTION_REMOTE_SQL_PLUGIN_PATH); - Preconditions.checkNotNull(remotePlugin); properties.setRemoteSqlPluginPath(remotePlugin); String name = Preconditions.checkNotNull(cl.getOptionValue(OPTION_NAME)); @@ -148,10 +145,6 @@ public LauncherOptionParser(String[] args) { if(StringUtils.isNotBlank(flinkJarPath)){ properties.setFlinkJarPath(flinkJarPath); } - - } catch (Exception e) { - throw new RuntimeException(e); - } } public LauncherOptions getLauncherOptions(){ diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java b/core/src/main/java/com/dtstack/flink/sql/options/LauncherOptions.java similarity index 94% rename from launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java rename to core/src/main/java/com/dtstack/flink/sql/options/LauncherOptions.java index d9c5bd1ed..07c349fda 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java +++ b/core/src/main/java/com/dtstack/flink/sql/options/LauncherOptions.java @@ -16,7 +16,10 @@ * limitations under the License. */ -package com.dtstack.flink.sql.launcher; +package com.dtstack.flink.sql.options; + +import com.dtstack.flink.sql.ClusterMode; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; /** * This class define commandline options for the Launcher program @@ -26,7 +29,7 @@ */ public class LauncherOptions { - private String mode; + private String mode = ClusterMode.local.name(); private String name; diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 02af5fff4..be8611757 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -42,7 +42,6 @@ import org.apache.hadoop.yarn.client.api.YarnClientApplication; import org.apache.hadoop.yarn.conf.YarnConfiguration; import java.io.File; -import java.io.FilenameFilter; import java.net.InetSocketAddress; import java.net.URLDecoder; import java.util.*; @@ -52,7 +51,7 @@ import java.io.IOException; import java.util.stream.Collectors; import java.util.stream.Stream; - +import com.dtstack.flink.sql.options.LauncherOptions; import static java.util.Objects.requireNonNull; /** diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index ec6a3e10c..f32ed9008 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -32,7 +32,6 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; - import com.dtstack.flink.sql.ClusterMode; import org.apache.flink.client.program.PackagedProgramUtils; import org.apache.flink.configuration.Configuration; @@ -41,6 +40,7 @@ import org.apache.flink.table.shaded.org.apache.commons.lang.StringUtils; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.table.shaded.org.apache.commons.lang.BooleanUtils; +import com.dtstack.flink.sql.options.*; /** * Date: 2017/2/20 diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java index 3cdf3f8d4..5c0032ce6 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.launcher.perjob; -import com.dtstack.flink.sql.launcher.LauncherOptions; +import com.dtstack.flink.sql.options.LauncherOptions; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; From b8cdfd286468074f270e909c2947bfff336880b9 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 19 Dec 2018 18:56:36 +0800 Subject: [PATCH 222/470] async cache get data order or unorder --- .../dtstack/flink/sql/side/SideTableInfo.java | 12 ++++++++++++ .../sql/side/operator/SideAsyncOperator.java | 15 +++++++++++++-- .../flink/sql/table/AbsSideTableParser.java | 16 ++++++++++++---- docs/mysqlSide.md | 2 ++ 4 files changed, 39 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java index 0abd55a92..02c9b4e02 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java @@ -45,6 +45,8 @@ public abstract class SideTableInfo extends TableInfo implements Serializable { public static final String PARTITIONED_JOIN_KEY = "partitionedJoin"; + public static final String CACHE_MODE_KEY = "cacheMode"; + private String cacheType = "none";//None or LRU or ALL private int cacheSize = 10000; @@ -53,6 +55,8 @@ public abstract class SideTableInfo extends TableInfo implements Serializable { private boolean partitionedJoin = false; + private String cacheMode="ordered"; + public RowTypeInfo getRowTypeInfo(){ Class[] fieldClass = getFieldClasses(); TypeInformation[] types = new TypeInformation[fieldClass.length]; @@ -95,4 +99,12 @@ public boolean isPartitionedJoin() { public void setPartitionedJoin(boolean partitionedJoin) { this.partitionedJoin = partitionedJoin; } + + public String getCacheMode() { + return cacheMode; + } + + public void setCacheMode(String cacheMode) { + this.cacheMode = cacheMode; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java index df9a1c175..95121faa8 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java @@ -25,6 +25,7 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.util.PluginUtil; +import com.sun.org.apache.bcel.internal.generic.I2F; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.AsyncDataStream; import org.apache.flink.streaming.api.datastream.DataStream; @@ -44,6 +45,9 @@ public class SideAsyncOperator { private static final String PATH_FORMAT = "%sasyncside"; + private static final String ORDERED = "ordered"; + + //TODO need to set by create table task private static int asyncCapacity = 100; @@ -62,8 +66,15 @@ private static AsyncReqRow loadAsyncReq(String sideType, String sqlRootDir, RowT public static DataStream getSideJoinDataStream(DataStream inputStream, String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { AsyncReqRow asyncDbReq = loadAsyncReq(sideType, sqlRootDir, rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + //TODO How much should be set for the degree of parallelism? Timeout? capacity settings? - return AsyncDataStream.orderedWait(inputStream, asyncDbReq, 10000, TimeUnit.MILLISECONDS, asyncCapacity) - .setParallelism(sideTableInfo.getParallelism()); + if (ORDERED.equals(sideTableInfo.getCacheMode())){ + return AsyncDataStream.orderedWait(inputStream, asyncDbReq, 10000, TimeUnit.MILLISECONDS, asyncCapacity) + .setParallelism(sideTableInfo.getParallelism()); + }else { + return AsyncDataStream.unorderedWait(inputStream, asyncDbReq, 10000, TimeUnit.MILLISECONDS, asyncCapacity) + .setParallelism(sideTableInfo.getParallelism()); + } + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java index f8ede801b..26305f7d6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java @@ -16,13 +16,14 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.table; import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.util.MathUtil; +import org.apache.commons.lang3.StringUtils; import java.util.Map; import java.util.regex.Matcher; @@ -65,9 +66,7 @@ protected void parseCacheProp(SideTableInfo sideTableInfo, Map p sideTableInfo.setCacheType(cacheType); if(props.containsKey(SideTableInfo.CACHE_SIZE_KEY.toLowerCase())){ Integer cacheSize = MathUtil.getIntegerVal(props.get(SideTableInfo.CACHE_SIZE_KEY.toLowerCase())); - if(cacheSize < 0){ - throw new RuntimeException("cache size need > 0."); - } + sideTableInfo.setCacheSize(cacheSize); } @@ -85,6 +84,15 @@ protected void parseCacheProp(SideTableInfo sideTableInfo, Map p sideTableInfo.setPartitionedJoin(true); } } + + if(props.containsKey(SideTableInfo.CACHE_MODE_KEY.toLowerCase())){ + String cachemode = MathUtil.getString(props.get(SideTableInfo.CACHE_MODE_KEY.toLowerCase())); + + if(!cachemode.equalsIgnoreCase("ordered") && !cachemode.equalsIgnoreCase("unordered")){ + throw new RuntimeException("cachemode must ordered or unordered!"); + } + sideTableInfo.setCacheMode(cachemode.toLowerCase()); + } } } } diff --git a/docs/mysqlSide.md b/docs/mysqlSide.md index d0fec5832..63e745776 100644 --- a/docs/mysqlSide.md +++ b/docs/mysqlSide.md @@ -52,6 +52,7 @@ * LRU: * cacheSize: 缓存的条目数量 * cacheTTLMs:缓存的过期时间(ms) + * cacheMode: (unordered|ordered)异步加载是有序还是无序,默认无序。 ## 5.样例 @@ -70,6 +71,7 @@ create table sideTable( cache ='LRU', cacheSize ='10000', cacheTTLMs ='60000', + cacheMode='unordered', parallelism ='1', partitionedJoin='false' ); From 6a6a7d3062a5780aa426f37abb599c0324c93e4d Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 19 Dec 2018 19:03:59 +0800 Subject: [PATCH 223/470] modify default cachemode order --- .../dtstack/flink/sql/side/operator/SideAsyncOperator.java | 1 - .../java/com/dtstack/flink/sql/table/AbsSideTableParser.java | 4 +++- docs/mysqlSide.md | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java index 95121faa8..121d557bf 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java @@ -25,7 +25,6 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.util.PluginUtil; -import com.sun.org.apache.bcel.internal.generic.I2F; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.AsyncDataStream; import org.apache.flink.streaming.api.datastream.DataStream; diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java index 26305f7d6..526a632dd 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java @@ -66,7 +66,9 @@ protected void parseCacheProp(SideTableInfo sideTableInfo, Map p sideTableInfo.setCacheType(cacheType); if(props.containsKey(SideTableInfo.CACHE_SIZE_KEY.toLowerCase())){ Integer cacheSize = MathUtil.getIntegerVal(props.get(SideTableInfo.CACHE_SIZE_KEY.toLowerCase())); - + if(cacheSize < 0){ + throw new RuntimeException("cache size need > 0."); + } sideTableInfo.setCacheSize(cacheSize); } diff --git a/docs/mysqlSide.md b/docs/mysqlSide.md index 63e745776..c6f5d741a 100644 --- a/docs/mysqlSide.md +++ b/docs/mysqlSide.md @@ -52,7 +52,7 @@ * LRU: * cacheSize: 缓存的条目数量 * cacheTTLMs:缓存的过期时间(ms) - * cacheMode: (unordered|ordered)异步加载是有序还是无序,默认无序。 + * cacheMode: (unordered|ordered)异步加载是有序还是无序,默认有序。 ## 5.样例 From 9187df5c6aa0e2f4b99ab4570efb19e2d4b58ea5 Mon Sep 17 00:00:00 2001 From: XuQianJin-Stars Date: Thu, 20 Dec 2018 12:36:56 +0800 Subject: [PATCH 224/470] add console sink --- README.md | 2 +- console/console-sink/pom.xml | 79 ++++ .../sql/sink/console/ConsoleOutputFormat.java | 118 ++++++ .../flink/sql/sink/console/ConsoleSink.java | 86 +++++ .../sink/console/table/ConsoleSinkParser.java | 45 +++ .../sink/console/table/ConsoleTableInfo.java | 46 +++ .../sink/console/table/TablePrintUtil.java | 336 ++++++++++++++++++ .../test/java/com/dtstack/flinkx/AppTest.java | 58 +++ console/pom.xml | 34 ++ docs/consoleSink.md | 50 +++ pom.xml | 1 + 11 files changed, 854 insertions(+), 1 deletion(-) create mode 100644 console/console-sink/pom.xml create mode 100644 console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java create mode 100644 console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleSink.java create mode 100644 console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleSinkParser.java create mode 100644 console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleTableInfo.java create mode 100644 console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java create mode 100644 console/console-sink/src/test/java/com/dtstack/flinkx/AppTest.java create mode 100644 console/pom.xml create mode 100644 docs/consoleSink.md diff --git a/README.md b/README.md index ad14222ca..e2056dd17 100644 --- a/README.md +++ b/README.md @@ -11,7 +11,7 @@ # 已支持 * 源表:kafka 0.9,1.x版本 * 维表:mysql,SQlServer,oracle,hbase,mongo,redis,cassandra - * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra + * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra,console # 后续开发计划 * 增加kafka结果表功能 diff --git a/console/console-sink/pom.xml b/console/console-sink/pom.xml new file mode 100644 index 000000000..a5ca2629f --- /dev/null +++ b/console/console-sink/pom.xml @@ -0,0 +1,79 @@ + + + + sql.console + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + console-sink + jar + + console-sink + http://maven.apache.org + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java new file mode 100644 index 000000000..55bed93be --- /dev/null +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java @@ -0,0 +1,118 @@ +/** + * 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 com.dtstack.flink.sql.sink.console; + +import com.dtstack.flink.sql.sink.MetricOutputFormat; +import com.dtstack.flink.sql.sink.console.table.TablePrintUtil; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Reason: + * Date: 2018/12/19 + * + * @author xuqianjin + */ +public class ConsoleOutputFormat extends MetricOutputFormat { + + private static final Logger LOG = LoggerFactory.getLogger(ConsoleOutputFormat.class); + + protected String[] fieldNames; + TypeInformation[] fieldTypes; + + @Override + public void configure(Configuration parameters) { + + } + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + initMetric(); + } + + @Override + public void writeRecord(Tuple2 tuple2) throws IOException { + Tuple2 tupleTrans = tuple2; + Boolean retract = tupleTrans.getField(0); + if (!retract) { + //FIXME 暂时不处理Mongo删除操作--->Mongo要求有key,所有认为都是可以执行update查找 + return; + } + + Row record = tupleTrans.getField(1); + if (record.getArity() != fieldNames.length) { + return; + } + + List data = new ArrayList<>(); + data.add(fieldNames); + data.add(record.toString().split(",")); + TablePrintUtil.build(data).print(); + + outRecords.inc(); + } + + @Override + public void close() throws IOException { + + } + + private ConsoleOutputFormat() { + } + + public static ConsoleOutputFormatBuilder buildOutputFormat() { + return new ConsoleOutputFormatBuilder(); + } + + public static class ConsoleOutputFormatBuilder { + + private final ConsoleOutputFormat format; + + protected ConsoleOutputFormatBuilder() { + this.format = new ConsoleOutputFormat(); + } + + public ConsoleOutputFormatBuilder setFieldNames(String[] fieldNames) { + format.fieldNames = fieldNames; + return this; + } + + public ConsoleOutputFormatBuilder setFieldTypes(TypeInformation[] fieldTypes) { + format.fieldTypes = fieldTypes; + return this; + } + + /** + * Finalizes the configuration and checks validity. + * + * @return Configured RetractConsoleCOutputFormat + */ + public ConsoleOutputFormat finish() { + return format; + } + } +} diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleSink.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleSink.java new file mode 100644 index 000000000..77a3efea2 --- /dev/null +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleSink.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 com.dtstack.flink.sql.sink.console; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.table.sinks.RetractStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +/** + * Reason: + * Date: 2018/12/19 + * + * @author xuqianjin + */ +public class ConsoleSink implements RetractStreamTableSink, IStreamSinkGener { + + protected String[] fieldNames; + TypeInformation[] fieldTypes; + + @Override + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + + @Override + public TupleTypeInfo> getOutputType() { + return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), getRecordType()); + } + + @Override + public TypeInformation getRecordType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + @Override + public void emitDataStream(DataStream> dataStream) { + ConsoleOutputFormat.ConsoleOutputFormatBuilder builder = ConsoleOutputFormat.buildOutputFormat(); + builder.setFieldNames(this.fieldNames) + .setFieldTypes(this.fieldTypes); + ConsoleOutputFormat outputFormat = builder.finish(); + RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(outputFormat); + dataStream.addSink(richSinkFunction); + } + + @Override + public ConsoleSink genStreamSink(TargetTableInfo targetTableInfo) { + return this; + } +} diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleSinkParser.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleSinkParser.java new file mode 100644 index 000000000..e77444bfd --- /dev/null +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleSinkParser.java @@ -0,0 +1,45 @@ +/** + * 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 com.dtstack.flink.sql.sink.console.table; + +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; + +/** + * Reason: + * Date: 2018/12/19 + * + * @author xuqianjin + */ +public class ConsoleSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + ConsoleTableInfo consoleTableInfo = new ConsoleTableInfo(); + consoleTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, consoleTableInfo); + + consoleTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase()))); + return consoleTableInfo; + } +} diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleTableInfo.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleTableInfo.java new file mode 100644 index 000000000..4b286c667 --- /dev/null +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/ConsoleTableInfo.java @@ -0,0 +1,46 @@ +/** + * 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 com.dtstack.flink.sql.sink.console.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; + +/** + * Reason: + * Date: 2018/12/19 + * + * @author xuqianjin + */ +public class ConsoleTableInfo extends TargetTableInfo { + + private static final String CURR_TYPE = "console"; + + public ConsoleTableInfo() { + setType(CURR_TYPE); + } + + @Override + public boolean check() { + return true; + } + + @Override + public String getType() { + return super.getType().toLowerCase(); + } +} diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java new file mode 100644 index 000000000..8934d4394 --- /dev/null +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java @@ -0,0 +1,336 @@ +package com.dtstack.flink.sql.sink.console.table; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Reason: + * Date: 2018/12/19 + * + * @author xuqianjin + */ +public class TablePrintUtil { + public static final int ALIGN_LEFT = 1;//左对齐 + public static final int ALIGN_RIGHT = 2;//右对齐 + public static final int ALIGN_CENTER = 3;//居中对齐 + + private int align = ALIGN_CENTER;//默认居中对齐 + private boolean equilong = false;//默认不等宽 + private int padding = 1;//左右边距默认为1 + private char h = '-';//默认水平分隔符 + private char v = '|';//默认竖直分隔符 + private char o = '+';//默认交叉分隔符 + private char s = ' ';//默认空白填充符 + private List data;//数据 + + private TablePrintUtil() { + } + + /** + * 链式调用入口方法 + * + * @param data + * @return + */ + public static TablePrintUtil build(String[][] data) { + TablePrintUtil self = new TablePrintUtil(); + self.data = new ArrayList<>(Arrays.asList(data)); + return self; + } + + /** + * 链式调用入口方法,T可以是String[]、List、任意实体类 + * 由于java泛型不同无法重载,所以这里要写if instanceof进行类型判断 + * + * @param data + * @param + * @return + */ + public static TablePrintUtil build(List data) { + TablePrintUtil self = new TablePrintUtil(); + self.data = new ArrayList<>(); + if (data.size() <= 0) throw new RuntimeException("数据源至少得有一行吧"); + Object obj = data.get(0); + + + if (obj instanceof String[]) { + //如果泛型为String数组,则直接设置 + self.data = (List) data; + } else if (obj instanceof List) { + //如果泛型为List,则把list中的item依次转为String[],再设置 + int length = ((List) obj).size(); + for (Object item : data) { + List col = (List) item; + if (col.size() != length) throw new RuntimeException("数据源每列长度必须一致"); + self.data.add(col.toArray(new String[length])); + } + } else { + //如果泛型为实体类,则利用反射获取get方法列表,从而推算出属性列表。 + //根据反射得来的属性列表设置表格第一行thead + List

colList = getColList(obj); + String[] header = new String[colList.size()]; + for (int i = 0; i < colList.size(); i++) { + header[i] = colList.get(i).colName; + } + self.data.add(header); + //利用反射调用相应get方法获取属性值来设置表格tbody + for (int i = 0; i < data.size(); i++) { + String[] item = new String[colList.size()]; + for (int j = 0; j < colList.size(); j++) { + String value = null; + try { + value = obj.getClass().getMethod(colList.get(j).getMethodName).invoke(data.get(i)).toString(); + } catch (IllegalAccessException | InvocationTargetException | NoSuchMethodException e) { + e.printStackTrace(); + } + item[j] = value == null ? "null" : value; + } + self.data.add(item); + } + } + return self; + } + + private static class Col { + private String colName;//列名 + private String getMethodName;//get方法名 + } + + /** + * 利用反射获取get方法名和属性名 + * + * @return + */ + private static ListgetColList(Object obj) { + ListcolList = new ArrayList<>(); + Method[] methods = obj.getClass().getMethods(); + for (Method m : methods) { + StringBuilder getMethodName = new StringBuilder(m.getName()); + if (getMethodName.substring(0, 3).equals("get") && !m.getName().equals("getClass")) { + Col col = new Col(); + col.getMethodName = getMethodName.toString(); + char first = Character.toLowerCase(getMethodName.delete(0, 3).charAt(0)); + getMethodName.delete(0, 1).insert(0, first); + col.colName = getMethodName.toString(); + colList.add(col); + } + } + return colList; + } + + /** + * 获取字符串占的字符位数 + * + * @param str + * @return + */ + private int getStringCharLength(String str) { + Pattern p = Pattern.compile("[\u4e00-\u9fa5]");//利用正则找到中文 + Matcher m = p.matcher(str); + int count = 0; + while (m.find()) { + count++; + } + return str.length() + count; + } + + /** + * 纵向遍历获取数据每列的长度 + * + * @return + */ + private int[] getColLengths() { + int[] result = new int[data.get(0).length]; + for (int x = 0; x < result.length; x++) { + int max = 0; + for (int y = 0; y < data.size(); y++) { + int len = getStringCharLength(data.get(y)[x]); + if (len > max) { + max = len; + } + } + result[x] = max; + } + if (equilong) {//如果等宽表格 + int max = 0; + for (int len : result) { + if (len > max) max = len; + } + for (int i = 0; i < result.length; i++) { + result[i] = max; + } + } + return result; + } + + /** + * 取得表格字符串 + * + * @return + */ + public String getTableString() { + StringBuilder sb = new StringBuilder(); + int[] colLengths = getColLengths();//获取每列文字宽度 + StringBuilder line = new StringBuilder();//表格横向分隔线 + line.append(o); + for (int len : colLengths) { + int allLen = len + padding * 2;//还需要加上边距和分隔符的长度 + for (int i = 0; i < allLen; i++) { + line.append(h); + } + line.append(o); + } + sb.append(line).append("\r\n"); + for (int y = 0; y < data.size(); y++) { + sb.append(v); + for (int x = 0; x < data.get(y).length; x++) { + String cell = data.get(y)[x]; + switch (align) { + case ALIGN_LEFT: + for (int i = 0; i < padding; i++) sb.append(s); + sb.append(cell); + for (int i = 0; i < colLengths[x] - getStringCharLength(cell) + padding; i++) sb.append(s); + break; + case ALIGN_RIGHT: + for (int i = 0; i < colLengths[x] - getStringCharLength(cell) + padding; i++) sb.append(s); + sb.append(cell); + for (int i = 0; i < padding; i++) sb.append(s); + break; + case ALIGN_CENTER: + int space = colLengths[x] - getStringCharLength(cell); + int left = space / 2; + int right = space - left; + for (int i = 0; i < left + padding; i++) sb.append(s); + sb.append(cell); + for (int i = 0; i < right + padding; i++) sb.append(s); + break; + } + sb.append(v); + } + sb.append("\r\n"); + sb.append(line).append("\r\n"); + } + return sb.toString(); + } + + /** + * 直接打印表格 + */ + public void print() { + System.out.println(getTableString()); + } + + //下面是链式调用的set方法 + public TablePrintUtil setAlign(int align) { + this.align = align; + return this; + } + + public TablePrintUtil setEquilong(boolean equilong) { + this.equilong = equilong; + return this; + } + + public TablePrintUtil setPadding(int padding) { + this.padding = padding; + return this; + } + + public TablePrintUtil setH(char h) { + this.h = h; + return this; + } + + public TablePrintUtil setV(char v) { + this.v = v; + return this; + } + + public TablePrintUtil setO(char o) { + this.o = o; + return this; + } + + public TablePrintUtil setS(char s) { + this.s = s; + return this; + } + + /** + * 使用示例 + * + * @param args + */ + public static void main(String[] args) { + List data1 = new ArrayList<>(); + data1.add(new String[]{"用户名", "密码", "姓名"}); + data1.add(new String[]{"xiaoming", "xm123", "小明"}); + data1.add(new String[]{"xiaohong", "xh123", "小红"}); + TablePrintUtil.build(data1).print(); + + List> data2 = new ArrayList<>(); + data2.add(new ArrayList<>()); + data2.add(new ArrayList<>()); + data2.add(new ArrayList<>()); + data2.get(0).add("用户名"); + data2.get(0).add("密码"); + data2.get(0).add("姓名"); + data2.get(1).add("xiaoming"); + data2.get(1).add("xm123"); + data2.get(1).add("小明"); + data2.get(2).add("xiaohong"); + data2.get(2).add("xh123"); + data2.get(2).add("小红"); + TablePrintUtil.build(data2) + .setAlign(TablePrintUtil.ALIGN_LEFT) + .setPadding(5) + .setEquilong(true) + .print(); + + + class User { + String username; + String password; + String name; + + User(String username, String password, String name) { + this.username = username; + this.password = password; + this.name = name; + } + + public String getUsername() { + return username; + } + + public void setUsername(String username) { + this.username = username; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + } + List data3 = new ArrayList<>(); + data3.add(new User("xiaoming", "xm123", "小明")); + data3.add(new User("xiaohong", "xh123", "小红")); + TablePrintUtil.build(data3).setH('=').setV('!').print(); + } +} diff --git a/console/console-sink/src/test/java/com/dtstack/flinkx/AppTest.java b/console/console-sink/src/test/java/com/dtstack/flinkx/AppTest.java new file mode 100644 index 000000000..e03e5451f --- /dev/null +++ b/console/console-sink/src/test/java/com/dtstack/flinkx/AppTest.java @@ -0,0 +1,58 @@ +/* + * 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 com.dtstack.flinkx; + +import junit.framework.Test; +import junit.framework.TestCase; +import junit.framework.TestSuite; + +/** + * Unit test for simple App. + */ +public class AppTest + extends TestCase +{ + /** + * Create the test case + * + * @param testName name of the test case + */ + public AppTest(String testName ) + { + super( testName ); + } + + /** + * @return the suite of tests being tested + */ + public static Test suite() + { + return new TestSuite( AppTest.class ); + } + + /** + * Rigourous Test :-) + */ + public void testApp() + { + assertTrue( true ); + } +} diff --git a/console/pom.xml b/console/pom.xml new file mode 100644 index 000000000..983e1c185 --- /dev/null +++ b/console/pom.xml @@ -0,0 +1,34 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + sql.console + pom + + + console-sink + + + + + junit + junit + 3.8.1 + test + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + + + \ No newline at end of file diff --git a/docs/consoleSink.md b/docs/consoleSink.md new file mode 100644 index 000000000..206d7faaa --- /dev/null +++ b/docs/consoleSink.md @@ -0,0 +1,50 @@ +## 1.格式: +``` +CREATE TABLE tableName( + colName colType, + ... + colNameX colType + )WITH( + type ='console', + parallelism ='parllNum' + ); + +``` + +## 2.支持版本 +没有限制 + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName| 在 sql 中使用的名称;即注册到flink-table-env上的名称| +| colName | 列名称| +| colType | 列类型 [colType支持的类型](colType.md)| + +## 4.参数: + +|参数名称|含义|是否必填|默认值| +|----|----|----|----| +|type |表明 输出表类型[console]|是|| +| parallelism | 并行度设置|否|1| + +## 5.样例: +``` +CREATE TABLE MyResult( + name VARCHAR, + channel VARCHAR + )WITH( + type ='console', + parallelism ='1' + ) + ``` + + ## 6.输出结果: + ``` + +------+---------+ + | name | channel | + +------+---------+ + | aa | 02 | + +------+---------+ + ``` \ No newline at end of file diff --git a/pom.xml b/pom.xml index de0f1f7d6..c3ad4d24b 100644 --- a/pom.xml +++ b/pom.xml @@ -25,6 +25,7 @@ cassandra kafka08 serversocket + console From 36d49f2186dc451073b54debfed5ab814634c757 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 20 Dec 2018 13:57:54 +0800 Subject: [PATCH 225/470] code --- .../flink/sql/sink/console/ConsoleOutputFormat.java | 1 - .../flink/sql/sink/console/table/TablePrintUtil.java | 12 ++++++------ 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java index 55bed93be..7658e9979 100644 --- a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/ConsoleOutputFormat.java @@ -59,7 +59,6 @@ public void writeRecord(Tuple2 tuple2) throws IOException { Tuple2 tupleTrans = tuple2; Boolean retract = tupleTrans.getField(0); if (!retract) { - //FIXME 暂时不处理Mongo删除操作--->Mongo要求有key,所有认为都是可以执行update查找 return; } diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java index 8934d4394..b9f42386b 100644 --- a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java @@ -192,22 +192,22 @@ public String getTableString() { String cell = data.get(y)[x]; switch (align) { case ALIGN_LEFT: - for (int i = 0; i < padding; i++) sb.append(s); + for (int i = 0; i < padding; i++) {sb.append(s);} sb.append(cell); - for (int i = 0; i < colLengths[x] - getStringCharLength(cell) + padding; i++) sb.append(s); + for (int i = 0; i < colLengths[x] - getStringCharLength(cell) + padding; i++) {sb.append(s);} break; case ALIGN_RIGHT: - for (int i = 0; i < colLengths[x] - getStringCharLength(cell) + padding; i++) sb.append(s); + for (int i = 0; i < colLengths[x] - getStringCharLength(cell) + padding; i++) {sb.append(s);} sb.append(cell); - for (int i = 0; i < padding; i++) sb.append(s); + for (int i = 0; i < padding; i++) {sb.append(s);} break; case ALIGN_CENTER: int space = colLengths[x] - getStringCharLength(cell); int left = space / 2; int right = space - left; - for (int i = 0; i < left + padding; i++) sb.append(s); + for (int i = 0; i < left + padding; i++) {sb.append(s);} sb.append(cell); - for (int i = 0; i < right + padding; i++) sb.append(s); + for (int i = 0; i < right + padding; i++) {sb.append(s);} break; } sb.append(v); From 42a6479b858b4a45a3ef5befbdb1ce5b4d641c7d Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 20 Dec 2018 19:33:49 +0800 Subject: [PATCH 226/470] fix kafka offset bug --- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 4daf83d6f..339cf6264 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -123,7 +123,7 @@ public boolean check() { Preconditions.checkNotNull(topic, "kafka of topic is required"); //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + || offsetReset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 93e7d7642..6ce03a19b 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -123,7 +123,7 @@ public boolean check() { Preconditions.checkNotNull(topic, "kafka of topic is required"); //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + || offsetReset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index b151a93e9..693b8668d 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -124,7 +124,7 @@ public boolean check() { Preconditions.checkNotNull(topic, "kafka of topic is required"); //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + || offsetReset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } From 82021a4fd2aed6c24f7393c5a904aa14ec5dfddf Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 20 Dec 2018 21:30:55 +0800 Subject: [PATCH 227/470] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E6=B3=A8=E9=87=8A?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/src/main/java/com/dtstack/flink/sql/ClusterMode.java | 4 ++++ core/src/main/java/com/dtstack/flink/sql/Main.java | 5 +++++ .../com/dtstack/flink/sql/parser/CreateFuncParser.java | 2 +- .../com/dtstack/flink/sql/parser/CreateTableParser.java | 2 +- .../dtstack/flink/sql/parser/CreateTmpTableParser.java | 6 ++++++ .../com/dtstack/flink/sql/parser/InsertSqlParser.java | 3 +-- .../main/java/com/dtstack/flink/sql/parser/SqlTree.java | 2 +- .../java/com/dtstack/flink/sql/side/SideSQLParser.java | 1 - .../flink/sql/side/operator/SideAsyncOperator.java | 4 +++- .../flink/sql/side/operator/SideWithAllCacheOperator.java | 4 +++- .../java/com/dtstack/flink/sql/table/TableInfoParser.java | 1 - .../sql/source/kafka/table/KafkaSourceTableInfo.java | 8 ++++---- .../sql/source/kafka/table/KafkaSourceTableInfo.java | 8 ++++---- .../sql/source/kafka/table/KafkaSourceTableInfo.java | 6 +++--- .../sql/source/kafka/table/KafkaSourceTableInfo.java | 7 ++++--- 15 files changed, 40 insertions(+), 23 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/ClusterMode.java b/core/src/main/java/com/dtstack/flink/sql/ClusterMode.java index 024a31854..82105d9e7 100644 --- a/core/src/main/java/com/dtstack/flink/sql/ClusterMode.java +++ b/core/src/main/java/com/dtstack/flink/sql/ClusterMode.java @@ -31,4 +31,8 @@ public enum ClusterMode { ClusterMode(int type){ this.type = type; } + + public int getType(){ + return this.type; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 5ff2c6450..01f4474a0 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -102,8 +102,10 @@ public class Main { private static final int delayInterval = 10; //sec public static void main(String[] args) throws Exception { + LauncherOptionParser optionParser = new LauncherOptionParser(args); LauncherOptions launcherOptions = optionParser.getLauncherOptions(); + String sql = launcherOptions.getSql(); String name =launcherOptions.getName(); String addJarListStr = launcherOptions.getAddjar(); @@ -111,13 +113,16 @@ public static void main(String[] args) throws Exception { String remoteSqlPluginPath = launcherOptions.getRemoteSqlPluginPath(); String deployMode = launcherOptions.getMode(); String confProp = launcherOptions.getConfProp(); + sql = URLDecoder.decode(sql, Charsets.UTF_8.name()); SqlParser.setLocalSqlPluginRoot(localSqlPluginPath); List addJarFileList = Lists.newArrayList(); + if(!Strings.isNullOrEmpty(addJarListStr)){ addJarListStr = URLDecoder.decode(addJarListStr, Charsets.UTF_8.name()); addJarFileList = objMapper.readValue(addJarListStr, List.class); } + ClassLoader threadClassLoader = Thread.currentThread().getContextClassLoader(); DtClassLoader dtClassLoader = new DtClassLoader(new URL[]{}, threadClassLoader); Thread.currentThread().setContextClassLoader(dtClassLoader); diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java index 9b7017743..793dd6baa 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java @@ -24,7 +24,7 @@ import java.util.regex.Pattern; /** - * 解析创建自定义方法sql + * parser register udf sql * Date: 2018/6/26 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java index b5e4a4aa4..5e126e786 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java @@ -29,7 +29,7 @@ import java.util.regex.Pattern; /** - * 解析创建表结构sql + * parser create table sql * Date: 2018/6/26 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java index 5840b26a8..db18986b7 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -33,6 +33,12 @@ import static org.apache.calcite.sql.SqlKind.IDENTIFIER; +/** + * parser create tmp table sql + * Date: 2018/6/26 + * Company: www.dtstack.com + * @author yanxi + */ public class CreateTmpTableParser implements IParser { //select table tableName as select diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index f62b09d04..40629b139 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -32,8 +32,7 @@ import static org.apache.calcite.sql.SqlKind.IDENTIFIER; /** - * 解析flink sql - * sql 只支持 insert 开头的 + * parser flink sql * Date: 2018/6/22 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java b/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java index 3ed37c51e..754de0819 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java @@ -29,7 +29,7 @@ import java.util.Map; /** - * 解析sql获得的对象结构 + * parser sql to get the Sql Tree structure * Date: 2018/6/25 * Company: www.dtstack.com * @author xuchao diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 388bb5497..b412b1d18 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -20,7 +20,6 @@ package com.dtstack.flink.sql.side; -import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.calcite.config.Lex; import org.apache.calcite.sql.JoinType; import org.apache.calcite.sql.SqlAsOperator; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java index 121d557bf..29f0fe2e4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java @@ -44,6 +44,8 @@ public class SideAsyncOperator { private static final String PATH_FORMAT = "%sasyncside"; + private static final String OPERATOR_TYPE = "Async"; + private static final String ORDERED = "ordered"; @@ -57,7 +59,7 @@ private static AsyncReqRow loadAsyncReq(String sideType, String sqlRootDir, RowT String pluginJarPath = PluginUtil.getJarFileDirPath(pathOfType, sqlRootDir); DtClassLoader dtClassLoader = (DtClassLoader) classLoader; PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); - String className = PluginUtil.getSqlSideClassName(sideType, "side", "Async"); + String className = PluginUtil.getSqlSideClassName(sideType, "side", OPERATOR_TYPE); return dtClassLoader.loadClass(className).asSubclass(AsyncReqRow.class) .getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, SideTableInfo.class).newInstance(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java index 72a67d00b..725798848 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java @@ -43,6 +43,8 @@ public class SideWithAllCacheOperator { private static final String PATH_FORMAT = "%sallside"; + private static final String OPERATOR_TYPE = "All"; + private static AllReqRow loadFlatMap(String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { @@ -53,7 +55,7 @@ private static AllReqRow loadFlatMap(String sideType, String sqlRootDir, RowType DtClassLoader dtClassLoader = (DtClassLoader) classLoader; PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); - String className = PluginUtil.getSqlSideClassName(sideType, "side", "All"); + String className = PluginUtil.getSqlSideClassName(sideType, "side", OPERATOR_TYPE); return dtClassLoader.loadClass(className).asSubclass(AllReqRow.class).getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, SideTableInfo.class) .newInstance(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java index b3a07d6d5..4c2f67f68 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java @@ -20,7 +20,6 @@ package com.dtstack.flink.sql.table; -import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.enums.ETableType; import com.dtstack.flink.sql.parser.CreateTableParser; import com.dtstack.flink.sql.side.SideTableInfo; diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 4fb59f7d2..b59bb8055 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -52,7 +52,7 @@ public KafkaSourceTableInfo(){ super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap(); + public static Map kafkaParam = new HashMap<>(); public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); @@ -70,9 +70,9 @@ public Set getKafkaParamKeys(){ public boolean check() { Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); - Preconditions.checkState(kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("earliest") - || kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); - + String offset = kafkaParam.get("auto.offset.reset"); + Preconditions.checkState(offset.equalsIgnoreCase("latest") + || offset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 874e2c3b9..5e875f48b 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -59,7 +59,7 @@ public KafkaSourceTableInfo(){ } - public static Map kafkaParam = new HashMap(); + public static Map kafkaParam = new HashMap<>(); public void addKafkaParam(String key,String value){ @@ -81,9 +81,9 @@ public Set getKafkaParamKeys(){ public boolean check() { Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); - //Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); - Preconditions.checkState(kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("earliest") - || kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + String offset = kafkaParam.get("auto.offset.reset"); + Preconditions.checkState(offset.equalsIgnoreCase("latest") + || offset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 3184f4df3..3aedce2c7 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -82,9 +82,9 @@ public Set getKafkaParamKeys(){ public boolean check() { Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); - //Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); - Preconditions.checkState(kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("earliest") - || kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("latest"), "kafka of offsetReset set fail"); + String offset = kafkaParam.get("auto.offset.reset"); + Preconditions.checkState(offset.equalsIgnoreCase("latest") + || offset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index ffc270486..942104732 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -56,7 +56,7 @@ public KafkaSourceTableInfo() { super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap(); + public static Map kafkaParam = new HashMap<>(); public void addKafkaParam(String key, String value) { kafkaParam.put(key, value); @@ -74,8 +74,9 @@ public Set getKafkaParamKeys() { public boolean check() { Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); - Preconditions.checkState(kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("latest") - || kafkaParam.get("auto.offset.reset").toString().equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); + String offset = kafkaParam.get("auto.offset.reset"); + Preconditions.checkState(offset.equalsIgnoreCase("latest") + || offset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); return false; } From b3b7d4779ebc8106ac8a2291aac78b5d80a43e89 Mon Sep 17 00:00:00 2001 From: XuQianJin-Stars Date: Thu, 20 Dec 2018 23:47:12 +0800 Subject: [PATCH 228/470] change the doc --- README.md | 2 ++ docs/{serversocketSource.md => serverSocketSource.md} | 0 2 files changed, 2 insertions(+) rename docs/{serversocketSource.md => serverSocketSource.md} (100%) diff --git a/README.md b/README.md index e2056dd17..808b40432 100644 --- a/README.md +++ b/README.md @@ -143,6 +143,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack ## 2 结构 ### 2.1 源表插件 * [kafka 源表插件](docs/kafkaSource.md) +* [serverSocket 源表插件](docs/serverSocketSource.md) ### 2.2 结果表插件 * [elasticsearch 结果表插件](docs/elasticsearchSink.md) @@ -151,6 +152,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * [mongo 结果表插件](docs/mongoSink.md) * [redis 结果表插件](docs/redisSink.md) * [cassandra 结果表插件](docs/cassandraSink.md) +* [console 结果表插件](docs/consoleSink.md) ### 2.3 维表插件 * [hbase 维表插件](docs/hbaseSide.md) diff --git a/docs/serversocketSource.md b/docs/serverSocketSource.md similarity index 100% rename from docs/serversocketSource.md rename to docs/serverSocketSource.md From eb07eff0d59fe64e967b4f5a759030dd664e8983 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 21 Dec 2018 10:48:04 +0800 Subject: [PATCH 229/470] code optimese --- .../java/com/dtstack/flink/sql/launcher/LauncherMain.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index f32ed9008..f4c715093 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -84,13 +84,6 @@ public static void main(String[] args) throws Exception { PerJobSubmitter.submit(launcherOptions, jobGraph); } else { ClusterClient clusterClient = ClusterClientFactory.createClusterClient(launcherOptions); - pluginRoot = launcherOptions.getLocalSqlPluginPath(); - jarFile = new File(getLocalCoreJarPath(pluginRoot)); - remoteArgs = argList.toArray(new String[argList.size()]); - program = new PackagedProgram(jarFile, Lists.newArrayList(), remoteArgs); - if(StringUtils.isNotBlank(launcherOptions.getSavePointPath())){ - program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getSavePointPath(), BooleanUtils.toBoolean(launcherOptions.getAllowNonRestoredState()))); - } clusterClient.run(program, launcherOptions.getDefaultParallelism()); clusterClient.shutdown(); System.exit(0); From 5d752d92314b53427bc9a2301e9db9cac41c0476 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Sat, 22 Dec 2018 17:51:02 +0800 Subject: [PATCH 230/470] add timezone --- .../flink/sql/table/SourceTableInfo.java | 29 + .../CustomerWaterMarkerForLong.java | 22 +- .../CustomerWaterMarkerForTimeStamp.java | 18 +- .../sql/watermarker/WaterMarkerAssigner.java | 9 +- docs/kafkaSource.md | 3 + docs/timeZone.md | 601 ++++++++++++++++++ .../source/kafka/table/KafkaSourceParser.java | 2 + .../source/kafka/table/KafkaSourceParser.java | 1 + .../source/kafka/table/KafkaSourceParser.java | 1 + .../source/kafka/table/KafkaSourceParser.java | 3 +- .../flink/sql/launcher/LauncherMain.java | 4 +- 11 files changed, 680 insertions(+), 13 deletions(-) create mode 100644 docs/timeZone.md diff --git a/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java index d4282a968..ebe675c75 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java @@ -20,10 +20,14 @@ package com.dtstack.flink.sql.table; +import com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import java.util.ArrayList; import java.util.Map; +import java.util.TimeZone; /** * Reason: @@ -38,8 +42,12 @@ public abstract class SourceTableInfo extends TableInfo { public static final String SOURCE_DATA_TYPE = "sourcedatatype"; + public static final String TIME_ZONE_KEY="timezone"; + private String sourceDataType = "json"; + private String timeZone="Asia/Shanghai"; + private String eventTimeField; private Integer maxOutOrderness = 10; @@ -113,4 +121,25 @@ public String getSourceDataType() { public void setSourceDataType(String sourceDataType) { this.sourceDataType = sourceDataType; } + + public String getTimeZone() { + return timeZone; + } + + public void setTimeZone(String timeZone) { + if (timeZone==null){ + return; + } + timeZoneCheck(timeZone); + this.timeZone = timeZone; + } + + private void timeZoneCheck(String timeZone) { + ArrayList zones = Lists.newArrayList(TimeZone.getAvailableIDs()); + if (!zones.contains(timeZone)){ + throw new IllegalArgumentException(" timezone is Incorrect!"); + } + } + + } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java index 0a6874a81..7a466f118 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java @@ -16,7 +16,7 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.watermarker; @@ -27,6 +27,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.TimeZone; + /** * Custom watermark --- for eventtime * Date: 2017/12/28 @@ -40,13 +42,16 @@ public class CustomerWaterMarkerForLong extends AbsCustomerWaterMarker { private static final long serialVersionUID = 1L; + private TimeZone timezone; + private int pos; private long lastTime = 0; - public CustomerWaterMarkerForLong(Time maxOutOfOrderness, int pos) { + public CustomerWaterMarkerForLong(Time maxOutOfOrderness, int pos,String timezone) { super(maxOutOfOrderness); this.pos = pos; + this.timezone= TimeZone.getTimeZone(timezone); } @Override @@ -55,12 +60,21 @@ public long extractTimestamp(Row row) { try{ Long eveTime = MathUtil.getLongVal(row.getField(pos)); lastTime = eveTime; - eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - eveTime)/1000)); - return eveTime; + + eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(eveTime))/1000)); + + long restime=lastTime + timezone.getOffset(lastTime); + + return restime; }catch (Exception e){ logger.error("", e); } return lastTime; } + + public long convertTimeZone(long evenTime){ + long res = evenTime - timezone.getOffset(evenTime) + TimeZone.getDefault().getOffset(evenTime); + return res; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java index b4aee0302..779f837ff 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java @@ -27,6 +27,7 @@ import org.slf4j.LoggerFactory; import java.sql.Timestamp; +import java.util.TimeZone; /** * Custom watermark --- for eventtime @@ -45,10 +46,13 @@ public class CustomerWaterMarkerForTimeStamp extends AbsCustomerWaterMarker private long lastTime = 0; + private TimeZone timezone; - public CustomerWaterMarkerForTimeStamp(Time maxOutOfOrderness, int pos) { + + public CustomerWaterMarkerForTimeStamp(Time maxOutOfOrderness, int pos,String timezone) { super(maxOutOfOrderness); this.pos = pos; + this.timezone= TimeZone.getTimeZone(timezone); } @Override @@ -57,13 +61,19 @@ public long extractTimestamp(Row row) { Timestamp time = (Timestamp) row.getField(pos); lastTime = time.getTime(); - eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - time.getTime())/1000)); - return time.getTime(); + eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(lastTime))/1000)); + + long restime=lastTime + timezone.getOffset(lastTime); + + return restime; } catch (RuntimeException e) { logger.error("", e); } return lastTime; } - + public long convertTimeZone(long evenTime){ + long res = evenTime - timezone.getOffset(evenTime) + TimeZone.getDefault().getOffset(evenTime); + return res; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java index a29e8391b..4f386d75f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.watermarker; import com.dtstack.flink.sql.table.SourceTableInfo; +import com.google.common.collect.Lists; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -31,6 +32,8 @@ import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; +import java.util.TimeZone; + /** * define watermarker * Date: 2018/6/29 @@ -54,6 +57,8 @@ public DataStream assignWaterMarker(DataStream dataStream, RowTypeInfo type int maxOutOrderness = sourceTableInfo.getMaxOutOrderness(); + String timeZone=sourceTableInfo.getTimeZone(); + String[] fieldNames = typeInfo.getFieldNames(); TypeInformation[] fieldTypes = typeInfo.getFieldTypes(); @@ -75,9 +80,9 @@ public DataStream assignWaterMarker(DataStream dataStream, RowTypeInfo type AbsCustomerWaterMarker waterMarker = null; if(fieldType.getTypeClass().getTypeName().equalsIgnoreCase("java.sql.Timestamp")){ - waterMarker = new CustomerWaterMarkerForTimeStamp(Time.milliseconds(maxOutOrderness), pos); + waterMarker = new CustomerWaterMarkerForTimeStamp(Time.milliseconds(maxOutOrderness), pos,timeZone); }else if(fieldType.getTypeClass().getTypeName().equalsIgnoreCase("java.lang.Long")){ - waterMarker = new CustomerWaterMarkerForLong(Time.milliseconds(maxOutOrderness), pos); + waterMarker = new CustomerWaterMarkerForLong(Time.milliseconds(maxOutOrderness), pos,timeZone); }else{ throw new IllegalArgumentException("not support type of " + fieldType + ", current only support(timestamp, long)."); } diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index 2f6ce202c..a78239421 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -15,6 +15,8 @@ CREATE TABLE tableName( kafka.auto.offset.reset ='latest', kafka.topic ='topicName', parallelism ='parllNum', + --timezone='America/Los_Angeles', + timezone='Asia/Shanghai', sourcedatatype ='json' #可不设置 ); ``` @@ -46,6 +48,7 @@ CREATE TABLE tableName( |kafka.auto.offset.reset | 读取的topic 的offset初始位置[latest\|earliest\|指定offset值({"0":12312,"1":12321,"2":12312},{"partition_no":offset_value})]|否|latest| |parallelism | 并行度设置|否|1| |sourcedatatype | 数据类型|否|json| +|timezone|时区设置[timezone支持的参数](timeZone.md)|否|'Asia/Shanghai' **kafka相关参数可以自定义,使用kafka.开头即可。** ## 5.样例: diff --git a/docs/timeZone.md b/docs/timeZone.md new file mode 100644 index 000000000..ec6aa0ca3 --- /dev/null +++ b/docs/timeZone.md @@ -0,0 +1,601 @@ +* Africa/Abidjan +* Africa/Accra +* Africa/Addis_Ababa +* Africa/Algiers +* Africa/Asmara +* Africa/Asmera +* Africa/Bamako +* Africa/Bangui +* Africa/Banjul +* Africa/Bissau +* Africa/Blantyre +* Africa/Brazzaville +* Africa/Bujumbura +* Africa/Cairo +* Africa/Casablanca +* Africa/Ceuta +* Africa/Conakry +* Africa/Dakar +* Africa/Dar_es_Salaam +* Africa/Djibouti +* Africa/Douala +* Africa/El_Aaiun +* Africa/Freetown +* Africa/Gaborone +* Africa/Harare +* Africa/Johannesburg +* Africa/Juba +* Africa/Kampala +* Africa/Khartoum +* Africa/Kigali +* Africa/Kinshasa +* Africa/Lagos +* Africa/Libreville +* Africa/Lome +* Africa/Luanda +* Africa/Lubumbashi +* Africa/Lusaka +* Africa/Malabo +* Africa/Maputo +* Africa/Maseru +* Africa/Mbabane +* Africa/Mogadishu +* Africa/Monrovia +* Africa/Nairobi +* Africa/Ndjamena +* Africa/Niamey +* Africa/Nouakchott +* Africa/Ouagadougou +* Africa/Porto-Novo +* Africa/Sao_Tome +* Africa/Timbuktu +* Africa/Tripoli +* Africa/Tunis +* Africa/Windhoek +* America/Adak +* America/Anchorage +* America/Anguilla +* America/Antigua +* America/Araguaina +* America/Argentina/Buenos_Aires +* America/Argentina/Catamarca +* America/Argentina/ComodRivadavia +* America/Argentina/Cordoba +* America/Argentina/Jujuy +* America/Argentina/La_Rioja +* America/Argentina/Mendoza +* America/Argentina/Rio_Gallegos +* America/Argentina/Salta +* America/Argentina/San_Juan +* America/Argentina/San_Luis +* America/Argentina/Tucuman +* America/Argentina/Ushuaia +* America/Aruba +* America/Asuncion +* America/Atikokan +* America/Atka +* America/Bahia +* America/Bahia_Banderas +* America/Barbados +* America/Belem +* America/Belize +* America/Blanc-Sablon +* America/Boa_Vista +* America/Bogota +* America/Boise +* America/Buenos_Aires +* America/Cambridge_Bay +* America/Campo_Grande +* America/Cancun +* America/Caracas +* America/Catamarca +* America/Cayenne +* America/Cayman +* America/Chicago +* America/Chihuahua +* America/Coral_Harbour +* America/Cordoba +* America/Costa_Rica +* America/Creston +* America/Cuiaba +* America/Curacao +* America/Danmarkshavn +* America/Dawson +* America/Dawson_Creek +* America/Denver +* America/Detroit +* America/Dominica +* America/Edmonton +* America/Eirunepe +* America/El_Salvador +* America/Ensenada +* America/Fort_Nelson +* America/Fort_Wayne +* America/Fortaleza +* America/Glace_Bay +* America/Godthab +* America/Goose_Bay +* America/Grand_Turk +* America/Grenada +* America/Guadeloupe +* America/Guatemala +* America/Guayaquil +* America/Guyana +* America/Halifax +* America/Havana +* America/Hermosillo +* America/Indiana/Indianapolis +* America/Indiana/Knox +* America/Indiana/Marengo +* America/Indiana/Petersburg +* America/Indiana/Tell_City +* America/Indiana/Vevay +* America/Indiana/Vincennes +* America/Indiana/Winamac +* America/Indianapolis +* America/Inuvik +* America/Iqaluit +* America/Jamaica +* America/Jujuy +* America/Juneau +* America/Kentucky/Louisville +* America/Kentucky/Monticello +* America/Knox_IN +* America/Kralendijk +* America/La_Paz +* America/Lima +* America/Los_Angeles +* America/Louisville +* America/Lower_Princes +* America/Maceio +* America/Managua +* America/Manaus +* America/Marigot +* America/Martinique +* America/Matamoros +* America/Mazatlan +* America/Mendoza +* America/Menominee +* America/Merida +* America/Metlakatla +* America/Mexico_City +* America/Miquelon +* America/Moncton +* America/Monterrey +* America/Montevideo +* America/Montreal +* America/Montserrat +* America/Nassau +* America/New_York +* America/Nipigon +* America/Nome +* America/Noronha +* America/North_Dakota/Beulah +* America/North_Dakota/Center +* America/North_Dakota/New_Salem +* America/Ojinaga +* America/Panama +* America/Pangnirtung +* America/Paramaribo +* America/Phoenix +* America/Port-au-Prince +* America/Port_of_Spain +* America/Porto_Acre +* America/Porto_Velho +* America/Puerto_Rico +* America/Punta_Arenas +* America/Rainy_River +* America/Rankin_Inlet +* America/Recife +* America/Regina +* America/Resolute +* America/Rio_Branco +* America/Rosario +* America/Santa_Isabel +* America/Santarem +* America/Santiago +* America/Santo_Domingo +* America/Sao_Paulo +* America/Scoresbysund +* America/Shiprock +* America/Sitka +* America/St_Barthelemy +* America/St_Johns +* America/St_Kitts +* America/St_Lucia +* America/St_Thomas +* America/St_Vincent +* America/Swift_Current +* America/Tegucigalpa +* America/Thule +* America/Thunder_Bay +* America/Tijuana +* America/Toronto +* America/Tortola +* America/Vancouver +* America/Virgin +* America/Whitehorse +* America/Winnipeg +* America/Yakutat +* America/Yellowknife +* Antarctica/Casey +* Antarctica/Davis +* Antarctica/DumontDUrville +* Antarctica/Macquarie +* Antarctica/Mawson +* Antarctica/McMurdo +* Antarctica/Palmer +* Antarctica/Rothera +* Antarctica/South_Pole +* Antarctica/Syowa +* Antarctica/Troll +* Antarctica/Vostok +* Arctic/Longyearbyen +* Asia/Aden +* Asia/Almaty +* Asia/Amman +* Asia/Anadyr +* Asia/Aqtau +* Asia/Aqtobe +* Asia/Ashgabat +* Asia/Ashkhabad +* Asia/Atyrau +* Asia/Baghdad +* Asia/Bahrain +* Asia/Baku +* Asia/Bangkok +* Asia/Barnaul +* Asia/Beirut +* Asia/Bishkek +* Asia/Brunei +* Asia/Calcutta +* Asia/Chita +* Asia/Choibalsan +* Asia/Chongqing +* Asia/Chungking +* Asia/Colombo +* Asia/Dacca +* Asia/Damascus +* Asia/Dhaka +* Asia/Dili +* Asia/Dubai +* Asia/Dushanbe +* Asia/Famagusta +* Asia/Gaza +* Asia/Harbin +* Asia/Hebron +* Asia/Ho_Chi_Minh +* Asia/Hong_Kong +* Asia/Hovd +* Asia/Irkutsk +* Asia/Istanbul +* Asia/Jakarta +* Asia/Jayapura +* Asia/Jerusalem +* Asia/Kabul +* Asia/Kamchatka +* Asia/Karachi +* Asia/Kashgar +* Asia/Kathmandu +* Asia/Katmandu +* Asia/Khandyga +* Asia/Kolkata +* Asia/Krasnoyarsk +* Asia/Kuala_Lumpur +* Asia/Kuching +* Asia/Kuwait +* Asia/Macao +* Asia/Macau +* Asia/Magadan +* Asia/Makassar +* Asia/Manila +* Asia/Muscat +* Asia/Nicosia +* Asia/Novokuznetsk +* Asia/Novosibirsk +* Asia/Omsk +* Asia/Oral +* Asia/Phnom_Penh +* Asia/Pontianak +* Asia/Pyongyang +* Asia/Qatar +* Asia/Qyzylorda +* Asia/Rangoon +* Asia/Riyadh +* Asia/Saigon +* Asia/Sakhalin +* Asia/Samarkand +* Asia/Seoul +* Asia/Shanghai +* Asia/Singapore +* Asia/Srednekolymsk +* Asia/Taipei +* Asia/Tashkent +* Asia/Tbilisi +* Asia/Tehran +* Asia/Tel_Aviv +* Asia/Thimbu +* Asia/Thimphu +* Asia/Tokyo +* Asia/Tomsk +* Asia/Ujung_Pandang +* Asia/Ulaanbaatar +* Asia/Ulan_Bator +* Asia/Urumqi +* Asia/Ust-Nera +* Asia/Vientiane +* Asia/Vladivostok +* Asia/Yakutsk +* Asia/Yangon +* Asia/Yekaterinburg +* Asia/Yerevan +* Atlantic/Azores +* Atlantic/Bermuda +* Atlantic/Canary +* Atlantic/Cape_Verde +* Atlantic/Faeroe +* Atlantic/Faroe +* Atlantic/Jan_Mayen +* Atlantic/Madeira +* Atlantic/Reykjavik +* Atlantic/South_Georgia +* Atlantic/St_Helena +* Atlantic/Stanley +* Australia/ACT +* Australia/Adelaide +* Australia/Brisbane +* Australia/Broken_Hill +* Australia/Canberra +* Australia/Currie +* Australia/Darwin +* Australia/Eucla +* Australia/Hobart +* Australia/LHI +* Australia/Lindeman +* Australia/Lord_Howe +* Australia/Melbourne +* Australia/NSW +* Australia/North +* Australia/Perth +* Australia/Queensland +* Australia/South +* Australia/Sydney +* Australia/Tasmania +* Australia/Victoria +* Australia/West +* Australia/Yancowinna +* Brazil/Acre +* Brazil/DeNoronha +* Brazil/East +* Brazil/West +* CET +* CST6CDT +* Canada/Atlantic +* Canada/Central +* Canada/Eastern +* Canada/Mountain +* Canada/Newfoundland +* Canada/Pacific +* Canada/Saskatchewan +* Canada/Yukon +* Chile/Continental +* Chile/EasterIsland +* Cuba +* EET +* EST5EDT +* Egypt +* Eire +* Etc/GMT +* Etc/GMT+0 +* Etc/GMT+1 +* Etc/GMT+10 +* Etc/GMT+11 +* Etc/GMT+12 +* Etc/GMT+2 +* Etc/GMT+3 +* Etc/GMT+4 +* Etc/GMT+5 +* Etc/GMT+6 +* Etc/GMT+7 +* Etc/GMT+8 +* Etc/GMT+9 +* Etc/GMT-0 +* Etc/GMT-1 +* Etc/GMT-10 +* Etc/GMT-11 +* Etc/GMT-12 +* Etc/GMT-13 +* Etc/GMT-14 +* Etc/GMT-2 +* Etc/GMT-3 +* Etc/GMT-4 +* Etc/GMT-5 +* Etc/GMT-6 +* Etc/GMT-7 +* Etc/GMT-8 +* Etc/GMT-9 +* Etc/GMT0 +* Etc/Greenwich +* Etc/UCT +* Etc/UTC +* Etc/Universal +* Etc/Zulu +* Europe/Amsterdam +* Europe/Andorra +* Europe/Astrakhan +* Europe/Athens +* Europe/Belfast +* Europe/Belgrade +* Europe/Berlin +* Europe/Bratislava +* Europe/Brussels +* Europe/Bucharest +* Europe/Budapest +* Europe/Busingen +* Europe/Chisinau +* Europe/Copenhagen +* Europe/Dublin +* Europe/Gibraltar +* Europe/Guernsey +* Europe/Helsinki +* Europe/Isle_of_Man +* Europe/Istanbul +* Europe/Jersey +* Europe/Kaliningrad +* Europe/Kiev +* Europe/Kirov +* Europe/Lisbon +* Europe/Ljubljana +* Europe/London +* Europe/Luxembourg +* Europe/Madrid +* Europe/Malta +* Europe/Mariehamn +* Europe/Minsk +* Europe/Monaco +* Europe/Moscow +* Europe/Nicosia +* Europe/Oslo +* Europe/Paris +* Europe/Podgorica +* Europe/Prague +* Europe/Riga +* Europe/Rome +* Europe/Samara +* Europe/San_Marino +* Europe/Sarajevo +* Europe/Saratov +* Europe/Simferopol +* Europe/Skopje +* Europe/Sofia +* Europe/Stockholm +* Europe/Tallinn +* Europe/Tirane +* Europe/Tiraspol +* Europe/Ulyanovsk +* Europe/Uzhgorod +* Europe/Vaduz +* Europe/Vatican +* Europe/Vienna +* Europe/Vilnius +* Europe/Volgograd +* Europe/Warsaw +* Europe/Zagreb +* Europe/Zaporozhye +* Europe/Zurich +* GB +* GB-Eire +* GMT +* GMT0 +* Greenwich +* Hongkong +* Iceland +* Indian/Antananarivo +* Indian/Chagos +* Indian/Christmas +* Indian/Cocos +* Indian/Comoro +* Indian/Kerguelen +* Indian/Mahe +* Indian/Maldives +* Indian/Mauritius +* Indian/Mayotte +* Indian/Reunion +* Iran +* Israel +* Jamaica +* Japan +* Kwajalein +* Libya +* MET +* MST7MDT +* Mexico/BajaNorte +* Mexico/BajaSur +* Mexico/General +* NZ +* NZ-CHAT +* Navajo +* PRC +* PST8PDT +* Pacific/Apia +* Pacific/Auckland +* Pacific/Bougainville +* Pacific/Chatham +* Pacific/Chuuk +* Pacific/Easter +* Pacific/Efate +* Pacific/Enderbury +* Pacific/Fakaofo +* Pacific/Fiji +* Pacific/Funafuti +* Pacific/Galapagos +* Pacific/Gambier +* Pacific/Guadalcanal +* Pacific/Guam +* Pacific/Honolulu +* Pacific/Johnston +* Pacific/Kiritimati +* Pacific/Kosrae +* Pacific/Kwajalein +* Pacific/Majuro +* Pacific/Marquesas +* Pacific/Midway +* Pacific/Nauru +* Pacific/Niue +* Pacific/Norfolk +* Pacific/Noumea +* Pacific/Pago_Pago +* Pacific/Palau +* Pacific/Pitcairn +* Pacific/Pohnpei +* Pacific/Ponape +* Pacific/Port_Moresby +* Pacific/Rarotonga +* Pacific/Saipan +* Pacific/Samoa +* Pacific/Tahiti +* Pacific/Tarawa +* Pacific/Tongatapu +* Pacific/Truk +* Pacific/Wake +* Pacific/Wallis +* Pacific/Yap +* Poland +* Portugal +* ROK +* Singapore +* SystemV/AST4 +* SystemV/AST4ADT +* SystemV/CST6 +* SystemV/CST6CDT +* SystemV/EST5 +* SystemV/EST5EDT +* SystemV/HST10 +* SystemV/MST7 +* SystemV/MST7MDT +* SystemV/PST8 +* SystemV/PST8PDT +* SystemV/YST9 +* SystemV/YST9YDT +* Turkey +* UCT +* US/Alaska +* US/Aleutian +* US/Arizona +* US/Central +* US/East-Indiana +* US/Eastern +* US/Hawaii +* US/Indiana-Starke +* US/Michigan +* US/Mountain +* US/Pacific +* US/Pacific-New +* US/Samoa +* UTC +* Universal +* W-SU +* WET +* Zulu + + diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 9af3c88ca..eb085ac78 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -46,6 +46,8 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map argList = optionParser.getProgramExeArgList(); if(mode.equals(ClusterMode.local.name())) { - String[] localArgs = argList.toArray(new String[argList.size()]); - Main.main(localArgs); + //String[] localArgs = argList.toArray(new String[argList.size()]); + Main.main(args); }else{ String pluginRoot = launcherOptions.getLocalSqlPluginPath(); File jarFile = new File(getLocalCoreJarPath(pluginRoot)); From 1af27f54fa341b05aa7a61dbb25d316ac7980b65 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Sat, 22 Dec 2018 18:13:49 +0800 Subject: [PATCH 231/470] fix watermark lasttime --- .../sql/watermarker/CustomerWaterMarkerForLong.java | 10 ++++------ .../watermarker/CustomerWaterMarkerForTimeStamp.java | 11 ++++++----- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java index 7a466f118..57143a5ea 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java @@ -59,17 +59,15 @@ public long extractTimestamp(Row row) { try{ Long eveTime = MathUtil.getLongVal(row.getField(pos)); - lastTime = eveTime; + Long extractTime=eveTime; - eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(eveTime))/1000)); + lastTime = extractTime + timezone.getOffset(extractTime); + eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(extractTime))/1000)); - long restime=lastTime + timezone.getOffset(lastTime); - - return restime; + return lastTime; }catch (Exception e){ logger.error("", e); } - return lastTime; } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java index 779f837ff..e4af63744 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java @@ -16,7 +16,7 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.watermarker; @@ -59,13 +59,14 @@ public CustomerWaterMarkerForTimeStamp(Time maxOutOfOrderness, int pos,String ti public long extractTimestamp(Row row) { try { Timestamp time = (Timestamp) row.getField(pos); - lastTime = time.getTime(); - eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(lastTime))/1000)); + long extractTime=time.getTime(); + + lastTime = extractTime + timezone.getOffset(extractTime); - long restime=lastTime + timezone.getOffset(lastTime); + eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(extractTime))/1000)); - return restime; + return lastTime; } catch (RuntimeException e) { logger.error("", e); } From 7ee6212cc3ee9fbcc53a97caada9956bc1b3d4d8 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Sat, 22 Dec 2018 22:19:13 +0800 Subject: [PATCH 232/470] side async param --- .../dtstack/flink/sql/side/SideTableInfo.java | 24 +++++++++++++++++++ .../sql/side/operator/SideAsyncOperator.java | 7 ++---- .../flink/sql/table/AbsSideTableParser.java | 16 +++++++++++++ .../CustomerWaterMarkerForLong.java | 1 + docs/mysqlSide.md | 5 +++- 5 files changed, 47 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java index 02c9b4e02..e21389ea7 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideTableInfo.java @@ -47,12 +47,20 @@ public abstract class SideTableInfo extends TableInfo implements Serializable { public static final String CACHE_MODE_KEY = "cacheMode"; + public static final String ASYNC_CAP_KEY = "asyncCapacity"; + + public static final String ASYNC_TIMEOUT_KEY = "asyncTimeout"; + private String cacheType = "none";//None or LRU or ALL private int cacheSize = 10000; private long cacheTimeout = 60 * 1000;// + private int asyncCapacity=100; + + private int asyncTimeout=10000; + private boolean partitionedJoin = false; private String cacheMode="ordered"; @@ -107,4 +115,20 @@ public String getCacheMode() { public void setCacheMode(String cacheMode) { this.cacheMode = cacheMode; } + + public int getAsyncCapacity() { + return asyncCapacity; + } + + public void setAsyncCapacity(int asyncCapacity) { + this.asyncCapacity = asyncCapacity; + } + + public int getAsyncTimeout() { + return asyncTimeout; + } + + public void setAsyncTimeout(int asyncTimeout) { + this.asyncTimeout = asyncTimeout; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java index 29f0fe2e4..688a19c35 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java @@ -49,9 +49,6 @@ public class SideAsyncOperator { private static final String ORDERED = "ordered"; - //TODO need to set by create table task - private static int asyncCapacity = 100; - private static AsyncReqRow loadAsyncReq(String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); @@ -70,10 +67,10 @@ public static DataStream getSideJoinDataStream(DataStream inputStream, String si //TODO How much should be set for the degree of parallelism? Timeout? capacity settings? if (ORDERED.equals(sideTableInfo.getCacheMode())){ - return AsyncDataStream.orderedWait(inputStream, asyncDbReq, 10000, TimeUnit.MILLISECONDS, asyncCapacity) + return AsyncDataStream.orderedWait(inputStream, asyncDbReq, sideTableInfo.getAsyncTimeout(), TimeUnit.MILLISECONDS, sideTableInfo.getAsyncCapacity()) .setParallelism(sideTableInfo.getParallelism()); }else { - return AsyncDataStream.unorderedWait(inputStream, asyncDbReq, 10000, TimeUnit.MILLISECONDS, asyncCapacity) + return AsyncDataStream.unorderedWait(inputStream, asyncDbReq, sideTableInfo.getAsyncTimeout(), TimeUnit.MILLISECONDS, sideTableInfo.getAsyncCapacity()) .setParallelism(sideTableInfo.getParallelism()); } diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java index 526a632dd..ae8135256 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java @@ -95,6 +95,22 @@ protected void parseCacheProp(SideTableInfo sideTableInfo, Map p } sideTableInfo.setCacheMode(cachemode.toLowerCase()); } + + if(props.containsKey(SideTableInfo.ASYNC_CAP_KEY.toLowerCase())){ + Integer asyncCap = MathUtil.getIntegerVal(props.get(SideTableInfo.ASYNC_CAP_KEY.toLowerCase())); + if(asyncCap < 0){ + throw new RuntimeException("asyncCapacity size need > 0."); + } + sideTableInfo.setAsyncCapacity(asyncCap); + } + + if(props.containsKey(SideTableInfo.ASYNC_TIMEOUT_KEY.toLowerCase())){ + Integer asyncTimeout = MathUtil.getIntegerVal(props.get(SideTableInfo.ASYNC_TIMEOUT_KEY.toLowerCase())); + if (asyncTimeout<0){ + throw new RuntimeException("asyncTimeout size need > 0."); + } + sideTableInfo.setAsyncTimeout(asyncTimeout); + } } } } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java index 57143a5ea..632c10c7b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java @@ -62,6 +62,7 @@ public long extractTimestamp(Row row) { Long extractTime=eveTime; lastTime = extractTime + timezone.getOffset(extractTime); + eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(extractTime))/1000)); return lastTime; diff --git a/docs/mysqlSide.md b/docs/mysqlSide.md index c6f5d741a..b17c72bb6 100644 --- a/docs/mysqlSide.md +++ b/docs/mysqlSide.md @@ -53,7 +53,8 @@ * cacheSize: 缓存的条目数量 * cacheTTLMs:缓存的过期时间(ms) * cacheMode: (unordered|ordered)异步加载是有序还是无序,默认有序。 - + * asyncCapacity:异步请求容量,默认1000 + * asyncTimeout:异步请求超时时间,默认10000毫秒 ## 5.样例 ``` @@ -72,6 +73,8 @@ create table sideTable( cacheSize ='10000', cacheTTLMs ='60000', cacheMode='unordered', + asyncCapacity='1000', + asyncTimeout='10000' parallelism ='1', partitionedJoin='false' ); From eea976f813405548f3dfac96068caa0328215052 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Mon, 24 Dec 2018 18:39:30 +0800 Subject: [PATCH 233/470] fix asyn rdb write only one record --- .../flink/sql/side/rdb/async/RdbAsyncReqRow.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index 73fab0e9f..6909b312a 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -90,11 +90,14 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except return; } else if (ECacheContentType.MultiLine == val.getType()) { + List rowList = Lists.newArrayList(); for (Object jsonArray : (List) val.getContent()) { Row row = fillData(input, jsonArray); - resultFuture.complete(Collections.singleton(row)); + rowList.add(row); } + resultFuture.complete(rowList); + } else { throw new RuntimeException("not support cache obj type " + val.getType()); } @@ -122,18 +125,18 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except int resultSize = rs.result().getResults().size(); if (resultSize > 0) { + List rowList = Lists.newArrayList(); for (JsonArray line : rs.result().getResults()) { - Row row = fillData(input, line); if (openCache()) { cacheContent.add(line); } - resultFuture.complete(Collections.singleton(row)); + rowList.add(row); } - if (openCache()) { putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); } + resultFuture.complete(rowList); } else { dealMissKey(input, resultFuture); if (openCache()) { From cd89ed0f5ebc3616c839e30ed43524e27704ea87 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 25 Dec 2018 11:54:28 +0800 Subject: [PATCH 234/470] fix parse bug --- core/src/main/java/com/dtstack/flink/sql/Main.java | 5 ----- .../java/com/dtstack/flink/sql/launcher/LauncherMain.java | 4 +--- 2 files changed, 1 insertion(+), 8 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 01f4474a0..9c71fce1c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -40,17 +40,12 @@ import org.apache.calcite.config.Lex; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlNode; -import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.CommandLineParser; -import org.apache.commons.cli.DefaultParser; -import org.apache.commons.cli.Options; import org.apache.commons.io.Charsets; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 4467d1a44..47718581f 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -67,13 +67,11 @@ public static void main(String[] args) throws Exception { String mode = launcherOptions.getMode(); List argList = optionParser.getProgramExeArgList(); if(mode.equals(ClusterMode.local.name())) { - //String[] localArgs = argList.toArray(new String[argList.size()]); Main.main(args); }else{ String pluginRoot = launcherOptions.getLocalSqlPluginPath(); File jarFile = new File(getLocalCoreJarPath(pluginRoot)); - String[] remoteArgs = argList.toArray(new String[argList.size()]); - PackagedProgram program = new PackagedProgram(jarFile, Lists.newArrayList(), remoteArgs); + PackagedProgram program = new PackagedProgram(jarFile, Lists.newArrayList(), args); if(StringUtils.isNotBlank(launcherOptions.getSavePointPath())){ program.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(launcherOptions.getSavePointPath(), BooleanUtils.toBoolean(launcherOptions.getAllowNonRestoredState()))); } From c3b55d9dac79b6975a74dd2658f9676fb8194105 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 25 Dec 2018 18:02:15 +0800 Subject: [PATCH 235/470] console sink add log and serversocketSource modify exception scope --- .../sink/console/table/TablePrintUtil.java | 5 +++ .../CustomerSocketTextStreamFunction.java | 44 ++++++++++--------- 2 files changed, 29 insertions(+), 20 deletions(-) diff --git a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java index b9f42386b..8813da619 100644 --- a/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java +++ b/console/console-sink/src/main/java/com/dtstack/flink/sql/sink/console/table/TablePrintUtil.java @@ -1,5 +1,8 @@ package com.dtstack.flink.sql.sink.console.table; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.ArrayList; @@ -15,6 +18,7 @@ * @author xuqianjin */ public class TablePrintUtil { + private static final Logger LOG = LoggerFactory.getLogger(TablePrintUtil.class); public static final int ALIGN_LEFT = 1;//左对齐 public static final int ALIGN_RIGHT = 2;//右对齐 public static final int ALIGN_CENTER = 3;//居中对齐 @@ -222,6 +226,7 @@ public String getTableString() { * 直接打印表格 */ public void print() { + LOG.info("\n"+getTableString()); System.out.println(getTableString()); } diff --git a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java index 8e6004e90..ce9a44895 100644 --- a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java @@ -25,6 +25,8 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.types.Row; import org.apache.flink.util.IOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.BufferedReader; import java.io.IOException; @@ -42,11 +44,12 @@ * @author maqi */ public class CustomerSocketTextStreamFunction implements SourceFunction { + private static final Logger LOG = LoggerFactory.getLogger(CustomerSocketTextStreamFunction.class); /** * Default delay between successive connection attempts. */ - private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 500; + private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 2000; /** * Default connection timeout when connecting to the server socket (infinite). @@ -92,32 +95,33 @@ public void run(SourceContext ctx) throws Exception { long attempt = 0; while (isRunning) { - - try (Socket socket = new Socket()) { + try { + Socket socket = new Socket(); currentSocket = socket; - socket.connect(new InetSocketAddress(tableInfo.getHostname(), tableInfo.getPort()), CONNECTION_TIMEOUT_TIME); - try (BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream()))) { - - char[] cbuf = new char[8192]; - int bytesRead; - while (isRunning && (bytesRead = reader.read(cbuf)) != -1) { - buffer.append(cbuf, 0, bytesRead); - int delimPos; - String delimiter = tableInfo.getDelimiter(); - while (buffer.length() >= delimiter.length() && (delimPos = buffer.indexOf(delimiter)) != -1) { - String record = buffer.substring(0, delimPos); - // truncate trailing carriage return - if (delimiter.equals("\n") && record.endsWith("\r")) { - record = record.substring(0, record.length() - 1); - } - ctx.collect(convertToRow(record)); - buffer.delete(0, delimPos + delimiter.length()); + + BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream())); + char[] cbuf = new char[8192]; + int bytesRead; + while (isRunning && (bytesRead = reader.read(cbuf)) != -1) { + buffer.append(cbuf, 0, bytesRead); + int delimPos; + String delimiter = tableInfo.getDelimiter(); + while (buffer.length() >= delimiter.length() && (delimPos = buffer.indexOf(delimiter)) != -1) { + String record = buffer.substring(0, delimPos); + // truncate trailing carriage return + if (delimiter.equals("\n") && record.endsWith("\r")) { + record = record.substring(0, record.length() - 1); } + ctx.collect(convertToRow(record)); + buffer.delete(0, delimPos + delimiter.length()); } } + } catch (Exception e) { + LOG.info("Connection server failed, Please check configuration !!!!!!!!!!!!!!!!"); } + // if we dropped out of this loop due to an EOF, sleep and retry if (isRunning) { attempt++; From 1d6747c62f1e59ace5d95bb9d02052b6bfe091c4 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 26 Dec 2018 16:23:43 +0800 Subject: [PATCH 236/470] fix kafka cvs pattern error and remove kafka08 monitor --- kafka08/kafka08-source/pom.xml | 2 +- .../com/dtstack/flink/sql/source/kafka/KafkaSource.java | 8 ++++---- .../sql/source/kafka/consumer/CustomerCsvConsumer.java | 9 --------- .../sql/source/kafka/consumer/CustomerJsonConsumer.java | 9 --------- .../deserialization/CustomerCommonDeserialization.java | 8 ++++---- .../deserialization/CustomerCsvDeserialization.java | 8 ++++---- .../deserialization/CustomerJsonDeserialization.java | 8 ++++---- .../com/dtstack/flink/sql/source/kafka/KafkaSource.java | 8 ++++---- .../com/dtstack/flink/sql/source/kafka/KafkaSource.java | 8 ++++---- .../com/dtstack/flink/sql/source/kafka/KafkaSource.java | 8 ++++---- 10 files changed, 29 insertions(+), 47 deletions(-) diff --git a/kafka08/kafka08-source/pom.xml b/kafka08/kafka08-source/pom.xml index df98c2603..64264fd46 100644 --- a/kafka08/kafka08-source/pom.xml +++ b/kafka08/kafka08-source/pom.xml @@ -20,7 +20,7 @@ org.apache.flink flink-connector-kafka-0.8_2.11 - 1.6.0 + ${flink.version} diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 2e7136666..bfbffdf14 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -90,19 +90,19 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } } else if ("csv".equalsIgnoreCase(kafka08SourceTableInfo.getSourceDataType())) { if (topicIsPattern) { - kafkaSrc = new CustomerCsvConsumer(topicName, + kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), new com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization(typeInformation, kafka08SourceTableInfo.getFieldDelimiter(), kafka08SourceTableInfo.getLengthCheckPolicy()), props); } else { - kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), + kafkaSrc = new CustomerCsvConsumer(topicName, new CustomerCsvDeserialization(typeInformation, kafka08SourceTableInfo.getFieldDelimiter(), kafka08SourceTableInfo.getLengthCheckPolicy()), props); } } else { if (topicIsPattern) { - kafkaSrc = new CustomerCommonConsumer(topicName, new com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization(), props); + kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new com.dtstack.flink.sql.source.kafka.deserialization.CustomerCommonDeserialization(), props); } else { - kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); + kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); } } diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java index ad859e485..7dc95450e 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerCsvConsumer.java @@ -20,20 +20,11 @@ import com.dtstack.flink.sql.source.AbsDeserialization; import com.dtstack.flink.sql.source.kafka.deserialization.CustomerCsvDeserialization; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; -import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; -import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; -import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.types.Row; -import org.apache.flink.util.SerializedValue; import java.util.Arrays; -import java.util.Map; import java.util.Properties; import java.util.regex.Pattern; diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java index 0b775a4dc..b627d81c0 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/consumer/CustomerJsonConsumer.java @@ -20,20 +20,11 @@ import com.dtstack.flink.sql.source.AbsDeserialization; import com.dtstack.flink.sql.source.kafka.deserialization.CustomerJsonDeserialization; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; -import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; -import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; -import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; -import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.types.Row; -import org.apache.flink.util.SerializedValue; import java.util.Arrays; -import java.util.Map; import java.util.Properties; import java.util.regex.Pattern; diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java index f35bacce7..c92ce4aa4 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java @@ -48,9 +48,9 @@ public class CustomerCommonDeserialization extends AbsDeserialization imple @Override public Row deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) { - numInRecord.inc(); - numInBytes.inc(message.length); - numInBytes.inc(messageKey.length); + //numInRecord.inc(); + //numInBytes.inc(message.length); + //numInBytes.inc(messageKey.length); try { Row row = Row.of( @@ -63,7 +63,7 @@ public Row deserialize(byte[] messageKey, byte[] message, String topic, int part return row; } catch (Throwable t) { LOG.error(t.getMessage()); - dirtyDataCounter.inc(); + // dirtyDataCounter.inc(); return null; } } diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java index e67d79400..0ca57e0b3 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -80,8 +80,8 @@ public CustomerCsvDeserialization(TypeInformation typeInfo, String fieldDel public Row deserialize(byte[] message) throws IOException { try { - numInRecord.inc(); - numInBytes.inc(message.length); + //numInRecord.inc(); + //numInBytes.inc(message.length); String[] fieldsList = null; if (message != null && message.length > 0){ fieldsList = new String(message).split(fieldDelimiter); @@ -103,11 +103,11 @@ public Row deserialize(byte[] message) throws IOException { } } - numInResolveRecord.inc(); + //numInResolveRecord.inc(); return row; } catch (Throwable t) { //add metric of dirty data - dirtyDataCounter.inc(); + //dirtyDataCounter.inc(); throw new RuntimeException(t); } } diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java index 72b8b2c8e..900717b98 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java @@ -80,8 +80,8 @@ public CustomerJsonDeserialization(TypeInformation typeInfo){ public Row deserialize(byte[] message) throws IOException { try { - numInRecord.inc(); - numInBytes.inc(message.length); + // numInRecord.inc(); + // numInBytes.inc(message.length); JsonNode root = objectMapper.readTree(message); Row row = new Row(fieldNames.length); @@ -102,12 +102,12 @@ public Row deserialize(byte[] message) throws IOException { } } - numInResolveRecord.inc(); + // numInResolveRecord.inc(); return row; } catch (Throwable t) { //add metric of dirty data LOG.error(t.getMessage()); - dirtyDataCounter.inc(); + // dirtyDataCounter.inc(); return null; } } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index c01366e4c..292f70f51 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -100,19 +100,19 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } } else if ("csv".equalsIgnoreCase(kafka09SourceTableInfo.getSourceDataType())) { if (topicIsPattern) { - kafkaSrc = new CustomerCsvConsumer(topicName, + kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), new CustomerCsvDeserialization(typeInformation, kafka09SourceTableInfo.getFieldDelimiter(), kafka09SourceTableInfo.getLengthCheckPolicy()), props); } else { - kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), + kafkaSrc = new CustomerCsvConsumer(topicName, new CustomerCsvDeserialization(typeInformation, kafka09SourceTableInfo.getFieldDelimiter(), kafka09SourceTableInfo.getLengthCheckPolicy()), props); } } else { if (topicIsPattern) { - kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); - } else { kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); + } else { + kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); } } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 586f046af..ef85e5b56 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -103,19 +103,19 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } } else if ("csv".equalsIgnoreCase(kafka010SourceTableInfo.getSourceDataType())) { if (topicIsPattern) { - kafkaSrc = new CustomerCsvConsumer(topicName, + kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), new CustomerCsvDeserialization(typeInformation, kafka010SourceTableInfo.getFieldDelimiter(), kafka010SourceTableInfo.getLengthCheckPolicy()), props); } else { - kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), + kafkaSrc = new CustomerCsvConsumer(topicName, new CustomerCsvDeserialization(typeInformation, kafka010SourceTableInfo.getFieldDelimiter(), kafka010SourceTableInfo.getLengthCheckPolicy()), props); } } else { if (topicIsPattern) { - kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); - } else { kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); + } else { + kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); } } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 02a6bd0b4..0f538aa1d 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -100,19 +100,19 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv } } else if ("csv".equalsIgnoreCase(kafka011SourceTableInfo.getSourceDataType())) { if (topicIsPattern) { - kafkaSrc = new CustomerCsvConsumer(topicName, + kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), new CustomerCsvDeserialization(typeInformation, kafka011SourceTableInfo.getFieldDelimiter(), kafka011SourceTableInfo.getLengthCheckPolicy()), props); } else { - kafkaSrc = new CustomerCsvConsumer(Pattern.compile(topicName), + kafkaSrc = new CustomerCsvConsumer(topicName, new CustomerCsvDeserialization(typeInformation, kafka011SourceTableInfo.getFieldDelimiter(), kafka011SourceTableInfo.getLengthCheckPolicy()), props); } } else { if (topicIsPattern) { - kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); - } else { kafkaSrc = new CustomerCommonConsumer(Pattern.compile(topicName), new CustomerCommonDeserialization(), props); + } else { + kafkaSrc = new CustomerCommonConsumer(topicName, new CustomerCommonDeserialization(), props); } } From 9e42b44e07391a579ff3189a21056d868726ae2b Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 26 Dec 2018 19:18:56 +0800 Subject: [PATCH 237/470] more kafka bugfix --- .../dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- .../dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- .../dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 3 +-- .../dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java | 2 +- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 2 +- 8 files changed, 8 insertions(+), 9 deletions(-) diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index 20e228d30..5dae21742 100644 --- a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -42,7 +42,7 @@ public KafkaSinkTableInfo(){ super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap(); + public Map kafkaParam = new HashMap(); public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index b59bb8055..9f7e36f56 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -52,7 +52,7 @@ public KafkaSourceTableInfo(){ super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap<>(); + public Map kafkaParam = new HashMap<>(); public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index bad84231d..48daf57dd 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -40,7 +40,7 @@ public KafkaSinkTableInfo(){ super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap(); + public Map kafkaParam = new HashMap(); public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 5e875f48b..d2f7d212d 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -59,7 +59,7 @@ public KafkaSourceTableInfo(){ } - public static Map kafkaParam = new HashMap<>(); + public Map kafkaParam = new HashMap<>(); public void addKafkaParam(String key,String value){ diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index 6557355d9..0235547d2 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -42,7 +42,7 @@ public KafkaSinkTableInfo(){ super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap(); + public Map kafkaParam = new HashMap(); public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 3aedce2c7..de75c27cb 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -57,8 +57,7 @@ public KafkaSourceTableInfo(){ super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap(); - + public Map kafkaParam = new HashMap(); public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index b6b2dd978..78edf17b4 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -41,7 +41,7 @@ public KafkaSinkTableInfo(){ super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap(); + public Map kafkaParam = new HashMap(); public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 942104732..983a1882a 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -56,7 +56,7 @@ public KafkaSourceTableInfo() { super.setType(CURR_TYPE); } - public static Map kafkaParam = new HashMap<>(); + public Map kafkaParam = new HashMap<>(); public void addKafkaParam(String key, String value) { kafkaParam.put(key, value); From 79397c8b1f08bf6520fc4ff34322e16ec3d6ba1b Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 27 Dec 2018 15:05:09 +0800 Subject: [PATCH 238/470] yarnper submit bugfix --- .../sql/launcher/ClusterClientFactory.java | 2 +- .../sql/launcher/perjob/PerJobSubmitter.java | 26 ++++++++++++------- 2 files changed, 18 insertions(+), 10 deletions(-) diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index be8611757..4513649fa 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -63,7 +63,7 @@ public static ClusterClient createClusterClient(LauncherOptions launcherOptions) String mode = launcherOptions.getMode(); if(mode.equals(ClusterMode.standalone.name())) { return createStandaloneClient(launcherOptions); - } else if(mode.equals(ClusterMode.yarn.name()) || mode.equals(ClusterMode.yarnPer.name())) { + } else if(mode.equals(ClusterMode.yarn.name())) { return createYarnClient(launcherOptions,mode); } throw new IllegalArgumentException("Unsupported cluster client type: "); diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java index 5c0032ce6..7944142a2 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java @@ -20,6 +20,7 @@ import com.dtstack.flink.sql.options.LauncherOptions; import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -27,7 +28,9 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - +import java.net.MalformedURLException; +import java.net.URL; +import java.util.Map; import java.util.Properties; /** @@ -39,28 +42,33 @@ public class PerJobSubmitter { + private final static String CLASS_FILE_NAME_PRESTR = "class_path"; + private static final Logger LOG = LoggerFactory.getLogger(PerJobSubmitter.class); public static String submit(LauncherOptions launcherOptions, JobGraph jobGraph) throws Exception { - Properties confProperties = PluginUtil.jsonStrToObject(launcherOptions.getConfProp(), Properties.class); ClusterSpecification clusterSpecification = FLinkPerJobResourceUtil.createClusterSpecification(confProperties); - PerJobClusterClientBuilder perJobClusterClientBuilder = new PerJobClusterClientBuilder(); perJobClusterClientBuilder.init(launcherOptions.getYarnconf()); - String flinkJarPath = launcherOptions.getFlinkJarPath(); - AbstractYarnClusterDescriptor yarnClusterDescriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(confProperties, flinkJarPath, launcherOptions.getQueue()); - ClusterClient clusterClient = yarnClusterDescriptor.deployJobCluster(clusterSpecification, jobGraph,true); - + ClusterClient clusterClient = yarnClusterDescriptor.deployJobCluster(clusterSpecification,fillJobGraphClassPath(jobGraph),true); String applicationId = clusterClient.getClusterId().toString(); String flinkJobId = jobGraph.getJobID().toString(); - String tips = String.format("deploy per_job with appId: %s, jobId: %s", applicationId, flinkJobId); System.out.println(tips); LOG.info(tips); - return applicationId; } + + private static JobGraph fillJobGraphClassPath(JobGraph jobGraph) throws MalformedURLException { + Map jobCacheFileConfig = jobGraph.getUserArtifacts(); + for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ + if(tmp.getKey().startsWith(CLASS_FILE_NAME_PRESTR)){ + jobGraph.getClasspaths().add(new URL("file:" + tmp.getValue().filePath)); + } + } + return jobGraph; + } } From 5d0b2cac8cde2ec0752e9615df9901483795c366 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 2 Jan 2019 12:04:18 +0800 Subject: [PATCH 239/470] upgrade v1.7.0 --- core/pom.xml | 15 +- .../flink/sql/parser/InsertSqlParser.java | 8 +- .../dtstack/flink/sql/side/SideSqlExec.java | 7 +- .../calcite/RelTimeIndicatorConverter.scala | 482 ----- .../flink/table/codegen/CodeGenerator.scala | 1920 ----------------- .../table/codegen/MatchCodeGenerator.scala | 574 ----- .../flink/table/codegen/generated.scala | 100 - .../nodes/datastream/DataStreamMatch.scala | 309 --- .../nodes/logical/FlinkLogicalMatch.scala | 115 - .../table/plan/rules/FlinkRuleSets.scala | 209 -- .../datastream/DataStreamMatchRule.scala | 46 - .../table/runtime/cepmatch/ConvertToRow.scala | 14 - .../cepmatch/IterativeConditionRunner.scala | 40 - .../table/runtime/cepmatch/MatchUtil.scala | 99 - .../PatternFlatSelectFunctionRunner.scala | 47 - .../PatternSelectFunctionRunner.scala | 45 - .../table/validate/FunctionCatalog.scala | 534 ----- .../flink/sql/side/SideSqlExecTest.java | 45 - .../sink/kafka/CustomerCsvSerialization.java | 2 +- .../flink/sql/sink/kafka/KafkaSink.java | 9 +- .../sink/kafka/table/KafkaSinkTableInfo.java | 2 +- .../sink/kafka/CustomerCsvSerialization.java | 4 +- .../flink/sql/sink/kafka/KafkaSink.java | 9 +- .../sink/kafka/table/KafkaSinkTableInfo.java | 2 +- kafka09/pom.xml | 2 +- .../sink/kafka/CustomerCsvSerialization.java | 4 +- .../flink/sql/sink/kafka/KafkaSink.java | 9 +- .../sink/kafka/table/KafkaSinkTableInfo.java | 3 +- kafka10/kafka10-source/pom.xml | 1 + .../sink/kafka/CustomerCsvSerialization.java | 4 +- .../flink/sql/sink/kafka/KafkaSink.java | 9 +- .../sink/kafka/table/KafkaSinkTableInfo.java | 2 +- kafka11/kafka11-source/pom.xml | 1 + .../flink/sql/source/kafka/KafkaSource.java | 1 - kafka11/pom.xml | 1 + .../flink/sql/launcher/LauncherMain.java | 4 +- pom.xml | 2 +- .../flink/sql/side/rdb/util/MathUtil.java | 2 +- 38 files changed, 49 insertions(+), 4633 deletions(-) delete mode 100644 core/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/codegen/generated.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/ConvertToRow.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/IterativeConditionRunner.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/MatchUtil.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternFlatSelectFunctionRunner.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternSelectFunctionRunner.scala delete mode 100644 core/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala diff --git a/core/pom.xml b/core/pom.xml index a38d818ae..300bc7bd7 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -17,6 +17,7 @@ UTF-8 core + 1.16.0 @@ -45,6 +46,12 @@ ${flink.version} + + org.apache.flink + flink-streaming-scala_2.11 + ${flink.version} + + org.apache.flink flink-table_2.11 @@ -55,7 +62,7 @@ org.apache.calcite calcite-server - 1.16.0 + ${calcite.server.version} @@ -70,12 +77,6 @@ ${flink.version} - - org.apache.flink - flink-streaming-scala_2.11 - ${flink.version} - - org.apache.flink flink-shaded-hadoop2 diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index 40629b139..10662a901 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -112,10 +112,10 @@ private static void parseNode(SqlNode sqlNode, SqlParseResult sqlParseResult){ sqlParseResult.addSourceTable(identifierNode.toString()); } break; - case MATCH_RECOGNIZE: - SqlMatchRecognize node = (SqlMatchRecognize) sqlNode; - sqlParseResult.addSourceTable(node.getTableRef().toString()); - break; +// case MATCH_RECOGNIZE: +// SqlMatchRecognize node = (SqlMatchRecognize) sqlNode; +// sqlParseResult.addSourceTable(node.getTableRef().toString()); +// break; case UNION: SqlNode unionLeft = ((SqlBasicCall)sqlNode).getOperands()[0]; SqlNode unionRight = ((SqlBasicCall)sqlNode).getOperands()[1]; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 5e5534438..e25a66ae6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -101,7 +101,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl tableEnv.sqlUpdate(pollSqlNode.toString()); }else if(pollSqlNode.getKind() == AS){ AliasInfo aliasInfo = parseASNode(pollSqlNode); - Table table = tableEnv.sql(aliasInfo.getName()); + Table table = tableEnv.sqlQuery(aliasInfo.getName()); tableEnv.registerTable(aliasInfo.getAlias(), table); localTableCache.put(aliasInfo.getAlias(), table); } @@ -528,7 +528,7 @@ public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, tableEnv.sqlUpdate(pollSqlNode.toString()); }else if(pollSqlNode.getKind() == AS){ AliasInfo aliasInfo = parseASNode(pollSqlNode); - Table table = tableEnv.sql(aliasInfo.getName()); + Table table = tableEnv.sqlQuery(aliasInfo.getName()); tableEnv.registerTable(aliasInfo.getAlias(), table); localTableCache.put(aliasInfo.getAlias(), table); } else if (pollSqlNode.getKind() == SELECT){ @@ -649,7 +649,8 @@ private boolean checkFieldsInfo(CreateTmpTableParser.SqlParserResult result, Tab fieldNames.add(fieldName); String fieldType = filed[filed.length - 1 ].trim(); Class fieldClass = ClassUtil.stringConvertClass(fieldType); - Class tableField = table.getSchema().getType(i).get().getTypeClass(); + //table.getSchema().getType(i).get().getTypeClass(); + Class tableField = table.getSchema().getFieldType(i).get().getTypeClass(); if (fieldClass == tableField){ continue; } else { diff --git a/core/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala b/core/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala deleted file mode 100644 index e120addb3..000000000 --- a/core/src/main/scala/org/apache/flink/table/calcite/RelTimeIndicatorConverter.scala +++ /dev/null @@ -1,482 +0,0 @@ -package org.apache.flink.table.calcite - -import org.apache.calcite.rel.`type`.RelDataType -import org.apache.calcite.rel.core._ -import org.apache.calcite.rel.logical._ -import org.apache.calcite.rel.{RelNode, RelShuttle} -import org.apache.calcite.rex._ -import org.apache.calcite.sql.fun.SqlStdOperatorTable -import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo -import org.apache.flink.table.api.{TableException, ValidationException} -import org.apache.flink.table.calcite.FlinkTypeFactory.{isRowtimeIndicatorType, _} -import org.apache.flink.table.functions.sql.ProctimeSqlFunction -import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate -import org.apache.flink.table.plan.schema.TimeIndicatorRelDataType -import org.apache.flink.table.validate.BasicOperatorTable - -import scala.collection.JavaConversions._ -import scala.collection.JavaConverters._ -import scala.collection.mutable - -/** - * Traverses a [[RelNode]] tree and converts fields with [[TimeIndicatorRelDataType]] type. If a - * time attribute is accessed for a calculation, it will be materialized. Forwarding is allowed in - * some cases, but not all. - */ -class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { - - private val timestamp = rexBuilder - .getTypeFactory - .asInstanceOf[FlinkTypeFactory] - .createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP, isNullable = false) - - override def visit(intersect: LogicalIntersect): RelNode = - throw new TableException("Logical intersect in a stream environment is not supported yet.") - - override def visit(union: LogicalUnion): RelNode = { - // visit children and update inputs - val inputs = union.getInputs.map(_.accept(this)) - - // make sure that time indicator types match - val inputTypes = inputs.map(_.getRowType) - - val head = inputTypes.head.getFieldList.map(_.getType) - - val isValid = inputTypes.forall { t => - val fieldTypes = t.getFieldList.map(_.getType) - - fieldTypes.zip(head).forall { case (l, r) => - // check if time indicators match - if (isTimeIndicatorType(l) && isTimeIndicatorType(r)) { - val leftTime = l.asInstanceOf[TimeIndicatorRelDataType].isEventTime - val rightTime = r.asInstanceOf[TimeIndicatorRelDataType].isEventTime - leftTime == rightTime - } - // one side is not an indicator - else if (isTimeIndicatorType(l) || isTimeIndicatorType(r)) { - false - } - // uninteresting types - else { - true - } - } - } - - if (!isValid) { - throw new ValidationException( - "Union fields with time attributes have different types.") - } - - LogicalUnion.create(inputs, union.all) - } - - override def visit(aggregate: LogicalAggregate): RelNode = convertAggregate(aggregate) - - override def visit(minus: LogicalMinus): RelNode = - throw new TableException("Logical minus in a stream environment is not supported yet.") - - override def visit(sort: LogicalSort): RelNode = { - - val input = sort.getInput.accept(this) - LogicalSort.create(input, sort.collation, sort.offset, sort.fetch) - } - - // override def visit(`match`: LogicalMatch): RelNode = - // throw new TableException("Logical match in a stream environment is not supported yet.") - // and cpe entry - override def visit(`match`: LogicalMatch): RelNode = { - val rowType = `match`.getInput.getRowType - - val materializer = new RexTimeIndicatorMaterializer( - rexBuilder, - rowType.getFieldList.map(_.getType)) - - val patternDefinitions = - `match`.getPatternDefinitions.foldLeft(mutable.Map[String, RexNode]()) { - case (m, (k, v)) => m += k -> v.accept(materializer) - } - - val measures = `match`.getMeasures.foldLeft(mutable.Map[String, RexNode]()) { - case (m, (k, v)) => m += k -> v.accept(materializer) - } - - val outputTypeBuilder = rexBuilder - .getTypeFactory - .asInstanceOf[FlinkTypeFactory] - .builder() - - `match`.getRowType.getFieldList.asScala - .foreach(x => measures.get(x.getName) match { - case Some(measure) => outputTypeBuilder.add(x.getName, measure.getType) - case None => outputTypeBuilder.add(x) - }) - - LogicalMatch.create( - `match`.getInput, - outputTypeBuilder.build(), - `match`.getPattern, - `match`.isStrictStart, - `match`.isStrictEnd, - patternDefinitions, - measures, - `match`.getAfter, - `match`.getSubsets.asInstanceOf[java.util.Map[String, java.util.TreeSet[String]]], - `match`.isAllRows, - `match`.getPartitionKeys, - `match`.getOrderKeys, - `match`.getInterval) - } - - - override def visit(other: RelNode): RelNode = other match { - - case uncollect: Uncollect => - // visit children and update inputs - val input = uncollect.getInput.accept(this) - Uncollect.create(uncollect.getTraitSet, input, uncollect.withOrdinality) - - case scan: LogicalTableFunctionScan => - scan - - case aggregate: LogicalWindowAggregate => - val convAggregate = convertAggregate(aggregate) - - LogicalWindowAggregate.create( - aggregate.getWindow, - aggregate.getNamedProperties, - convAggregate) - - case _ => - throw new TableException(s"Unsupported logical operator: ${other.getClass.getSimpleName}") - } - - - override def visit(exchange: LogicalExchange): RelNode = - throw new TableException("Logical exchange in a stream environment is not supported yet.") - - override def visit(scan: TableScan): RelNode = scan - - override def visit(scan: TableFunctionScan): RelNode = - throw new TableException("Table function scan in a stream environment is not supported yet.") - - override def visit(values: LogicalValues): RelNode = values - - override def visit(filter: LogicalFilter): RelNode = { - // visit children and update inputs - val input = filter.getInput.accept(this) - - // We do not materialize time indicators in conditions because they can be locally evaluated. - // Some conditions are evaluated by special operators (e.g., time window joins). - // Time indicators in remaining conditions are materialized by Calc before the code generation. - LogicalFilter.create(input, filter.getCondition) - } - - override def visit(project: LogicalProject): RelNode = { - // visit children and update inputs - val input = project.getInput.accept(this) - - // check if input field contains time indicator type - // materialize field if no time indicator is present anymore - // if input field is already materialized, change to timestamp type - val materializer = new RexTimeIndicatorMaterializer( - rexBuilder, - input.getRowType.getFieldList.map(_.getType)) - - val projects = project.getProjects.map(_.accept(materializer)) - val fieldNames = project.getRowType.getFieldNames - LogicalProject.create(input, projects, fieldNames) - } - - override def visit(join: LogicalJoin): RelNode = { - val left = join.getLeft.accept(this) - val right = join.getRight.accept(this) - - LogicalJoin.create(left, right, join.getCondition, join.getVariablesSet, join.getJoinType) - - } - - - override def visit(correlate: LogicalCorrelate): RelNode = { - // visit children and update inputs - val inputs = correlate.getInputs.map(_.accept(this)) - - val right = inputs(1) match { - case scan: LogicalTableFunctionScan => - // visit children and update inputs - val scanInputs = scan.getInputs.map(_.accept(this)) - - // check if input field contains time indicator type - // materialize field if no time indicator is present anymore - // if input field is already materialized, change to timestamp type - val materializer = new RexTimeIndicatorMaterializer( - rexBuilder, - inputs.head.getRowType.getFieldList.map(_.getType)) - - val call = scan.getCall.accept(materializer) - LogicalTableFunctionScan.create( - scan.getCluster, - scanInputs, - call, - scan.getElementType, - scan.getRowType, - scan.getColumnMappings) - - case _ => - inputs(1) - } - - LogicalCorrelate.create( - inputs.head, - right, - correlate.getCorrelationId, - correlate.getRequiredColumns, - correlate.getJoinType) - } - - - - - private def convertAggregate(aggregate: Aggregate): LogicalAggregate = { - // visit children and update inputs - val input = aggregate.getInput.accept(this) - - // add a project to materialize aggregation arguments/grouping keys - - val refIndices = mutable.Set[Int]() - - // check arguments of agg calls - aggregate.getAggCallList.foreach(call => if (call.getArgList.size() == 0) { - // count(*) has an empty argument list - (0 until input.getRowType.getFieldCount).foreach(refIndices.add) - } else { - // for other aggregations - call.getArgList.map(_.asInstanceOf[Int]).foreach(refIndices.add) - }) - - // check grouping sets - aggregate.getGroupSets.foreach(set => - set.asList().map(_.asInstanceOf[Int]).foreach(refIndices.add) - ) - - val needsMaterialization = refIndices.exists(idx => - isTimeIndicatorType(input.getRowType.getFieldList.get(idx).getType)) - - // create project if necessary - val projectedInput = if (needsMaterialization) { - - // insert or merge with input project if - // a time attribute is accessed and needs to be materialized - input match { - - // merge - case lp: LogicalProject => - val projects = lp.getProjects.zipWithIndex.map { case (expr, idx) => - if (isTimeIndicatorType(expr.getType) && refIndices.contains(idx)) { - if (isRowtimeIndicatorType(expr.getType)) { - // cast rowtime indicator to regular timestamp - rexBuilder.makeAbstractCast(timestamp, expr) - } else { - // generate proctime access - rexBuilder.makeCall(ProctimeSqlFunction, expr) - } - } else { - expr - } - } - - LogicalProject.create( - lp.getInput, - projects, - input.getRowType.getFieldNames) - - // new project - case _ => - val projects = input.getRowType.getFieldList.map { field => - if (isTimeIndicatorType(field.getType) && refIndices.contains(field.getIndex)) { - if (isRowtimeIndicatorType(field.getType)) { - // cast rowtime indicator to regular timestamp - rexBuilder.makeAbstractCast( - timestamp, - new RexInputRef(field.getIndex, field.getType)) - } else { - // generate proctime access - rexBuilder.makeCall( - ProctimeSqlFunction, - new RexInputRef(field.getIndex, field.getType)) - } - } else { - new RexInputRef(field.getIndex, field.getType) - } - } - - LogicalProject.create( - input, - projects, - input.getRowType.getFieldNames) - } - } else { - // no project necessary - input - } - - // remove time indicator type as agg call return type - val updatedAggCalls = aggregate.getAggCallList.map { call => - val callType = if (isTimeIndicatorType(call.getType)) { - timestamp - } else { - call.getType - } - AggregateCall.create( - call.getAggregation, - call.isDistinct, - call.getArgList, - call.filterArg, - callType, - call.name) - } - - LogicalAggregate.create( - projectedInput, - aggregate.indicator, - aggregate.getGroupSet, - aggregate.getGroupSets, - updatedAggCalls) - } - -} - -object RelTimeIndicatorConverter { - - def convert(rootRel: RelNode, rexBuilder: RexBuilder): RelNode = { - val converter = new RelTimeIndicatorConverter(rexBuilder) - val convertedRoot = rootRel.accept(converter) - - var needsConversion = false - - // materialize remaining proctime indicators - val projects = convertedRoot.getRowType.getFieldList.map(field => - if (isProctimeIndicatorType(field.getType)) { - needsConversion = true - rexBuilder.makeCall( - ProctimeSqlFunction, - new RexInputRef(field.getIndex, field.getType)) - } else { - new RexInputRef(field.getIndex, field.getType) - } - ) - - // add final conversion if necessary - if (needsConversion) { - LogicalProject.create( - convertedRoot, - projects, - convertedRoot.getRowType.getFieldNames) - } else { - convertedRoot - } - } - - /** - * Materializes time indicator accesses in an expression. - * - * @param expr The expression in which time indicators are materialized. - * @param rowType The input schema of the expression. - * @param rexBuilder A RexBuilder. - * - * @return The expression with materialized time indicators. - */ - def convertExpression(expr: RexNode, rowType: RelDataType, rexBuilder: RexBuilder): RexNode = { - val materializer = new RexTimeIndicatorMaterializer( - rexBuilder, - rowType.getFieldList.map(_.getType)) - - expr.accept(materializer) - } -} - -class RexTimeIndicatorMaterializer( - private val rexBuilder: RexBuilder, - private val input: Seq[RelDataType]) - extends RexShuttle { - - private val timestamp = rexBuilder - .getTypeFactory - .asInstanceOf[FlinkTypeFactory] - .createTypeFromTypeInfo(SqlTimeTypeInfo.TIMESTAMP, isNullable = false) - - override def visitInputRef(inputRef: RexInputRef): RexNode = { - // reference is interesting - if (isTimeIndicatorType(inputRef.getType)) { - val resolvedRefType = input(inputRef.getIndex) - // input is a valid time indicator - if (isTimeIndicatorType(resolvedRefType)) { - inputRef - } - // input has been materialized - else { - new RexInputRef(inputRef.getIndex, resolvedRefType) - } - } - // reference is a regular field - else { - super.visitInputRef(inputRef) - } - } - - override def visitCall(call: RexCall): RexNode = { - val updatedCall = super.visitCall(call).asInstanceOf[RexCall] - - // materialize operands with time indicators - val materializedOperands = updatedCall.getOperator match { - - // skip materialization for special operators - case BasicOperatorTable.SESSION | BasicOperatorTable.HOP | BasicOperatorTable.TUMBLE => - updatedCall.getOperands.toList - - case _ => - updatedCall.getOperands.map { o => - if (isTimeIndicatorType(o.getType)) { - if (isRowtimeIndicatorType(o.getType)) { - // cast rowtime indicator to regular timestamp - rexBuilder.makeAbstractCast(timestamp, o) - } else { - // generate proctime access - rexBuilder.makeCall(ProctimeSqlFunction, o) - } - } else { - o - } - } - } - - // remove time indicator return type - updatedCall.getOperator match { - - // we do not modify AS if operand has not been materialized - case SqlStdOperatorTable.AS if - isTimeIndicatorType(updatedCall.getOperands.get(0).getType) => - updatedCall - - // do not modify window time attributes - case BasicOperatorTable.TUMBLE_ROWTIME | - BasicOperatorTable.TUMBLE_PROCTIME | - BasicOperatorTable.HOP_ROWTIME | - BasicOperatorTable.HOP_PROCTIME | - BasicOperatorTable.SESSION_ROWTIME | - BasicOperatorTable.SESSION_PROCTIME - // since we materialize groupings on time indicators, - // we cannot check the operands anymore but the return type at least - if isTimeIndicatorType(updatedCall.getType) => - updatedCall - - // materialize function's result and operands - case _ if isTimeIndicatorType(updatedCall.getType) => - updatedCall.clone(timestamp, materializedOperands) - - // materialize function's operands only - case _ => - updatedCall.clone(updatedCall.getType, materializedOperands) - } - } -} diff --git a/core/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala b/core/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala deleted file mode 100644 index 439e3a7f2..000000000 --- a/core/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala +++ /dev/null @@ -1,1920 +0,0 @@ -package org.apache.flink.table.codegen - -import java.math.{BigDecimal => JBigDecimal} - -import org.apache.calcite.avatica.util.DateTimeUtils -import org.apache.calcite.rex._ -import org.apache.calcite.sql.SqlOperator -import org.apache.calcite.sql.`type`.SqlTypeName._ -import org.apache.calcite.sql.`type`.{ReturnTypes, SqlTypeName} -import org.apache.calcite.sql.fun.SqlStdOperatorTable.{ROW, _} -import org.apache.commons.lang3.StringEscapeUtils -import org.apache.flink.api.common.functions._ -import org.apache.flink.api.common.typeinfo._ -import org.apache.flink.api.common.typeutils.CompositeType -import org.apache.flink.api.java.typeutils._ -import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo -import org.apache.flink.streaming.api.functions.ProcessFunction -import org.apache.flink.table.api.{TableConfig, TableException} -import org.apache.flink.table.calcite.FlinkTypeFactory -import org.apache.flink.table.codegen.CodeGenUtils._ -import org.apache.flink.table.codegen.GeneratedExpression.{ALWAYS_NULL, NEVER_NULL, NO_CODE} -import org.apache.flink.table.codegen.calls.ScalarOperators._ -import org.apache.flink.table.codegen.calls.{CurrentTimePointCallGen, FunctionGenerator} -import org.apache.flink.table.functions.sql.{ProctimeSqlFunction, ScalarSqlFunctions, StreamRecordTimestampSqlFunction} -import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils -import org.apache.flink.table.functions.{FunctionContext, UserDefinedFunction} -import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo -import org.apache.flink.table.typeutils.TypeCheckUtils._ -import org.joda.time.format.DateTimeFormatter - -import scala.collection.JavaConversions._ -import scala.collection.mutable - -/** - * [[CodeGenerator]] is the base code generator for generating Flink - * [[org.apache.flink.api.common.functions.Function]]s. - * It is responsible for expression generation and tracks the context (member variables etc). - * - * @param config configuration that determines runtime behavior - * @param nullableInput input(s) can be null. - * @param input1 type information about the first input of the Function - * @param input2 type information about the second input if the Function is binary - * @param input1FieldMapping additional mapping information for input1. - * POJO types have no deterministic field order and some input fields might not be read. - * The input1FieldMapping is also used to inject time indicator attributes. - * @param input2FieldMapping additional mapping information for input2. - * POJO types have no deterministic field order and some input fields might not be read. - */ -abstract class CodeGenerator( - config: TableConfig, - nullableInput: Boolean, - input1: TypeInformation[_ <: Any], - input2: Option[TypeInformation[_ <: Any]] = None, - input1FieldMapping: Option[Array[Int]] = None, - input2FieldMapping: Option[Array[Int]] = None) - extends RexVisitor[GeneratedExpression] { - - // check if nullCheck is enabled when inputs can be null - if (nullableInput && !config.getNullCheck) { - throw new CodeGenException("Null check must be enabled if entire rows can be null.") - } - - // check for POJO input1 mapping - input1 match { - case pt: PojoTypeInfo[_] => - input1FieldMapping.getOrElse( - throw new CodeGenException("No input mapping is specified for input1 of type POJO.")) - case _ => // ok - } - - // check for POJO input2 mapping - input2 match { - case Some(pt: PojoTypeInfo[_]) => - input2FieldMapping.getOrElse( - throw new CodeGenException("No input mapping is specified for input2 of type POJO.")) - case _ => // ok - } - - protected val input1Mapping: Array[Int] = input1FieldMapping match { - case Some(mapping) => mapping - case _ => (0 until input1.getArity).toArray - } - - protected val input2Mapping: Array[Int] = input2FieldMapping match { - case Some(mapping) => mapping - case _ => input2 match { - case Some(input) => (0 until input.getArity).toArray - case _ => Array[Int]() - } - } - - // set of member statements that will be added only once - // we use a LinkedHashSet to keep the insertion order - protected val reusableMemberStatements: mutable.LinkedHashSet[String] = - mutable.LinkedHashSet[String]() - - // set of constructor statements that will be added only once - // we use a LinkedHashSet to keep the insertion order - protected val reusableInitStatements: mutable.LinkedHashSet[String] = - mutable.LinkedHashSet[String]() - - // set of open statements for RichFunction that will be added only once - // we use a LinkedHashSet to keep the insertion order - protected val reusableOpenStatements: mutable.LinkedHashSet[String] = - mutable.LinkedHashSet[String]() - - // set of close statements for RichFunction that will be added only once - // we use a LinkedHashSet to keep the insertion order - protected val reusableCloseStatements: mutable.LinkedHashSet[String] = - mutable.LinkedHashSet[String]() - - // set of statements that will be added only once per record; - // code should only update member variables because local variables are not accessible if - // the code needs to be split; - // we use a LinkedHashSet to keep the insertion order - protected val reusablePerRecordStatements: mutable.LinkedHashSet[String] = - mutable.LinkedHashSet[String]() - - // map of initial input unboxing expressions that will be added only once - // (inputTerm, index) -> expr - protected val reusableInputUnboxingExprs: mutable.Map[(String, Int), GeneratedExpression] = - mutable.Map[(String, Int), GeneratedExpression]() - - // set of constructor statements that will be added only once - // we use a LinkedHashSet to keep the insertion order - protected val reusableConstructorStatements: mutable.LinkedHashSet[(String, String)] = - mutable.LinkedHashSet[(String, String)]() - - /** - * Flag that indicates that the generated code needed to be split into several methods. - */ - protected var hasCodeSplits: Boolean = false - - /** - * @return code block of statements that need to be placed in the member area of the Function - * (e.g. member variables and their initialization) - */ - def reuseMemberCode(): String = { - reusableMemberStatements.mkString("", "\n", "\n") - } - - /** - * @return code block of statements that need to be placed in the constructor of the Function - */ - def reuseInitCode(): String = { - reusableInitStatements.mkString("", "\n", "\n") - } - - /** - * @return code block of statements that need to be placed in the open() method of RichFunction - */ - def reuseOpenCode(): String = { - reusableOpenStatements.mkString("", "\n", "\n") - } - - /** - * @return code block of statements that need to be placed in the close() method of RichFunction - */ - def reuseCloseCode(): String = { - reusableCloseStatements.mkString("", "\n", "\n") - } - - /** - * @return code block of statements that need to be placed in the SAM of the Function - */ - def reusePerRecordCode(): String = { - reusablePerRecordStatements.mkString("", "\n", "\n") - } - - /** - * @return code block of statements that unbox input variables to a primitive variable - * and a corresponding null flag variable - */ - def reuseInputUnboxingCode(): String = { - reusableInputUnboxingExprs.values.map(_.code).mkString("", "\n", "\n") - } - - /** - * @return code block of constructor statements for the Function - */ - def reuseConstructorCode(className: String): String = { - reusableConstructorStatements.map { case (params, body) => - s""" - |public $className($params) throws Exception { - | this(); - | $body - |} - |""".stripMargin - }.mkString("", "\n", "\n") - } - - /** - * @return term of the (casted and possibly boxed) first input - */ - var input1Term = "in1" - - /** - * @return term of the (casted and possibly boxed) second input - */ - var input2Term = "in2" - - /** - * @return term of the (casted) output collector - */ - var collectorTerm = "c" - - /** - * @return term of the output record (possibly defined in the member area e.g. Row, Tuple) - */ - var outRecordTerm = "out" - - /** - * @return term of the [[ProcessFunction]]'s context - */ - var contextTerm = "ctx" - - /** - * @return returns if null checking is enabled - */ - def nullCheck: Boolean = config.getNullCheck - - /** - * Generates an expression from a RexNode. If objects or variables can be reused, they will be - * added to reusable code sections internally. - * - * @param rex Calcite row expression - * @return instance of GeneratedExpression - */ - def generateExpression(rex: RexNode): GeneratedExpression = { - rex.accept(this) - } - - /** - * Generates an expression that converts the first input (and second input) into the given type. - * If two inputs are converted, the second input is appended. If objects or variables can - * be reused, they will be added to reusable code sections internally. The evaluation result - * may be stored in the global result variable (see [[outRecordTerm]]). - * - * @param returnType conversion target type. Inputs and output must have the same arity. - * @param resultFieldNames result field names necessary for a mapping to POJO fields. - * @param rowtimeExpression an expression to extract the value of a rowtime field from - * the input data. Required if the field indices include a rowtime - * marker. - * @return instance of GeneratedExpression - */ - def generateConverterResultExpression( - returnType: TypeInformation[_ <: Any], - resultFieldNames: Seq[String], - rowtimeExpression: Option[RexNode] = None) - : GeneratedExpression = { - - val input1AccessExprs = input1Mapping.map { - case TimeIndicatorTypeInfo.ROWTIME_STREAM_MARKER | - TimeIndicatorTypeInfo.ROWTIME_BATCH_MARKER if rowtimeExpression.isDefined => - // generate rowtime attribute from expression - generateExpression(rowtimeExpression.get) - case TimeIndicatorTypeInfo.ROWTIME_STREAM_MARKER | - TimeIndicatorTypeInfo.ROWTIME_BATCH_MARKER => - throw TableException("Rowtime extraction expression missing. Please report a bug.") - case TimeIndicatorTypeInfo.PROCTIME_STREAM_MARKER => - // attribute is proctime indicator. - // we use a null literal and generate a timestamp when we need it. - generateNullLiteral(TimeIndicatorTypeInfo.PROCTIME_INDICATOR) - case TimeIndicatorTypeInfo.PROCTIME_BATCH_MARKER => - // attribute is proctime field in a batch query. - // it is initialized with the current time. - generateCurrentTimestamp() - case idx => - generateInputAccess(input1, input1Term, idx) - } - - val input2AccessExprs = input2 match { - case Some(ti) => - input2Mapping.map(idx => generateInputAccess(ti, input2Term, idx)).toSeq - case None => Seq() // add nothing - } - - generateResultExpression(input1AccessExprs ++ input2AccessExprs, returnType, resultFieldNames) - } - - /** - * Generates an expression from the left input and the right table function. - */ - def generateCorrelateAccessExprs: (Seq[GeneratedExpression], Seq[GeneratedExpression]) = { - val input1AccessExprs = input1Mapping.map { idx => - generateInputAccess(input1, input1Term, idx) - } - - val input2AccessExprs = input2 match { - case Some(ti) => - // use generateFieldAccess instead of generateInputAccess to avoid the generated table - // function's field access code is put on the top of function body rather than - // the while loop - input2Mapping.map { idx => - generateFieldAccess(ti, input2Term, idx) - }.toSeq - case None => throw new CodeGenException("Type information of input2 must not be null.") - } - (input1AccessExprs, input2AccessExprs) - } - - /** - * Generates an expression from a sequence of RexNode. If objects or variables can be reused, - * they will be added to reusable code sections internally. The evaluation result - * may be stored in the global result variable (see [[outRecordTerm]]). - * - * @param returnType conversion target type. Type must have the same arity than rexNodes. - * @param resultFieldNames result field names necessary for a mapping to POJO fields. - * @param rexNodes sequence of RexNode to be converted - * @return instance of GeneratedExpression - */ - def generateResultExpression( - returnType: TypeInformation[_ <: Any], - resultFieldNames: Seq[String], - rexNodes: Seq[RexNode]) - : GeneratedExpression = { - val fieldExprs = rexNodes.map(generateExpression) - generateResultExpression(fieldExprs, returnType, resultFieldNames) - } - - /** - * Generates an expression from a sequence of other expressions. If objects or variables can - * be reused, they will be added to reusable code sections internally. The evaluation result - * may be stored in the global result variable (see [[outRecordTerm]]). - * - * @param fieldExprs field expressions to be converted - * @param returnType conversion target type. Type must have the same arity than fieldExprs. - * @param resultFieldNames result field names necessary for a mapping to POJO fields. - * @return instance of GeneratedExpression - */ - def generateResultExpression( - fieldExprs: Seq[GeneratedExpression], - returnType: TypeInformation[_ <: Any], - resultFieldNames: Seq[String]) - : GeneratedExpression = { - // initial type check - if (returnType.getArity != fieldExprs.length) { - throw new CodeGenException( - s"Arity [${returnType.getArity}] of result type [$returnType] does not match " + - s"number [${fieldExprs.length}] of expressions [$fieldExprs].") - } - if (resultFieldNames.length != fieldExprs.length) { - throw new CodeGenException( - s"Arity [${resultFieldNames.length}] of result field names [$resultFieldNames] does not " + - s"match number [${fieldExprs.length}] of expressions [$fieldExprs].") - } - // type check - returnType match { - case pt: PojoTypeInfo[_] => - fieldExprs.zipWithIndex foreach { - case (fieldExpr, i) if fieldExpr.resultType != pt.getTypeAt(resultFieldNames(i)) => - throw new CodeGenException( - s"Incompatible types of expression and result type. Expression [$fieldExpr] type is" + - s" [${fieldExpr.resultType}], result type is [${pt.getTypeAt(resultFieldNames(i))}]") - - case _ => // ok - } - - case ct: CompositeType[_] => - fieldExprs.zipWithIndex foreach { - case (fieldExpr, i) if fieldExpr.resultType != ct.getTypeAt(i) => - throw new CodeGenException( - s"Incompatible types of expression and result type. Expression[$fieldExpr] type is " + - s"[${fieldExpr.resultType}], result type is [${ct.getTypeAt(i)}]") - case _ => // ok - } - - case t: TypeInformation[_] if t != fieldExprs.head.resultType => - throw new CodeGenException( - s"Incompatible types of expression and result type. Expression [${fieldExprs.head}] " + - s"type is [${fieldExprs.head.resultType}], result type is [$t]") - - case _ => // ok - } - - val returnTypeTerm = boxedTypeTermForTypeInfo(returnType) - val boxedFieldExprs = fieldExprs.map(generateOutputFieldBoxing) - - // generate result expression - returnType match { - case ri: RowTypeInfo => - addReusableOutRecord(ri) - val resultSetters = boxedFieldExprs.zipWithIndex map { - case (fieldExpr, i) => - if (nullCheck) { - s""" - |${fieldExpr.code} - |if (${fieldExpr.nullTerm}) { - | $outRecordTerm.setField($i, null); - |} - |else { - | $outRecordTerm.setField($i, ${fieldExpr.resultTerm}); - |} - |""".stripMargin - } - else { - s""" - |${fieldExpr.code} - |$outRecordTerm.setField($i, ${fieldExpr.resultTerm}); - |""".stripMargin - } - } - - val code = generateCodeSplits(resultSetters) - - GeneratedExpression(outRecordTerm, NEVER_NULL, code, returnType) - - case pt: PojoTypeInfo[_] => - addReusableOutRecord(pt) - val resultSetters = boxedFieldExprs.zip(resultFieldNames) map { - case (fieldExpr, fieldName) => - val accessor = getFieldAccessor(pt.getTypeClass, fieldName) - - accessor match { - // Reflective access of primitives/Objects - case ObjectPrivateFieldAccessor(field) => - val fieldTerm = addReusablePrivateFieldAccess(pt.getTypeClass, fieldName) - - val defaultIfNull = if (isFieldPrimitive(field)) { - primitiveDefaultValue(fieldExpr.resultType) - } else { - "null" - } - - if (nullCheck) { - s""" - |${fieldExpr.code} - |if (${fieldExpr.nullTerm}) { - | ${reflectiveFieldWriteAccess( - fieldTerm, - field, - outRecordTerm, - defaultIfNull)}; - |} - |else { - | ${reflectiveFieldWriteAccess( - fieldTerm, - field, - outRecordTerm, - fieldExpr.resultTerm)}; - |} - |""".stripMargin - } - else { - s""" - |${fieldExpr.code} - |${reflectiveFieldWriteAccess( - fieldTerm, - field, - outRecordTerm, - fieldExpr.resultTerm)}; - |""".stripMargin - } - - // primitive or Object field access (implicit boxing) - case _ => - if (nullCheck) { - s""" - |${fieldExpr.code} - |if (${fieldExpr.nullTerm}) { - | $outRecordTerm.$fieldName = null; - |} - |else { - | $outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; - |} - |""".stripMargin - } - else { - s""" - |${fieldExpr.code} - |$outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; - |""".stripMargin - } - } - } - - val code = generateCodeSplits(resultSetters) - - GeneratedExpression(outRecordTerm, NEVER_NULL, code, returnType) - - case tup: TupleTypeInfo[_] => - addReusableOutRecord(tup) - val resultSetters = boxedFieldExprs.zipWithIndex map { - case (fieldExpr, i) => - val fieldName = "f" + i - if (nullCheck) { - s""" - |${fieldExpr.code} - |if (${fieldExpr.nullTerm}) { - | throw new NullPointerException("Null result cannot be stored in a Tuple."); - |} - |else { - | $outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; - |} - |""".stripMargin - } - else { - s""" - |${fieldExpr.code} - |$outRecordTerm.$fieldName = ${fieldExpr.resultTerm}; - |""".stripMargin - } - } - - val code = generateCodeSplits(resultSetters) - - GeneratedExpression(outRecordTerm, NEVER_NULL, code, returnType) - - case _: CaseClassTypeInfo[_] => - val fieldCodes: String = boxedFieldExprs.map(_.code).mkString("\n") - val constructorParams: String = boxedFieldExprs.map(_.resultTerm).mkString(", ") - val resultTerm = newName(outRecordTerm) - - val nullCheckCode = if (nullCheck) { - boxedFieldExprs map { (fieldExpr) => - s""" - |if (${fieldExpr.nullTerm}) { - | throw new NullPointerException("Null result cannot be stored in a Case Class."); - |} - |""".stripMargin - } mkString "\n" - } else { - "" - } - - val resultCode = - s""" - |$fieldCodes - |$nullCheckCode - |$returnTypeTerm $resultTerm = new $returnTypeTerm($constructorParams); - |""".stripMargin - - // case classes are not splittable - GeneratedExpression(resultTerm, NEVER_NULL, resultCode, returnType) - - case _: TypeInformation[_] => - val fieldExpr = boxedFieldExprs.head - val nullCheckCode = if (nullCheck) { - s""" - |if (${fieldExpr.nullTerm}) { - | throw new NullPointerException("Null result cannot be used for atomic types."); - |} - |""".stripMargin - } else { - "" - } - val resultCode = - s""" - |${fieldExpr.code} - |$nullCheckCode - |""".stripMargin - - // other types are not splittable - GeneratedExpression(fieldExpr.resultTerm, fieldExpr.nullTerm, resultCode, returnType) - - case _ => - throw new CodeGenException(s"Unsupported result type: $returnType") - } - } - - // ---------------------------------------------------------------------------------------------- - // RexVisitor methods - // ---------------------------------------------------------------------------------------------- - - override def visitInputRef(inputRef: RexInputRef): GeneratedExpression = { - // if inputRef index is within size of input1 we work with input1, input2 otherwise - val input = if (inputRef.getIndex < input1.getArity) { - (input1, input1Term) - } else { - (input2.getOrElse(throw new CodeGenException("Invalid input access.")), input2Term) - } - - val index = if (input._2 == input1Term) { - inputRef.getIndex - } else { - inputRef.getIndex - input1.getArity - } - - generateInputAccess(input._1, input._2, index) - } - - override def visitTableInputRef(rexTableInputRef: RexTableInputRef): GeneratedExpression = - visitInputRef(rexTableInputRef) - - override def visitFieldAccess(rexFieldAccess: RexFieldAccess): GeneratedExpression = { - val refExpr = rexFieldAccess.getReferenceExpr.accept(this) - val index = rexFieldAccess.getField.getIndex - generateFieldAccess(refExpr, index) - } - - override def visitLiteral(literal: RexLiteral): GeneratedExpression = { - val resultType = FlinkTypeFactory.toTypeInfo(literal.getType) - val value = literal.getValue3 - // null value with type - if (value == null) { - return generateNullLiteral(resultType) - } - // non-null values - literal.getType.getSqlTypeName match { - - case BOOLEAN => - generateNonNullLiteral(resultType, literal.getValue3.toString) - - case TINYINT => - val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) - if (decimal.isValidByte) { - generateNonNullLiteral(resultType, decimal.byteValue().toString) - } - else { - throw new CodeGenException("Decimal can not be converted to byte.") - } - - case SMALLINT => - val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) - if (decimal.isValidShort) { - generateNonNullLiteral(resultType, decimal.shortValue().toString) - } - else { - throw new CodeGenException("Decimal can not be converted to short.") - } - - case INTEGER => - val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) - if (decimal.isValidInt) { - generateNonNullLiteral(resultType, decimal.intValue().toString) - } - else { - throw new CodeGenException("Decimal can not be converted to integer.") - } - - case BIGINT => - val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) - if (decimal.isValidLong) { - generateNonNullLiteral(resultType, decimal.longValue().toString + "L") - } - else { - throw new CodeGenException("Decimal can not be converted to long.") - } - - case FLOAT => - val floatValue = value.asInstanceOf[JBigDecimal].floatValue() - floatValue match { - case Float.NaN => generateNonNullLiteral(resultType, "java.lang.Float.NaN") - case Float.NegativeInfinity => - generateNonNullLiteral(resultType, "java.lang.Float.NEGATIVE_INFINITY") - case Float.PositiveInfinity => - generateNonNullLiteral(resultType, "java.lang.Float.POSITIVE_INFINITY") - case _ => generateNonNullLiteral(resultType, floatValue.toString + "f") - } - - case DOUBLE => - val doubleValue = value.asInstanceOf[JBigDecimal].doubleValue() - doubleValue match { - case Double.NaN => generateNonNullLiteral(resultType, "java.lang.Double.NaN") - case Double.NegativeInfinity => - generateNonNullLiteral(resultType, "java.lang.Double.NEGATIVE_INFINITY") - case Double.PositiveInfinity => - generateNonNullLiteral(resultType, "java.lang.Double.POSITIVE_INFINITY") - case _ => generateNonNullLiteral(resultType, doubleValue.toString + "d") - } - case DECIMAL => - val decimalField = addReusableDecimal(value.asInstanceOf[JBigDecimal]) - generateNonNullLiteral(resultType, decimalField) - - case VARCHAR | CHAR => - val escapedValue = StringEscapeUtils.escapeJava( - StringEscapeUtils.unescapeJava(value.toString) - ) - generateNonNullLiteral(resultType, "\"" + escapedValue + "\"") - - case SYMBOL => - generateSymbol(value.asInstanceOf[Enum[_]]) - - case DATE => - generateNonNullLiteral(resultType, value.toString) - - case TIME => - generateNonNullLiteral(resultType, value.toString) - - case TIMESTAMP => - generateNonNullLiteral(resultType, value.toString + "L") - - case typeName if YEAR_INTERVAL_TYPES.contains(typeName) => - val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) - if (decimal.isValidInt) { - generateNonNullLiteral(resultType, decimal.intValue().toString) - } else { - throw new CodeGenException( - s"Decimal '$decimal' can not be converted to interval of months.") - } - - case typeName if DAY_INTERVAL_TYPES.contains(typeName) => - val decimal = BigDecimal(value.asInstanceOf[JBigDecimal]) - if (decimal.isValidLong) { - generateNonNullLiteral(resultType, decimal.longValue().toString + "L") - } else { - throw new CodeGenException( - s"Decimal '$decimal' can not be converted to interval of milliseconds.") - } - - case t@_ => - throw new CodeGenException(s"Type not supported: $t") - } - } - - override def visitCorrelVariable(correlVariable: RexCorrelVariable): GeneratedExpression = { - GeneratedExpression(input1Term, NEVER_NULL, NO_CODE, input1) - } - - override def visitLocalRef(localRef: RexLocalRef): GeneratedExpression = - throw new CodeGenException("Local variables are not supported yet.") - - override def visitRangeRef(rangeRef: RexRangeRef): GeneratedExpression = - throw new CodeGenException("Range references are not supported yet.") - - override def visitDynamicParam(dynamicParam: RexDynamicParam): GeneratedExpression = - throw new CodeGenException("Dynamic parameter references are not supported yet.") - - override def visitCall(call: RexCall): GeneratedExpression = { - - // special case: time materialization - if (call.getOperator == ProctimeSqlFunction) { - return generateProctimeTimestamp() - } - - val resultType = FlinkTypeFactory.toTypeInfo(call.getType) - - // convert operands and help giving untyped NULL literals a type - val operands = call.getOperands.zipWithIndex.map { - - // this helps e.g. for AS(null) - // we might need to extend this logic in case some rules do not create typed NULLs - case (operandLiteral: RexLiteral, 0) if - operandLiteral.getType.getSqlTypeName == SqlTypeName.NULL && - call.getOperator.getReturnTypeInference == ReturnTypes.ARG0 => - generateNullLiteral(resultType) - - case (o@_, _) => - o.accept(this) - } - generateCall(call, operands, resultType) - } - def generateCall( - call: RexCall, - operands: Seq[GeneratedExpression], - resultType: TypeInformation[_]) - : GeneratedExpression = { - - call.getOperator match { - // arithmetic - case PLUS if isNumeric(resultType) => - val left = operands.head - val right = operands(1) - requireNumeric(left) - requireNumeric(right) - generateArithmeticOperator("+", nullCheck, resultType, left, right, config) - - case PLUS | DATETIME_PLUS if isTemporal(resultType) => - val left = operands.head - val right = operands(1) - requireTemporal(left) - requireTemporal(right) - generateTemporalPlusMinus(plus = true, nullCheck, left, right, config) - - case MINUS if isNumeric(resultType) => - val left = operands.head - val right = operands(1) - requireNumeric(left) - requireNumeric(right) - generateArithmeticOperator("-", nullCheck, resultType, left, right, config) - - case MINUS | MINUS_DATE if isTemporal(resultType) => - val left = operands.head - val right = operands(1) - requireTemporal(left) - requireTemporal(right) - generateTemporalPlusMinus(plus = false, nullCheck, left, right, config) - - case MULTIPLY if isNumeric(resultType) => - val left = operands.head - val right = operands(1) - requireNumeric(left) - requireNumeric(right) - generateArithmeticOperator("*", nullCheck, resultType, left, right, config) - - case MULTIPLY if isTimeInterval(resultType) => - val left = operands.head - val right = operands(1) - requireTimeInterval(left) - requireNumeric(right) - generateArithmeticOperator("*", nullCheck, resultType, left, right, config) - - case DIVIDE | DIVIDE_INTEGER if isNumeric(resultType) => - val left = operands.head - val right = operands(1) - requireNumeric(left) - requireNumeric(right) - generateArithmeticOperator("/", nullCheck, resultType, left, right, config) - - case MOD if isNumeric(resultType) => - val left = operands.head - val right = operands(1) - requireNumeric(left) - requireNumeric(right) - generateArithmeticOperator("%", nullCheck, resultType, left, right, config) - - case UNARY_MINUS if isNumeric(resultType) => - val operand = operands.head - requireNumeric(operand) - generateUnaryArithmeticOperator("-", nullCheck, resultType, operand) - - case UNARY_MINUS if isTimeInterval(resultType) => - val operand = operands.head - requireTimeInterval(operand) - generateUnaryIntervalPlusMinus(plus = false, nullCheck, operand) - - case UNARY_PLUS if isNumeric(resultType) => - val operand = operands.head - requireNumeric(operand) - generateUnaryArithmeticOperator("+", nullCheck, resultType, operand) - - case UNARY_PLUS if isTimeInterval(resultType) => - val operand = operands.head - requireTimeInterval(operand) - generateUnaryIntervalPlusMinus(plus = true, nullCheck, operand) - - // comparison - case EQUALS => - val left = operands.head - val right = operands(1) - generateEquals(nullCheck, left, right) - - case NOT_EQUALS => - val left = operands.head - val right = operands(1) - generateNotEquals(nullCheck, left, right) - - case GREATER_THAN => - val left = operands.head - val right = operands(1) - requireComparable(left) - requireComparable(right) - generateComparison(">", nullCheck, left, right) - - case GREATER_THAN_OR_EQUAL => - val left = operands.head - val right = operands(1) - requireComparable(left) - requireComparable(right) - generateComparison(">=", nullCheck, left, right) - - case LESS_THAN => - val left = operands.head - val right = operands(1) - requireComparable(left) - requireComparable(right) - generateComparison("<", nullCheck, left, right) - - case LESS_THAN_OR_EQUAL => - val left = operands.head - val right = operands(1) - requireComparable(left) - requireComparable(right) - generateComparison("<=", nullCheck, left, right) - - case IS_NULL => - val operand = operands.head - generateIsNull(nullCheck, operand) - - case IS_NOT_NULL => - val operand = operands.head - generateIsNotNull(nullCheck, operand) - - // logic - case AND => - operands.reduceLeft { (left: GeneratedExpression, right: GeneratedExpression) => - requireBoolean(left) - requireBoolean(right) - generateAnd(nullCheck, left, right) - } - - case OR => - operands.reduceLeft { (left: GeneratedExpression, right: GeneratedExpression) => - requireBoolean(left) - requireBoolean(right) - generateOr(nullCheck, left, right) - } - - case NOT => - val operand = operands.head - requireBoolean(operand) - generateNot(nullCheck, operand) - - case CASE => - generateIfElse(nullCheck, operands, resultType) - - case IS_TRUE => - val operand = operands.head - requireBoolean(operand) - generateIsTrue(operand) - - case IS_NOT_TRUE => - val operand = operands.head - requireBoolean(operand) - generateIsNotTrue(operand) - - case IS_FALSE => - val operand = operands.head - requireBoolean(operand) - generateIsFalse(operand) - - case IS_NOT_FALSE => - val operand = operands.head - requireBoolean(operand) - generateIsNotFalse(operand) - - case IN => - val left = operands.head - val right = operands.tail - generateIn(this, left, right) - - // casting - case CAST | REINTERPRET => - val operand = operands.head - generateCast(nullCheck, operand, resultType) - - // as / renaming - case AS => - operands.head - - // string arithmetic - case CONCAT => - val left = operands.head - val right = operands(1) - requireString(left) - generateArithmeticOperator("+", nullCheck, resultType, left, right, config) - - // rows - case ROW => - generateRow(this, resultType, operands) - - // arrays - case ARRAY_VALUE_CONSTRUCTOR => - generateArray(this, resultType, operands) - - // maps - case MAP_VALUE_CONSTRUCTOR => - generateMap(this, resultType, operands) - - case ITEM => - operands.head.resultType match { - case t: TypeInformation[_] if isArray(t) => - val array = operands.head - val index = operands(1) - requireInteger(index) - generateArrayElementAt(this, array, index) - - case t: TypeInformation[_] if isMap(t) => - val key = operands(1) - generateMapGet(this, operands.head, key) - - case _ => throw new CodeGenException("Expect an array or a map.") - } - - case CARDINALITY => - operands.head.resultType match { - case t: TypeInformation[_] if isArray(t) => - val array = operands.head - generateArrayCardinality(nullCheck, array) - - case t: TypeInformation[_] if isMap(t) => - val map = operands.head - generateMapCardinality(nullCheck, map) - - case _ => throw new CodeGenException("Expect an array or a map.") - } - - case ELEMENT => - val array = operands.head - requireArray(array) - generateArrayElement(this, array) - - case DOT => - // Due to https://issues.apache.org/jira/browse/CALCITE-2162, expression such as - // "array[1].a.b" won't work now. - if (operands.size > 2) { - throw new CodeGenException( - "A DOT operator with more than 2 operands is not supported yet.") - } - val fieldName = call.operands.get(1).asInstanceOf[RexLiteral].getValueAs(classOf[String]) - val fieldIdx = operands - .head - .resultType - .asInstanceOf[CompositeType[_]] - .getFieldIndex(fieldName) - generateFieldAccess(operands.head, fieldIdx) - - case ScalarSqlFunctions.CONCAT => - generateConcat(this.nullCheck, operands) - - case ScalarSqlFunctions.CONCAT_WS => - generateConcatWs(operands) - - case StreamRecordTimestampSqlFunction => - generateStreamRecordRowtimeAccess() - - // advanced scalar functions - case sqlOperator: SqlOperator => - val callGen = FunctionGenerator.getCallGenerator( - sqlOperator, - operands.map(_.resultType), - resultType) - callGen - .getOrElse(throw new CodeGenException(s"Unsupported call: $sqlOperator \n" + - s"If you think this function should be supported, " + - s"you can create an issue and start a discussion for it.")) - .generate(this, operands) - - // unknown or invalid - case call@_ => - throw new CodeGenException(s"Unsupported call: $call") - } - } - - override def visitOver(over: RexOver): GeneratedExpression = - throw new CodeGenException("Aggregate functions over windows are not supported yet.") - - override def visitSubQuery(subQuery: RexSubQuery): GeneratedExpression = - throw new CodeGenException("Subqueries are not supported yet.") - - override def visitPatternFieldRef(fieldRef: RexPatternFieldRef): GeneratedExpression = - throw new CodeGenException("Pattern field references are not supported yet.") - - // ---------------------------------------------------------------------------------------------- - // generator helping methods - // ---------------------------------------------------------------------------------------------- - - private def generateCodeSplits(splits: Seq[String]): String = { - val totalLen = splits.map(_.length + 1).sum // 1 for a line break - - // split - if (totalLen > config.getMaxGeneratedCodeLength) { - - hasCodeSplits = true - - // add input unboxing to member area such that all split functions can access it - reusableInputUnboxingExprs.foreach { case (_, expr) => - - // declaration - val resultTypeTerm = primitiveTypeTermForTypeInfo(expr.resultType) - if (nullCheck) { - reusableMemberStatements.add(s"private boolean ${expr.nullTerm};") - } - reusableMemberStatements.add(s"private $resultTypeTerm ${expr.resultTerm};") - - // assignment - if (nullCheck) { - reusablePerRecordStatements.add(s"this.${expr.nullTerm} = ${expr.nullTerm};") - } - reusablePerRecordStatements.add(s"this.${expr.resultTerm} = ${expr.resultTerm};") - } - - // add split methods to the member area and return the code necessary to call those methods - val methodCalls = splits.map { split => - val methodName = newName(s"split") - - val method = - s""" - |private final void $methodName() throws Exception { - | $split - |} - |""".stripMargin - reusableMemberStatements.add(method) - - // create method call - s"$methodName();" - } - - methodCalls.mkString("\n") - } - // don't split - else { - splits.mkString("\n") - } - } - - def generateFieldAccess(refExpr: GeneratedExpression, index: Int): GeneratedExpression = { - - val fieldAccessExpr = generateFieldAccess( - refExpr.resultType, - refExpr.resultTerm, - index) - - val resultTerm = newName("result") - val nullTerm = newName("isNull") - val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldAccessExpr.resultType) - val defaultValue = primitiveDefaultValue(fieldAccessExpr.resultType) - val resultCode = if (nullCheck) { - s""" - |${refExpr.code} - |$resultTypeTerm $resultTerm; - |boolean $nullTerm; - |if (${refExpr.nullTerm}) { - | $resultTerm = $defaultValue; - | $nullTerm = true; - |} - |else { - | ${fieldAccessExpr.code} - | $resultTerm = ${fieldAccessExpr.resultTerm}; - | $nullTerm = ${fieldAccessExpr.nullTerm}; - |} - |""".stripMargin - } else { - s""" - |${refExpr.code} - |${fieldAccessExpr.code} - |$resultTypeTerm $resultTerm = ${fieldAccessExpr.resultTerm}; - |""".stripMargin - } - - GeneratedExpression(resultTerm, nullTerm, resultCode, fieldAccessExpr.resultType) - } - - private def generateInputAccess( - inputType: TypeInformation[_ <: Any], - inputTerm: String, - index: Int) - : GeneratedExpression = { - // if input has been used before, we can reuse the code that - // has already been generated - val inputExpr = reusableInputUnboxingExprs.get((inputTerm, index)) match { - // input access and unboxing has already been generated - case Some(expr) => - expr - - // generate input access and unboxing if necessary - case None => - val expr = if (nullableInput) { - generateNullableInputFieldAccess(inputType, inputTerm, index) - } else { - generateFieldAccess(inputType, inputTerm, index) - } - - reusableInputUnboxingExprs((inputTerm, index)) = expr - expr - } - // hide the generated code as it will be executed only once - GeneratedExpression(inputExpr.resultTerm, inputExpr.nullTerm, "", inputExpr.resultType) - } - - private def generateNullableInputFieldAccess( - inputType: TypeInformation[_ <: Any], - inputTerm: String, - index: Int) - : GeneratedExpression = { - val resultTerm = newName("result") - val nullTerm = newName("isNull") - - val fieldType = inputType match { - case ct: CompositeType[_] => ct.getTypeAt(index) - case t: TypeInformation[_] => t - } - val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType) - val defaultValue = primitiveDefaultValue(fieldType) - val fieldAccessExpr = generateFieldAccess(inputType, inputTerm, index) - - val inputCheckCode = - s""" - |$resultTypeTerm $resultTerm; - |boolean $nullTerm; - |if ($inputTerm == null) { - | $resultTerm = $defaultValue; - | $nullTerm = true; - |} - |else { - | ${fieldAccessExpr.code} - | $resultTerm = ${fieldAccessExpr.resultTerm}; - | $nullTerm = ${fieldAccessExpr.nullTerm}; - |} - |""".stripMargin - - GeneratedExpression(resultTerm, nullTerm, inputCheckCode, fieldType) - } - - def generateFieldAccess( - inputType: TypeInformation[_], - inputTerm: String, - index: Int) - : GeneratedExpression = { - inputType match { - - case ct: CompositeType[_] => - val accessor = fieldAccessorFor(ct, index) - val fieldType: TypeInformation[Any] = ct.getTypeAt(index) - val fieldTypeTerm = boxedTypeTermForTypeInfo(fieldType) - - accessor match { - case ObjectFieldAccessor(field) => - // primitive - if (isFieldPrimitive(field)) { - generateTerm(fieldType, s"$inputTerm.${field.getName}") - } - // Object - else { - generateInputFieldUnboxing( - fieldType, - s"($fieldTypeTerm) $inputTerm.${field.getName}") - } - - case ObjectGenericFieldAccessor(fieldName) => - // Object - val inputCode = s"($fieldTypeTerm) $inputTerm.$fieldName" - generateInputFieldUnboxing(fieldType, inputCode) - - case ObjectMethodAccessor(methodName) => - // Object - val inputCode = s"($fieldTypeTerm) $inputTerm.$methodName()" - generateInputFieldUnboxing(fieldType, inputCode) - - case ProductAccessor(i) => - // Object - val inputCode = s"($fieldTypeTerm) $inputTerm.getField($i)" - generateInputFieldUnboxing(fieldType, inputCode) - - case ObjectPrivateFieldAccessor(field) => - val fieldTerm = addReusablePrivateFieldAccess(ct.getTypeClass, field.getName) - val reflectiveAccessCode = reflectiveFieldReadAccess(fieldTerm, field, inputTerm) - // primitive - if (isFieldPrimitive(field)) { - generateTerm(fieldType, reflectiveAccessCode) - } - // Object - else { - generateInputFieldUnboxing(fieldType, reflectiveAccessCode) - } - } - - case t: TypeInformation[_] => - val fieldTypeTerm = boxedTypeTermForTypeInfo(t) - val inputCode = s"($fieldTypeTerm) $inputTerm" - generateInputFieldUnboxing(t, inputCode) - } - } - - private def generateNullLiteral(resultType: TypeInformation[_]): GeneratedExpression = { - val resultTerm = newName("result") - val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType) - val defaultValue = primitiveDefaultValue(resultType) - - if (nullCheck) { - val wrappedCode = s""" - |$resultTypeTerm $resultTerm = $defaultValue; - |""".stripMargin - - // mark this expression as a constant literal - GeneratedExpression(resultTerm, ALWAYS_NULL, wrappedCode, resultType, literal = true) - } else { - throw new CodeGenException("Null literals are not allowed if nullCheck is disabled.") - } - } - - private[flink] def generateNonNullLiteral( - literalType: TypeInformation[_], - literalCode: String) - : GeneratedExpression = { - - // mark this expression as a constant literal - generateTerm(literalType, literalCode).copy(literal = true) - } - - private[flink] def generateSymbol(enum: Enum[_]): GeneratedExpression = { - GeneratedExpression( - qualifyEnum(enum), - NEVER_NULL, - NO_CODE, - new GenericTypeInfo(enum.getDeclaringClass)) - } - - /** - * Generates access to a term (e.g. a field) that does not require unboxing logic. - * - * @param fieldType type of field - * @param fieldTerm expression term of field (already unboxed) - * @return internal unboxed field representation - */ - private[flink] def generateTerm( - fieldType: TypeInformation[_], - fieldTerm: String) - : GeneratedExpression = { - val resultTerm = newName("result") - val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType) - - val resultCode = s""" - |$resultTypeTerm $resultTerm = $fieldTerm; - |""".stripMargin - - GeneratedExpression(resultTerm, NEVER_NULL, resultCode, fieldType) - } - - /** - * Converts the external boxed format to an internal mostly primitive field representation. - * Wrapper types can autoboxed to their corresponding primitive type (Integer -> int). External - * objects are converted to their internal representation (Timestamp -> internal timestamp - * in long). - * - * @param fieldType type of field - * @param fieldTerm expression term of field to be unboxed - * @return internal unboxed field representation - */ - private[flink] def generateInputFieldUnboxing( - fieldType: TypeInformation[_], - fieldTerm: String) - : GeneratedExpression = { - val resultTerm = newName("result") - val nullTerm = newName("isNull") - val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType) - val defaultValue = primitiveDefaultValue(fieldType) - - // explicit unboxing - val unboxedFieldCode = if (isTimePoint(fieldType)) { - timePointToInternalCode(fieldType, fieldTerm) - } else { - fieldTerm - } - - val wrappedCode = if (nullCheck && !isReference(fieldType)) { - // assumes that fieldType is a boxed primitive. - s""" - |boolean $nullTerm = $fieldTerm == null; - |$resultTypeTerm $resultTerm; - |if ($nullTerm) { - | $resultTerm = $defaultValue; - |} - |else { - | $resultTerm = $fieldTerm; - |} - |""".stripMargin - } else if (nullCheck) { - s""" - |boolean $nullTerm = $fieldTerm == null; - |$resultTypeTerm $resultTerm; - |if ($nullTerm) { - | $resultTerm = $defaultValue; - |} - |else { - | $resultTerm = ($resultTypeTerm) $unboxedFieldCode; - |} - |""".stripMargin - } else { - s""" - |$resultTypeTerm $resultTerm = ($resultTypeTerm) $unboxedFieldCode; - |""".stripMargin - } - - GeneratedExpression(resultTerm, nullTerm, wrappedCode, fieldType) - } - - /** - * Converts the internal mostly primitive field representation to an external boxed format. - * Primitive types can autoboxed to their corresponding object type (int -> Integer). Internal - * representations are converted to their external objects (internal timestamp - * in long -> Timestamp). - * - * @param expr expression to be boxed - * @return external boxed field representation - */ - private[flink] def generateOutputFieldBoxing(expr: GeneratedExpression): GeneratedExpression = { - expr.resultType match { - // convert internal date/time/timestamp to java.sql.* objects - case SqlTimeTypeInfo.DATE | SqlTimeTypeInfo.TIME | SqlTimeTypeInfo.TIMESTAMP => - val resultTerm = newName("result") - val resultTypeTerm = boxedTypeTermForTypeInfo(expr.resultType) - val convMethod = internalToTimePointCode(expr.resultType, expr.resultTerm) - - val resultCode = if (nullCheck) { - s""" - |${expr.code} - |$resultTypeTerm $resultTerm; - |if (${expr.nullTerm}) { - | $resultTerm = null; - |} - |else { - | $resultTerm = $convMethod; - |} - |""".stripMargin - } else { - s""" - |${expr.code} - |$resultTypeTerm $resultTerm = $convMethod; - |""".stripMargin - } - - GeneratedExpression(resultTerm, expr.nullTerm, resultCode, expr.resultType) - - // other types are autoboxed or need no boxing - case _ => expr - } - } - - private[flink] def generateNullableOutputBoxing( - expr: GeneratedExpression, - typeInfo: TypeInformation[_]) - : GeneratedExpression = { - val boxedExpr = generateOutputFieldBoxing(generateCast(nullCheck, expr, typeInfo)) - val boxedTypeTerm = boxedTypeTermForTypeInfo(typeInfo) - val exprOrNull: String = if (nullCheck) { - s"${boxedExpr.nullTerm} ? null : ($boxedTypeTerm) ${boxedExpr.resultTerm}" - } else { - boxedExpr.resultTerm - } - boxedExpr.copy(resultTerm = exprOrNull) - } - - private[flink] def generateStreamRecordRowtimeAccess(): GeneratedExpression = { - val resultTerm = newName("result") - val nullTerm = newName("isNull") - - val accessCode = - s""" - |Long $resultTerm = $contextTerm.timestamp(); - |if ($resultTerm == null) { - | throw new RuntimeException("Rowtime timestamp is null. Please make sure that a proper " + - | "TimestampAssigner is defined and the stream environment uses the EventTime time " + - | "characteristic."); - |} - |boolean $nullTerm = false; - """.stripMargin - - GeneratedExpression(resultTerm, nullTerm, accessCode, Types.LONG) - } - - private[flink] def generateProctimeTimestamp(): GeneratedExpression = { - val resultTerm = newName("result") - - val resultCode = - s""" - |long $resultTerm = $contextTerm.timerService().currentProcessingTime(); - |""".stripMargin - GeneratedExpression(resultTerm, NEVER_NULL, resultCode, SqlTimeTypeInfo.TIMESTAMP) - } - - private[flink] def generateCurrentTimestamp(): GeneratedExpression = { - new CurrentTimePointCallGen(Types.SQL_TIMESTAMP, false).generate(this, Seq()) - } - - // ---------------------------------------------------------------------------------------------- - // Reusable code snippets - // ---------------------------------------------------------------------------------------------- - - /** - * Adds a reusable output record to the member area of the generated [[Function]]. - * The passed [[TypeInformation]] defines the type class to be instantiated. - * - * @param ti type information of type class to be instantiated during runtime - * @return member variable term - */ - def addReusableOutRecord(ti: TypeInformation[_]): Unit = { - val statement = ti match { - case rt: RowTypeInfo => - s""" - |final ${ti.getTypeClass.getCanonicalName} $outRecordTerm = - | new ${ti.getTypeClass.getCanonicalName}(${rt.getArity}); - |""".stripMargin - case _ => - s""" - |final ${ti.getTypeClass.getCanonicalName} $outRecordTerm = - | new ${ti.getTypeClass.getCanonicalName}(); - |""".stripMargin - } - reusableMemberStatements.add(statement) - } - - /** - * Adds a reusable [[java.lang.reflect.Field]] to the member area of the generated [[Function]]. - * The field can be used for accessing POJO fields more efficiently during runtime, however, - * the field does not have to be public. - * - * @param clazz class of containing field - * @param fieldName name of field to be extracted and instantiated during runtime - * @return member variable term - */ - def addReusablePrivateFieldAccess(clazz: Class[_], fieldName: String): String = { - val fieldTerm = s"field_${clazz.getCanonicalName.replace('.', '$')}_$fieldName" - val fieldExtraction = - s""" - |final java.lang.reflect.Field $fieldTerm = - | org.apache.flink.api.java.typeutils.TypeExtractor.getDeclaredField( - | ${clazz.getCanonicalName}.class, "$fieldName"); - |""".stripMargin - reusableMemberStatements.add(fieldExtraction) - - val fieldAccessibility = - s""" - |$fieldTerm.setAccessible(true); - |""".stripMargin - reusableInitStatements.add(fieldAccessibility) - - fieldTerm - } - - /** - * Adds a reusable [[java.math.BigDecimal]] to the member area of the generated [[Function]]. - * - * @param decimal decimal object to be instantiated during runtime - * @return member variable term - */ - def addReusableDecimal(decimal: JBigDecimal): String = decimal match { - case JBigDecimal.ZERO => "java.math.BigDecimal.ZERO" - case JBigDecimal.ONE => "java.math.BigDecimal.ONE" - case JBigDecimal.TEN => "java.math.BigDecimal.TEN" - case _ => - val fieldTerm = newName("decimal") - val fieldDecimal = - s""" - |final java.math.BigDecimal $fieldTerm = - | new java.math.BigDecimal("${decimal.toString}"); - |""".stripMargin - reusableMemberStatements.add(fieldDecimal) - fieldTerm - } - - /** - * Adds a reusable [[java.util.Random]] to the member area of the generated [[Function]]. - * - * The seed parameter must be a literal/constant expression. - * - * @return member variable term - */ - def addReusableRandom(seedExpr: Option[GeneratedExpression]): String = { - val fieldTerm = newName("random") - - val field = - s""" - |final java.util.Random $fieldTerm; - |""".stripMargin - reusableMemberStatements.add(field) - - val fieldInit = seedExpr match { - case Some(s) if nullCheck => - s""" - |${s.code} - |if(!${s.nullTerm}) { - | $fieldTerm = new java.util.Random(${s.resultTerm}); - |} - |else { - | $fieldTerm = new java.util.Random(); - |} - |""".stripMargin - case Some(s) => - s""" - |${s.code} - |$fieldTerm = new java.util.Random(${s.resultTerm}); - |""".stripMargin - case _ => - s""" - |$fieldTerm = new java.util.Random(); - |""".stripMargin - } - - reusableInitStatements.add(fieldInit) - fieldTerm - } - - /** - * Adds a reusable DateFormatter to the member area of the generated [[Function]]. - * - * @return member variable term - */ - def addReusableDateFormatter(format: GeneratedExpression): String = { - val fieldTerm = newName("dateFormatter") - - val field = - s""" - |final ${classOf[DateTimeFormatter].getCanonicalName} $fieldTerm; - |""".stripMargin - reusableMemberStatements.add(field) - - val fieldInit = - s""" - |${format.code} - |$fieldTerm = org.apache.flink.table.runtime.functions. - |DateTimeFunctions$$.MODULE$$.createDateTimeFormatter(${format.resultTerm}); - |""".stripMargin - - reusableInitStatements.add(fieldInit) - fieldTerm - } - - /** - * Adds a reusable [[UserDefinedFunction]] to the member area of the generated [[Function]]. - * - * @param function [[UserDefinedFunction]] object to be instantiated during runtime - * @param contextTerm [[RuntimeContext]] term to access the [[RuntimeContext]] - * @return member variable term - */ - def addReusableFunction(function: UserDefinedFunction, contextTerm: String = null): String = { - val classQualifier = function.getClass.getCanonicalName - val functionSerializedData = UserDefinedFunctionUtils.serialize(function) - val fieldTerm = s"function_${function.functionIdentifier}" - - val fieldFunction = - s""" - |final $classQualifier $fieldTerm; - |""".stripMargin - reusableMemberStatements.add(fieldFunction) - - val functionDeserialization = - s""" - |$fieldTerm = ($classQualifier) - |${UserDefinedFunctionUtils.getClass.getName.stripSuffix("$")} - |.deserialize("$functionSerializedData"); - """.stripMargin - - reusableInitStatements.add(functionDeserialization) - - val openFunction = if (contextTerm != null) { - s""" - |$fieldTerm.open(new ${classOf[FunctionContext].getCanonicalName}($contextTerm)); - """.stripMargin - } else { - s""" - |$fieldTerm.open(new ${classOf[FunctionContext].getCanonicalName}(getRuntimeContext())); - """.stripMargin - } - reusableOpenStatements.add(openFunction) - - val closeFunction = - s""" - |$fieldTerm.close(); - """.stripMargin - reusableCloseStatements.add(closeFunction) - - fieldTerm - } - - /** - * Adds a reusable constructor statement with the given parameter types. - * - * @param parameterTypes The parameter types to construct the function - * @return member variable terms - */ - def addReusableConstructor(parameterTypes: Class[_]*): Array[String] = { - val parameters = mutable.ListBuffer[String]() - val fieldTerms = mutable.ListBuffer[String]() - val body = mutable.ListBuffer[String]() - - parameterTypes.zipWithIndex.foreach { case (t, index) => - val classQualifier = t.getCanonicalName - val fieldTerm = newName(s"instance_${classQualifier.replace('.', '$')}") - val field = s"final $classQualifier $fieldTerm;" - reusableMemberStatements.add(field) - fieldTerms += fieldTerm - parameters += s"$classQualifier arg$index" - body += s"$fieldTerm = arg$index;" - } - - reusableConstructorStatements.add((parameters.mkString(","), body.mkString("", "\n", "\n"))) - - fieldTerms.toArray - } - - /** - * Adds a reusable [[org.apache.flink.types.Row]] - * to the member area of the generated [[Function]]. - */ - def addReusableRow(arity: Int): String = { - val fieldTerm = newName("row") - val fieldRow = - s""" - |final org.apache.flink.types.Row $fieldTerm = - | new org.apache.flink.types.Row($arity); - |""".stripMargin - reusableMemberStatements.add(fieldRow) - fieldTerm - } - - /** - * Adds a reusable array to the member area of the generated [[Function]]. - */ - def addReusableArray(clazz: Class[_], size: Int): String = { - val fieldTerm = newName("array") - val classQualifier = clazz.getCanonicalName // works also for int[] etc. - val initArray = classQualifier.replaceFirst("\\[", s"[$size") - val fieldArray = - s""" - |final $classQualifier $fieldTerm = - | new $initArray; - |""".stripMargin - reusableMemberStatements.add(fieldArray) - fieldTerm - } - - /** - * Adds a reusable hash map to the member area of the generated [[Function]]. - */ - def addReusableMap(): String = { - val fieldTerm = newName("map") - val classQualifier = "java.util.Map" - val initMap = "java.util.HashMap()" - val fieldMap = - s""" - |final $classQualifier $fieldTerm = - | new $initMap; - |""".stripMargin - reusableMemberStatements.add(fieldMap) - fieldTerm - } - - /** - * Adds a reusable timestamp to the beginning of the SAM of the generated [[Function]]. - */ - def addReusableTimestamp(): String = { - val fieldTerm = s"timestamp" - - // declaration - reusableMemberStatements.add(s"private long $fieldTerm;") - - // assignment - val field = - s""" - |$fieldTerm = java.lang.System.currentTimeMillis(); - |""".stripMargin - reusablePerRecordStatements.add(field) - fieldTerm - } - - /** - * Adds a reusable local timestamp to the beginning of the SAM of the generated [[Function]]. - */ - def addReusableLocalTimestamp(): String = { - val fieldTerm = s"localtimestamp" - - val timestamp = addReusableTimestamp() - - // declaration - reusableMemberStatements.add(s"private long $fieldTerm;") - - // assignment - val field = - s""" - |$fieldTerm = $timestamp + java.util.TimeZone.getDefault().getOffset($timestamp); - |""".stripMargin - reusablePerRecordStatements.add(field) - fieldTerm - } - - /** - * Adds a reusable time to the beginning of the SAM of the generated [[Function]]. - */ - def addReusableTime(): String = { - val fieldTerm = s"time" - - val timestamp = addReusableTimestamp() - - // declaration - reusableMemberStatements.add(s"private int $fieldTerm;") - - // assignment - // adopted from org.apache.calcite.runtime.SqlFunctions.currentTime() - val field = - s""" - |$fieldTerm = (int) ($timestamp % ${DateTimeUtils.MILLIS_PER_DAY}); - |if (time < 0) { - | time += ${DateTimeUtils.MILLIS_PER_DAY}; - |} - |""".stripMargin - reusablePerRecordStatements.add(field) - fieldTerm - } - - /** - * Adds a reusable local time to the beginning of the SAM of the generated [[Function]]. - */ - def addReusableLocalTime(): String = { - val fieldTerm = s"localtime" - - val localtimestamp = addReusableLocalTimestamp() - - // declaration - reusableMemberStatements.add(s"private int $fieldTerm;") - - // assignment - // adopted from org.apache.calcite.runtime.SqlFunctions.localTime() - val field = - s""" - |$fieldTerm = (int) ($localtimestamp % ${DateTimeUtils.MILLIS_PER_DAY}); - |""".stripMargin - reusablePerRecordStatements.add(field) - fieldTerm - } - - - /** - * Adds a reusable date to the beginning of the SAM of the generated [[Function]]. - */ - def addReusableDate(): String = { - val fieldTerm = s"date" - - val timestamp = addReusableTimestamp() - val time = addReusableTime() - - // declaration - reusableMemberStatements.add(s"private int $fieldTerm;") - - // assignment - // adopted from org.apache.calcite.runtime.SqlFunctions.currentDate() - val field = - s""" - |$fieldTerm = (int) ($timestamp / ${DateTimeUtils.MILLIS_PER_DAY}); - |if ($time < 0) { - | $fieldTerm -= 1; - |} - |""".stripMargin - reusablePerRecordStatements.add(field) - fieldTerm - } - - /** - * Adds a reusable [[java.util.HashSet]] to the member area of the generated [[Function]]. - * - * @param elements elements to be added to the set (including null) - * @return member variable term - */ - def addReusableSet(elements: Seq[GeneratedExpression]): String = { - val fieldTerm = newName("set") - - val field = - s""" - |final java.util.Set $fieldTerm; - |""".stripMargin - reusableMemberStatements.add(field) - - val init = - s""" - |$fieldTerm = new java.util.HashSet(); - |""".stripMargin - reusableInitStatements.add(init) - - elements.foreach { element => - val content = - s""" - |${element.code} - |if (${element.nullTerm}) { - | $fieldTerm.add(null); - |} else { - | $fieldTerm.add(${element.resultTerm}); - |} - |""".stripMargin - - reusableInitStatements.add(content) - } - - fieldTerm - } - - /** - * Adds a reusable constant to the member area of the generated [[Function]]. - * - * @param constant constant expression - * @return member variable term - */ - def addReusableBoxedConstant(constant: GeneratedExpression): String = { - require(constant.literal, "Literal expected") - - val fieldTerm = newName("constant") - - val boxed = generateOutputFieldBoxing(constant) - val boxedType = boxedTypeTermForTypeInfo(boxed.resultType) - - val field = - s""" - |final $boxedType $fieldTerm; - |""".stripMargin - reusableMemberStatements.add(field) - - val init = - s""" - |${boxed.code} - |$fieldTerm = ${boxed.resultTerm}; - |""".stripMargin - reusableInitStatements.add(init) - - fieldTerm - } - - /** - * Adds a reusable MessageDigest to the member area of the generated [[Function]]. - * - * @return member variable term - */ - def addReusableMessageDigest(algorithm: String): String = { - val fieldTerm = newName("messageDigest") - - val field = - s""" - |final java.security.MessageDigest $fieldTerm; - |""".stripMargin - reusableMemberStatements.add(field) - - val fieldInit = - s""" - |try { - | $fieldTerm = java.security.MessageDigest.getInstance("$algorithm"); - |} catch (java.security.NoSuchAlgorithmException e) { - | throw new RuntimeException("Algorithm for '$algorithm' is not available.", e); - |} - |""".stripMargin - - reusableInitStatements.add(fieldInit) - fieldTerm - } - - def addReusableInitStatement(initStatement: String): Unit = { - reusableInitStatements.add(initStatement) - } - - def addReusableMemberStatement(memberStatement: String): Unit = { - reusableMemberStatements.add(memberStatement) - } -} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala b/core/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala deleted file mode 100644 index d08dd570f..000000000 --- a/core/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala +++ /dev/null @@ -1,574 +0,0 @@ -package org.apache.flink.table.codegen - -import java.math.{BigDecimal => JBigDecimal} -import java.util - -import org.apache.calcite.rel.RelCollation -import org.apache.calcite.rex._ -import org.apache.calcite.sql.fun.SqlStdOperatorTable.{CLASSIFIER, FINAL, FIRST, LAST, MATCH_NUMBER, NEXT, PREV, RUNNING} -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.cep.{PatternFlatSelectFunction, PatternSelectFunction} -import org.apache.flink.cep.pattern.conditions.IterativeCondition -import org.apache.flink.table.api.TableConfig -import org.apache.flink.table.calcite.FlinkTypeFactory -import org.apache.flink.table.codegen.Indenter.toISC -import org.apache.flink.table.codegen.CodeGenUtils.{boxedTypeTermForTypeInfo, newName, primitiveDefaultValue} -import org.apache.flink.table.plan.schema.RowSchema -import org.apache.flink.types.Row - -import scala.collection.JavaConverters._ -import scala.collection.mutable - -/** - * A code generator for generating CEP related functions. - * - * @param config configuration that determines runtime behavior - * @param nullableInput input(s) can be null. - * @param input type information about the first input of the Function - * @param patternNames the names of patterns - * @param generateCondition whether the code generator is generating [[IterativeCondition]] - * @param patternName the name of current pattern - */ -class MatchCodeGenerator( - config: TableConfig, - nullableInput: Boolean, - input: TypeInformation[_ <: Any], - patternNames: Seq[String], - generateCondition: Boolean, - patternName: Option[String] = None) - extends CodeGenerator(config,nullableInput,input){ - /** - * @return term of pattern names - */ - private val patternNameListTerm = newName("patternNameList") - - /** - * @return term of current pattern which is processing - */ - private val currPatternTerm = newName("currPattern") - - /** - * @return term of current event which is processing - */ - private val currEventTerm = newName("currEvent") - - private val buildPatternNameList: String = { - for (patternName <- patternNames) yield - s""" - |$patternNameListTerm.add("$patternName"); - |""".stripMargin - }.mkString("\n") - - def addReusableStatements(): Unit = { - val eventTypeTerm = boxedTypeTermForTypeInfo(input) - val memberStatement = - s""" - |$eventTypeTerm $currEventTerm = null; - |String $currPatternTerm = null; - |java.util.List $patternNameListTerm = new java.util.ArrayList(); - |""".stripMargin - addReusableMemberStatement(memberStatement) - - addReusableInitStatement(buildPatternNameList) - } - - /** - * Generates a [[IterativeCondition]] that can be passed to Java compiler. - * - * @param name Class name of the function. Must not be unique but has to be a - * valid Java class identifier. - * @param bodyCode body code for the function - * @return a GeneratedIterativeCondition - */ - def generateIterativeCondition( - name: String, - bodyCode: String) - : GeneratedIterativeCondition = { - - val funcName = newName(name) - val inputTypeTerm = boxedTypeTermForTypeInfo(input) - - val funcCode = j""" - public class $funcName - extends ${classOf[IterativeCondition[_]].getCanonicalName} { - - ${reuseMemberCode()} - - public $funcName() throws Exception { - ${reuseInitCode()} - } - - @Override - public boolean filter( - Object _in1, ${classOf[IterativeCondition.Context[_]].getCanonicalName} $contextTerm) - throws Exception { - - $inputTypeTerm $input1Term = ($inputTypeTerm) _in1; - ${reusePerRecordCode()} - ${reuseInputUnboxingCode()} - $bodyCode - } - } - """.stripMargin - - GeneratedIterativeCondition(funcName, funcCode) - } - - /** - * Generates a [[PatternSelectFunction]] that can be passed to Java compiler. - * - * @param name Class name of the function. Must not be unique but has to be a - * valid Java class identifier. - * @param bodyCode body code for the function - * @return a GeneratedPatternSelectFunction - */ - def generatePatternSelectFunction( - name: String, - bodyCode: String) - : GeneratedPatternSelectFunction = { - - val funcName = newName(name) - val inputTypeTerm = - classOf[java.util.Map[java.lang.String, java.util.List[Row]]].getCanonicalName - - val funcCode = j""" - public class $funcName - implements ${classOf[PatternSelectFunction[_, _]].getCanonicalName} { - - ${reuseMemberCode()} - - public $funcName() throws Exception { - ${reuseInitCode()} - } - - @Override - public Object select(java.util.Map> _in1) - throws Exception { - - $inputTypeTerm $input1Term = ($inputTypeTerm) _in1; - ${reusePerRecordCode()} - ${reuseInputUnboxingCode()} - $bodyCode - } - } - """.stripMargin - - GeneratedPatternSelectFunction(funcName, funcCode) - } - - /** - * Generates a [[PatternFlatSelectFunction]] that can be passed to Java compiler. - * - * @param name Class name of the function. Must not be unique but has to be a - * valid Java class identifier. - * @param bodyCode body code for the function - * @return a GeneratedPatternFlatSelectFunction - */ - def generatePatternFlatSelectFunction( - name: String, - bodyCode: String) - : GeneratedPatternFlatSelectFunction = { - - val funcName = newName(name) - val inputTypeTerm = - classOf[java.util.Map[java.lang.String, java.util.List[Row]]].getCanonicalName - - val funcCode = j""" - public class $funcName - implements ${classOf[PatternFlatSelectFunction[_, _]].getCanonicalName} { - - ${reuseMemberCode()} - - public $funcName() throws Exception { - ${reuseInitCode()} - } - - @Override - public void flatSelect(java.util.Map> _in1, - org.apache.flink.util.Collector $collectorTerm) - throws Exception { - - $inputTypeTerm $input1Term = ($inputTypeTerm) _in1; - ${reusePerRecordCode()} - ${reuseInputUnboxingCode()} - $bodyCode - } - } - """.stripMargin - - GeneratedPatternFlatSelectFunction(funcName, funcCode) - } - - def generateSelectOutputExpression( - partitionKeys: util.List[RexNode], - measures: util.Map[String, RexNode], - returnType: RowSchema) - : GeneratedExpression = { - - val eventNameTerm = newName("event") - val eventTypeTerm = boxedTypeTermForTypeInfo(input) - - // For "ONE ROW PER MATCH", the output columns include: - // 1) the partition columns; - // 2) the columns defined in the measures clause. - val resultExprs = - partitionKeys.asScala.map { case inputRef: RexInputRef => - generateFieldAccess(input, eventNameTerm, inputRef.getIndex) - } ++ returnType.fieldNames.filter(measures.containsKey(_)).map { fieldName => - generateExpression(measures.get(fieldName)) - } - - val resultExpression = generateResultExpression( - resultExprs, - returnType.typeInfo, - returnType.fieldNames) - - val resultCode = - s""" - |$eventTypeTerm $eventNameTerm = null; - |if (${partitionKeys.size()} > 0) { - | for (java.util.Map.Entry entry : $input1Term.entrySet()) { - | java.util.List value = (java.util.List) entry.getValue(); - | if (value != null && value.size() > 0) { - | $eventNameTerm = ($eventTypeTerm) value.get(0); - | break; - | } - | } - |} - | - |${resultExpression.code} - |""".stripMargin - - resultExpression.copy(code = resultCode) - } - - def generateFlatSelectOutputExpression( - partitionKeys: util.List[RexNode], - orderKeys: RelCollation, - measures: util.Map[String, RexNode], - returnType: RowSchema) - : GeneratedExpression = { - - val patternNameTerm = newName("patternName") - val eventNameTerm = newName("event") - val eventNameListTerm = newName("eventList") - val eventTypeTerm = boxedTypeTermForTypeInfo(input) - val listTypeTerm = classOf[java.util.List[_]].getCanonicalName - - // For "ALL ROWS PER MATCH", the output columns include: - // 1) the partition columns; - // 2) the ordering columns; - // 3) the columns defined in the measures clause; - // 4) any remaining columns defined of the input. - val fieldsAccessed = mutable.Set[Int]() - val resultExprs = - partitionKeys.asScala.map { case inputRef: RexInputRef => - fieldsAccessed += inputRef.getIndex - generateFieldAccess(input, eventNameTerm, inputRef.getIndex) - } ++ orderKeys.getFieldCollations.asScala.map { fieldCollation => - fieldsAccessed += fieldCollation.getFieldIndex - generateFieldAccess(input, eventNameTerm, fieldCollation.getFieldIndex) - } ++ (0 until input.getArity).filterNot(fieldsAccessed.contains).map { idx => - generateFieldAccess(input, eventNameTerm, idx) - } ++ returnType.fieldNames.filter(measures.containsKey(_)).map { fieldName => - generateExpression(measures.get(fieldName)) - } - - val resultExpression = generateResultExpression( - resultExprs, - returnType.typeInfo, - returnType.fieldNames) - - val resultCode = - s""" - |for (String $patternNameTerm : $patternNameListTerm) { - | $currPatternTerm = $patternNameTerm; - | $listTypeTerm $eventNameListTerm = ($listTypeTerm) $input1Term.get($patternNameTerm); - | if ($eventNameListTerm != null) { - | for ($eventTypeTerm $eventNameTerm : $eventNameListTerm) { - | $currEventTerm = $eventNameTerm; - | ${resultExpression.code} - | $collectorTerm.collect(${resultExpression.resultTerm}); - | } - | } - |} - |$currPatternTerm = null; - |$currEventTerm = null; - |""".stripMargin - - GeneratedExpression("", "false", resultCode, null) - } - - override def visitCall(call: RexCall): GeneratedExpression = { - val resultType = FlinkTypeFactory.toTypeInfo(call.getType) - call.getOperator match { - case PREV => - val countLiteral = call.operands.get(1).asInstanceOf[RexLiteral] - val count = countLiteral.getValue3.asInstanceOf[JBigDecimal].intValue() - generatePrev( - call.operands.get(0), - count, - resultType) - - case NEXT | CLASSIFIER | MATCH_NUMBER => - throw new CodeGenException(s"Unsupported call: $call") - - case FIRST | LAST => - val countLiteral = call.operands.get(1).asInstanceOf[RexLiteral] - val count = countLiteral.getValue3.asInstanceOf[JBigDecimal].intValue() - generateFirstLast( - call.operands.get(0), - count, - resultType, - running = true, - call.getOperator == FIRST) - - case RUNNING | FINAL => - generateRunningFinal( - call.operands.get(0), - resultType, - call.getOperator == RUNNING) - - case _ => super.visitCall(call) - } - } - - private def generatePrev( - rexNode: RexNode, - count: Int, - resultType: TypeInformation[_]) - : GeneratedExpression = { - rexNode match { - case patternFieldRef: RexPatternFieldRef => - if (count == 0 && patternFieldRef.getAlpha == patternName.get) { - // return current one - return visitInputRef(patternFieldRef) - } - - val listName = newName("patternEvents") - val resultTerm = newName("result") - val nullTerm = newName("isNull") - val indexTerm = newName("eventIndex") - val visitedEventNumberTerm = newName("visitedEventNumber") - val eventTerm = newName("event") - val resultTypeTerm = boxedTypeTermForTypeInfo(resultType) - val defaultValue = primitiveDefaultValue(resultType) - - val eventTypeTerm = boxedTypeTermForTypeInfo(input) - - val patternNamesToVisit = patternNames - .take(patternNames.indexOf(patternFieldRef.getAlpha) + 1) - .reverse - def findEventByPhysicalPosition: String = { - val init: String = - s""" - |java.util.List $listName = new java.util.ArrayList(); - |""".stripMargin - - val getResult: String = { - for (tmpPatternName <- patternNamesToVisit) yield - s""" - |for ($eventTypeTerm $eventTerm : $contextTerm - | .getEventsForPattern("$tmpPatternName")) { - | $listName.add($eventTerm); - |} - | - |$indexTerm = $listName.size() - ($count - $visitedEventNumberTerm); - |if ($indexTerm >= 0) { - | $resultTerm = ($resultTypeTerm) (($eventTypeTerm) $listName.get($indexTerm)) - | .getField(${patternFieldRef.getIndex}); - | $nullTerm = false; - | break; - |} - | - |$visitedEventNumberTerm += $listName.size(); - |$listName.clear(); - |""".stripMargin - }.mkString("\n") - - s""" - |$init - |$getResult - |""".stripMargin - } - - val resultCode = - s""" - |int $visitedEventNumberTerm = 0; - |int $indexTerm; - |$resultTypeTerm $resultTerm = $defaultValue; - |boolean $nullTerm = true; - |do { - | $findEventByPhysicalPosition - |} while (false); - |""".stripMargin - - GeneratedExpression(resultTerm, nullTerm, resultCode, resultType) - - case rexCall: RexCall => - val operands = rexCall.operands.asScala.map { - operand => generatePrev( - operand, - count, - FlinkTypeFactory.toTypeInfo(operand.getType)) - } - - generateCall(rexCall, operands, resultType) - - case _ => - generateExpression(rexNode) - } - } - - private def generateFirstLast( - rexNode: RexNode, - count: Int, - resultType: TypeInformation[_], - running: Boolean, - first: Boolean) - : GeneratedExpression = { - rexNode match { - case patternFieldRef: RexPatternFieldRef => - - val eventNameTerm = newName("event") - val resultTerm = newName("result") - val listName = newName("patternEvents") - val nullTerm = newName("isNull") - val patternNameTerm = newName("patternName") - val eventNameListTerm = newName("eventNameList") - val resultTypeTerm = boxedTypeTermForTypeInfo(resultType) - val defaultValue = primitiveDefaultValue(resultType) - - val eventTypeTerm = boxedTypeTermForTypeInfo(input) - val listTypeTerm = classOf[java.util.List[_]].getCanonicalName - - def findEventByLogicalPosition: String = { - val init = - s""" - |java.util.List $listName = new java.util.ArrayList(); - |""".stripMargin - - val findEventsByPatterName = if (generateCondition) { - s""" - |for ($eventTypeTerm $eventNameTerm : $contextTerm - | .getEventsForPattern("${patternFieldRef.getAlpha}")) { - | $listName.add($eventNameTerm); - |} - |""".stripMargin - } else { - s""" - |for (String $patternNameTerm : $patternNameListTerm) { - | if ($patternNameTerm.equals("${patternFieldRef.getAlpha}") || - | ${patternFieldRef.getAlpha.equals("*")}) { - | boolean skipLoop = false; - | $listTypeTerm $eventNameListTerm = - | ($listTypeTerm) $input1Term.get($patternNameTerm); - | if ($eventNameListTerm != null) { - | for ($eventTypeTerm $eventNameTerm : $eventNameListTerm) { - | $listName.add($eventNameTerm); - | if ($running && $eventNameTerm == $currEventTerm) { - | skipLoop = true; - | break; - | } - | } - | } - | - | if (skipLoop) { - | break; - | } - | } - | - | if ($running && $patternNameTerm.equals($currPatternTerm)) { - | break; - | } - |} - |""".stripMargin - } - - val getResult = - s""" - |if ($listName.size() > $count) { - | if ($first) { - | $resultTerm = ($resultTypeTerm) (($eventTypeTerm) - | $listName.get($count)) - | .getField(${patternFieldRef.getIndex}); - | } else { - | $resultTerm = ($resultTypeTerm) (($eventTypeTerm) - | $listName.get($listName.size() - $count - 1)) - | .getField(${patternFieldRef.getIndex}); - | } - | $nullTerm = false; - |} - |""".stripMargin - - s""" - |$init - |$findEventsByPatterName - |$getResult - |""".stripMargin - } - - val resultCode = - s""" - |$resultTypeTerm $resultTerm = $defaultValue; - |boolean $nullTerm = true; - |$findEventByLogicalPosition - |""".stripMargin - - GeneratedExpression(resultTerm, nullTerm, resultCode, resultType) - - case rexCall: RexCall => - val operands = rexCall.operands.asScala.map { - operand => generateFirstLast( - operand, - count, - FlinkTypeFactory.toTypeInfo(operand.getType), - running, - first) - } - - generateCall(rexCall, operands, resultType) - - case _ => - generateExpression(rexNode) - } - } - - /* - - - */ - private def generateRunningFinal( - rexNode: RexNode, - resultType: TypeInformation[_], - running: Boolean): GeneratedExpression - = { - rexNode match { - case _: RexPatternFieldRef => - generateFirstLast(rexNode, 0, resultType, running, first = false) - - case rexCall: RexCall if rexCall.getOperator == FIRST || rexCall.getOperator == LAST => - val countLiteral = rexCall.operands.get(1).asInstanceOf[RexLiteral] - val count = countLiteral.getValue3.asInstanceOf[JBigDecimal].intValue() - generateFirstLast( - rexCall.operands.get(0), - count, - resultType, - running, - rexCall.getOperator == FIRST) - - case rexCall: RexCall => - val operands = rexCall.operands.asScala.map { - operand => generateRunningFinal( - operand, - FlinkTypeFactory.toTypeInfo(operand.getType), - running) - } - - generateCall(rexCall, operands, resultType) - - case _ => - generateExpression(rexNode) - } - } -} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/flink/table/codegen/generated.scala b/core/src/main/scala/org/apache/flink/table/codegen/generated.scala deleted file mode 100644 index 3551d80ed..000000000 --- a/core/src/main/scala/org/apache/flink/table/codegen/generated.scala +++ /dev/null @@ -1,100 +0,0 @@ -package org.apache.flink.table.codegen - -import org.apache.flink.api.common.functions -import org.apache.flink.api.common.functions.Function -import org.apache.flink.api.common.io.InputFormat -import org.apache.flink.api.common.typeinfo.TypeInformation - -/** - * Describes a generated expression. - * - * @param resultTerm term to access the result of the expression - * @param nullTerm boolean term that indicates if expression is null - * @param code code necessary to produce resultTerm and nullTerm - * @param resultType type of the resultTerm - * @param literal flag to indicate a constant expression do not reference input and can thus - * be used in the member area (e.g. as constructor parameter of a reusable - * instance) - */ -case class GeneratedExpression( - resultTerm: String, - nullTerm: String, - code: String, - resultType: TypeInformation[_], - literal: Boolean = false) - -object GeneratedExpression { - val ALWAYS_NULL = "true" - val NEVER_NULL = "false" - val NO_CODE = "" -} - -/** - * Describes a generated [[functions.Function]] - * - * @param name class name of the generated Function. - * @param returnType the type information of the result type - * @param code code of the generated Function. - * @tparam F type of function - * @tparam T type of function - */ -case class GeneratedFunction[F <: Function, T <: Any]( - name: String, - returnType: TypeInformation[T], - code: String) - -/** - * Describes a generated aggregate helper function - * - * @param name class name of the generated Function. - * @param code code of the generated Function. - */ -case class GeneratedAggregationsFunction( - name: String, - code: String) - -/** - * Describes a generated [[InputFormat]]. - * - * @param name class name of the generated input function. - * @param returnType the type information of the result type - * @param code code of the generated Function. - * @tparam F type of function - * @tparam T type of function - */ -case class GeneratedInput[F <: InputFormat[_, _], T <: Any]( - name: String, - returnType: TypeInformation[T], - code: String) - -/** - * Describes a generated [[org.apache.flink.util.Collector]]. - * - * @param name class name of the generated Collector. - * @param code code of the generated Collector. - */ -case class GeneratedCollector(name: String, code: String) - -/** - * Describes a generated [[org.apache.flink.cep.pattern.conditions.IterativeCondition]]. - * - * @param name class name of the generated IterativeCondition. - * @param code code of the generated IterativeCondition. - */ -case class GeneratedIterativeCondition(name: String, code: String) - -/** - * Describes a generated [[org.apache.flink.cep.PatternSelectFunction]]. - * - * @param name class name of the generated PatternSelectFunction. - * @param code code of the generated PatternSelectFunction. - */ -case class GeneratedPatternSelectFunction(name: String, code: String) - -/** - * Describes a generated [[org.apache.flink.cep.PatternFlatSelectFunction]]. - * - * @param name class name of the generated PatternFlatSelectFunction. - * @param code code of the generated PatternFlatSelectFunction. - */ -case class GeneratedPatternFlatSelectFunction(name: String, code: String) diff --git a/core/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala b/core/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala deleted file mode 100644 index 10f60ba88..000000000 --- a/core/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamMatch.scala +++ /dev/null @@ -1,309 +0,0 @@ -package org.apache.flink.table.plan.nodes.datastream - -import java.math.{BigDecimal => JBigDecimal} -import java.util - -import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} -import org.apache.calcite.rel._ -import org.apache.calcite.rel.`type`.RelDataType -import org.apache.calcite.rex._ -import org.apache.calcite.sql.`type`.SqlTypeName._ -import org.apache.calcite.sql.fun.SqlStdOperatorTable._ -import org.apache.flink.cep.{CEP, PatternStream} -import org.apache.flink.cep.pattern.Pattern -import org.apache.flink.streaming.api.datastream.DataStream -import org.apache.flink.streaming.api.windowing.time.Time -import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment, TableException} -import org.apache.flink.table.calcite.FlinkTypeFactory -import org.apache.flink.table.plan.schema.RowSchema -import org.apache.flink.table.runtime.RowtimeProcessFunction -import org.apache.flink.table.runtime.cepmatch.{ConvertToRow, MatchUtil} -import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} -import org.apache.flink.types.Row - -import scala.collection.JavaConverters._ -import scala.collection.mutable.ListBuffer - -/** - * Flink RelNode which matches along with LogicalMatch. - */ -class DataStreamMatch( - cluster: RelOptCluster, - traitSet: RelTraitSet, - input: RelNode, - pattern: RexNode, - strictStart: Boolean, - strictEnd: Boolean, - patternDefinitions: util.Map[String, RexNode], - measures: util.Map[String, RexNode], - after: RexNode, - subsets: util.Map[String, util.SortedSet[String]], - allRows: Boolean, - partitionKeys: util.List[RexNode], - orderKeys: RelCollation, - interval: RexNode, - schema: RowSchema, - inputSchema: RowSchema) - extends SingleRel(cluster, traitSet, input) - with DataStreamRel { - - override def deriveRowType(): RelDataType = schema.relDataType - - override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = { - new DataStreamMatch( - cluster, - traitSet, - inputs.get(0), - pattern, - strictStart, - strictEnd, - patternDefinitions, - measures, - after, - subsets, - allRows, - partitionKeys, - orderKeys, - interval, - schema, - inputSchema) - } - - override def toString: String = { - s"Match(${ - if (!partitionKeys.isEmpty) { - s"PARTITION BY: ${partitionKeys.toArray.map(_.toString).mkString(", ")}, " - } else { - "" - } - }${ - if (!orderKeys.getFieldCollations.isEmpty) { - s"ORDER BY: ${orderKeys.getFieldCollations.asScala.map { - x => inputSchema.relDataType.getFieldList.get(x.getFieldIndex).getName - }.mkString(", ")}, " - } else { - "" - } - }${ - if (!measures.isEmpty) { - s"MEASURES: ${measures.asScala.map { - case (k, v) => s"${v.toString} AS $k" - }.mkString(", ")}, " - } else { - "" - } - }${ - if (allRows) { - s"ALL ROWS PER MATCH, " - } else { - s"ONE ROW PER MATCH, " - } - }${ - s"${after.toString}, " - }${ - s"PATTERN: (${pattern.toString})" - }${ - if (interval != null) { - s"WITHIN INTERVAL: $interval, " - } else { - s", " - } - }${ - if (!subsets.isEmpty) { - s"SUBSET: ${subsets.asScala.map { - case (k, v) => s"$k = (${v.toArray.mkString(", ")})" - }.mkString(", ")}, " - } else { - "" - } - }${ - s"DEFINE: ${patternDefinitions.asScala.map { - case (k, v) => s"$k AS ${v.toString}" - }.mkString(", ")}" - })" - } - - override def explainTerms(pw: RelWriter): RelWriter = { - pw.input("input", getInput()) - .itemIf("partitionBy", - partitionKeys.toArray.map(_.toString).mkString(", "), - !partitionKeys.isEmpty) - .itemIf("orderBy", - orderKeys.getFieldCollations.asScala.map { - x => inputSchema.relDataType.getFieldList.get(x.getFieldIndex).getName - }.mkString(", "), - !orderKeys.getFieldCollations.isEmpty) - .itemIf("measures", - measures.asScala.map { case (k, v) => s"${v.toString} AS $k"}.mkString(", "), - !measures.isEmpty) - .item("allrows", allRows) - .item("after", after.toString) - .item("pattern", pattern.toString) - .itemIf("within interval", - if (interval != null) { - interval.toString - } else { - null - }, - interval != null) - .itemIf("subset", - subsets.asScala.map { case (k, v) => s"$k = (${v.toArray.mkString(", ")})"}.mkString(", "), - !subsets.isEmpty) - .item("define", - patternDefinitions.asScala.map { case (k, v) => s"$k AS ${v.toString}"}.mkString(", ")) - } - - override def translateToPlan( - tableEnv: StreamTableEnvironment, - queryConfig: StreamQueryConfig): DataStream[CRow] = { - - val config = tableEnv.config - val inputTypeInfo = inputSchema.typeInfo - - val crowInput: DataStream[CRow] = getInput - .asInstanceOf[DataStreamRel] - .translateToPlan(tableEnv, queryConfig) - - val rowtimeFields = inputSchema.relDataType - .getFieldList.asScala - .filter(f => FlinkTypeFactory.isRowtimeIndicatorType(f.getType)) - - //主要是针对rowtime做处理,对存在rowtime的将其复制到StreamRecord的时间戳字段内 - val timestampedInput = if (rowtimeFields.nonEmpty) { - // copy the rowtime field into the StreamRecord timestamp field - val timeIdx = rowtimeFields.head.getIndex - - crowInput - .process(new RowtimeProcessFunction(timeIdx, CRowTypeInfo(inputTypeInfo))) - .setParallelism(crowInput.getParallelism) - .name(s"rowtime field: (${rowtimeFields.head})") - } else { - crowInput - } - - val inputDS: DataStream[Row] = timestampedInput - .map(new ConvertToRow) - .setParallelism(timestampedInput.getParallelism) - .name("ConvertToRow") - .returns(inputTypeInfo) - - def translatePattern( - rexNode: RexNode, - currentPattern: Pattern[Row, Row], - patternNames: ListBuffer[String]): Pattern[Row, Row] = rexNode match { - //Conditions - case literal: RexLiteral => - val patternName = literal.getValue3.toString - patternNames += patternName - val newPattern = next(currentPattern, patternName) - - val patternDefinition = patternDefinitions.get(patternName) - if (patternDefinition != null) { - val condition = MatchUtil.generateIterativeCondition( - config, - inputSchema, - patternName, - patternNames, - patternDefinition, - inputTypeInfo) - - newPattern.where(condition) - } else { - newPattern - } - - case call: RexCall => - - call.getOperator match { - case PATTERN_CONCAT => - val left = call.operands.get(0) - val right = call.operands.get(1) - translatePattern(right, - translatePattern(left, currentPattern, patternNames), - patternNames) - // Quantifiers - case PATTERN_QUANTIFIER => - val name = call.operands.get(0).asInstanceOf[RexLiteral] - val newPattern = translatePattern(name, currentPattern, patternNames) - - val startNum = call.operands.get(1).asInstanceOf[RexLiteral] - .getValue3.asInstanceOf[JBigDecimal].intValue() - val endNum = call.operands.get(2).asInstanceOf[RexLiteral] - .getValue3.asInstanceOf[JBigDecimal].intValue() - - if (startNum == 0 && endNum == -1) { // zero or more - newPattern.oneOrMore().optional().consecutive() - } else if (startNum == 1 && endNum == -1) { // one or more - newPattern.oneOrMore().consecutive() - } else if (startNum == 0 && endNum == 1) { // optional - newPattern.optional() - } else if (endNum != -1) { // times - newPattern.times(startNum, endNum).consecutive() - } else { // times or more - newPattern.timesOrMore(startNum).consecutive() - } - - case PATTERN_ALTER => - throw TableException("Currently, CEP doesn't support branching patterns.") - - case PATTERN_PERMUTE => - throw TableException("Currently, CEP doesn't support PERMUTE patterns.") - - case PATTERN_EXCLUDE => - throw TableException("Currently, CEP doesn't support '{-' '-}' patterns.") - } - - case _ => - throw TableException("") - } - - val patternNames: ListBuffer[String] = ListBuffer() - val cepPattern = translatePattern(pattern, null, patternNames) - if (interval != null) { - val intervalLiteral = interval.asInstanceOf[RexLiteral] - val intervalValue = interval.asInstanceOf[RexLiteral].getValueAs(classOf[java.lang.Long]) - val intervalMs: Long = intervalLiteral.getTypeName match { - case INTERVAL_YEAR | INTERVAL_YEAR_MONTH | INTERVAL_MONTH => - // convert from months to milliseconds, suppose 1 month = 30 days - intervalValue * 30L * 24 * 3600 * 1000 - case _ => intervalValue - } - - cepPattern.within(Time.milliseconds(intervalMs)) - } - - val patternStream: PatternStream[Row] = CEP.pattern[Row](inputDS, cepPattern) - - val outTypeInfo = CRowTypeInfo(schema.typeInfo) - if (allRows) { - val patternFlatSelectFunction = - MatchUtil.generatePatternFlatSelectFunction( - config, - schema, - patternNames, - partitionKeys, - orderKeys, - measures, - inputTypeInfo) - patternStream.flatSelect[CRow](patternFlatSelectFunction, outTypeInfo) - } else { - val patternSelectFunction = - MatchUtil.generatePatternSelectFunction( - config, - schema, - patternNames, - partitionKeys, - measures, - inputTypeInfo) - patternStream.select[CRow](patternSelectFunction, outTypeInfo) - } - } - - private def next(currentPattern: Pattern[Row, Row], patternName: String): Pattern[Row, Row] = { - if (currentPattern == null) { - Pattern.begin(patternName) - } else { - currentPattern.next(patternName) - } - } -} diff --git a/core/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala b/core/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala deleted file mode 100644 index 693e4d314..000000000 --- a/core/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalMatch.scala +++ /dev/null @@ -1,115 +0,0 @@ -package org.apache.flink.table.plan.nodes.logical - -import java.util - -import org.apache.calcite.plan._ -import org.apache.calcite.rel.`type`.RelDataType -import org.apache.calcite.rel.convert.ConverterRule -import org.apache.calcite.rel.core.Match -import org.apache.calcite.rel.logical.LogicalMatch -import org.apache.calcite.rel.{RelCollation, RelNode} -import org.apache.calcite.rex.RexNode -import org.apache.flink.table.plan.nodes.FlinkConventions - - -class FlinkLogicalMatch( - cluster: RelOptCluster, - traitSet: RelTraitSet, - input: RelNode, - rowType: RelDataType, - pattern: RexNode, - strictStart: Boolean, - strictEnd: Boolean, - patternDefinitions: util.Map[String, RexNode], - measures: util.Map[String, RexNode], - after: RexNode, - subsets: util.Map[String, _ <: util.SortedSet[String]], - allRows: Boolean, - partitionKeys: util.List[RexNode], - orderKeys: RelCollation, - interval: RexNode) - extends Match( - cluster, - traitSet, - input, - rowType, - pattern, - strictStart, - strictEnd, - patternDefinitions, - measures, - after, - subsets, - allRows, - partitionKeys, - orderKeys, - interval) - with FlinkLogicalRel { - - override def copy( - input: RelNode, - rowType: RelDataType, - pattern: RexNode, - strictStart: Boolean, - strictEnd: Boolean, - patternDefinitions: util.Map[String, RexNode], - measures: util.Map[String, RexNode], - after: RexNode, - subsets: util.Map[String, _ <: util.SortedSet[String]], - allRows: Boolean, - partitionKeys: util.List[RexNode], - orderKeys: RelCollation, - interval: RexNode): Match = { - new FlinkLogicalMatch( - cluster, - traitSet, - input, - rowType, - pattern, - strictStart, - strictEnd, - patternDefinitions, - measures, - after, - subsets, - allRows, - partitionKeys, - orderKeys, - interval) - } -} - -private class FlinkLogicalMatchConverter - extends ConverterRule( - classOf[LogicalMatch], - Convention.NONE, - FlinkConventions.LOGICAL, - "FlinkLogicalMatchConverter") { - - override def convert(rel: RelNode): RelNode = { - val logicalMatch = rel.asInstanceOf[LogicalMatch] - val traitSet = rel.getTraitSet.replace(FlinkConventions.LOGICAL) - val newInput = RelOptRule.convert(logicalMatch.getInput, FlinkConventions.LOGICAL) - - new FlinkLogicalMatch( - rel.getCluster, - traitSet, - newInput, - logicalMatch.getRowType, - logicalMatch.getPattern, - logicalMatch.isStrictStart, - logicalMatch.isStrictEnd, - logicalMatch.getPatternDefinitions, - logicalMatch.getMeasures, - logicalMatch.getAfter, - logicalMatch.getSubsets, - logicalMatch.isAllRows, - logicalMatch.getPartitionKeys, - logicalMatch.getOrderKeys, - logicalMatch.getInterval) - } -} - -object FlinkLogicalMatch { - val CONVERTER: ConverterRule = new FlinkLogicalMatchConverter() -} diff --git a/core/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/core/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala deleted file mode 100644 index 5a0ff9d65..000000000 --- a/core/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala +++ /dev/null @@ -1,209 +0,0 @@ -package org.apache.flink.table.plan.rules - -import org.apache.calcite.rel.core.RelFactories -import org.apache.calcite.rel.rules._ -import org.apache.calcite.tools.{RuleSet, RuleSets} -import org.apache.flink.table.plan.rules.common._ -import org.apache.flink.table.plan.rules.logical._ -import org.apache.flink.table.plan.rules.dataSet._ -import org.apache.flink.table.plan.rules.datastream.{DataStreamMatchRule, _} -import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalMatch, _} - -object FlinkRuleSets { - - /** - * Convert sub-queries before query decorrelation. - */ - val TABLE_SUBQUERY_RULES: RuleSet = RuleSets.ofList( - SubQueryRemoveRule.FILTER, - SubQueryRemoveRule.PROJECT, - SubQueryRemoveRule.JOIN) - - /** - * Convert table references before query decorrelation. - */ - val TABLE_REF_RULES: RuleSet = RuleSets.ofList( - TableScanRule.INSTANCE, - EnumerableToLogicalTableScan.INSTANCE) - - val LOGICAL_OPT_RULES: RuleSet = RuleSets.ofList( - - // push a filter into a join - FilterJoinRule.FILTER_ON_JOIN, - // push filter into the children of a join - FilterJoinRule.JOIN, - // push filter through an aggregation - FilterAggregateTransposeRule.INSTANCE, - // push filter through set operation - FilterSetOpTransposeRule.INSTANCE, - // push project through set operation - ProjectSetOpTransposeRule.INSTANCE, - - // aggregation and projection rules - AggregateProjectMergeRule.INSTANCE, - AggregateProjectPullUpConstantsRule.INSTANCE, - // push a projection past a filter or vice versa - ProjectFilterTransposeRule.INSTANCE, - FilterProjectTransposeRule.INSTANCE, - // push a projection to the children of a join - // push all expressions to handle the time indicator correctly - new ProjectJoinTransposeRule(PushProjector.ExprCondition.FALSE, RelFactories.LOGICAL_BUILDER), - // merge projections - ProjectMergeRule.INSTANCE, - // remove identity project - ProjectRemoveRule.INSTANCE, - // reorder sort and projection - SortProjectTransposeRule.INSTANCE, - ProjectSortTransposeRule.INSTANCE, - - // join rules - JoinPushExpressionsRule.INSTANCE, - - // remove union with only a single child - UnionEliminatorRule.INSTANCE, - // convert non-all union into all-union + distinct - UnionToDistinctRule.INSTANCE, - - // remove aggregation if it does not aggregate and input is already distinct - AggregateRemoveRule.INSTANCE, - // push aggregate through join - AggregateJoinTransposeRule.EXTENDED, - // aggregate union rule - AggregateUnionAggregateRule.INSTANCE, - // expand distinct aggregate to normal aggregate with groupby - AggregateExpandDistinctAggregatesRule.JOIN, - - // reduce aggregate functions like AVG, STDDEV_POP etc. - AggregateReduceFunctionsRule.INSTANCE, - - // remove unnecessary sort rule - SortRemoveRule.INSTANCE, - - // prune empty results rules - PruneEmptyRules.AGGREGATE_INSTANCE, - PruneEmptyRules.FILTER_INSTANCE, - PruneEmptyRules.JOIN_LEFT_INSTANCE, - PruneEmptyRules.JOIN_RIGHT_INSTANCE, - PruneEmptyRules.PROJECT_INSTANCE, - PruneEmptyRules.SORT_INSTANCE, - PruneEmptyRules.UNION_INSTANCE, - - // calc rules - FilterCalcMergeRule.INSTANCE, - ProjectCalcMergeRule.INSTANCE, - FilterToCalcRule.INSTANCE, - ProjectToCalcRule.INSTANCE, - CalcMergeRule.INSTANCE, - - // scan optimization - PushProjectIntoTableSourceScanRule.INSTANCE, - PushFilterIntoTableSourceScanRule.INSTANCE, - - // unnest rule - LogicalUnnestRule.INSTANCE, - - // translate to flink logical rel nodes - FlinkLogicalAggregate.CONVERTER, - FlinkLogicalWindowAggregate.CONVERTER, - FlinkLogicalOverWindow.CONVERTER, - FlinkLogicalCalc.CONVERTER, - FlinkLogicalCorrelate.CONVERTER, - FlinkLogicalIntersect.CONVERTER, - FlinkLogicalJoin.CONVERTER, - FlinkLogicalMinus.CONVERTER, - FlinkLogicalSort.CONVERTER, - FlinkLogicalUnion.CONVERTER, - FlinkLogicalValues.CONVERTER, - FlinkLogicalTableSourceScan.CONVERTER, - FlinkLogicalTableFunctionScan.CONVERTER, - FlinkLogicalNativeTableScan.CONVERTER, - FlinkLogicalMatch.CONVERTER - ) - - - /** - * RuleSet to normalize plans for batch / DataSet execution - */ - val DATASET_NORM_RULES: RuleSet = RuleSets.ofList( - // simplify expressions rules - ReduceExpressionsRule.FILTER_INSTANCE, - ReduceExpressionsRule.PROJECT_INSTANCE, - ReduceExpressionsRule.CALC_INSTANCE, - ReduceExpressionsRule.JOIN_INSTANCE, - ProjectToWindowRule.PROJECT, - - // Transform grouping sets - DecomposeGroupingSetRule.INSTANCE, - // Transform window to LogicalWindowAggregate - DataSetLogicalWindowAggregateRule.INSTANCE, - WindowPropertiesRule.INSTANCE, - WindowPropertiesHavingRule.INSTANCE - ) - - /** - * RuleSet to optimize plans for batch / DataSet execution - */ - val DATASET_OPT_RULES: RuleSet = RuleSets.ofList( - // translate to Flink DataSet nodes - DataSetWindowAggregateRule.INSTANCE, - DataSetAggregateRule.INSTANCE, - DataSetDistinctRule.INSTANCE, - DataSetCalcRule.INSTANCE, - DataSetJoinRule.INSTANCE, - DataSetSingleRowJoinRule.INSTANCE, - DataSetScanRule.INSTANCE, - DataSetUnionRule.INSTANCE, - DataSetIntersectRule.INSTANCE, - DataSetMinusRule.INSTANCE, - DataSetSortRule.INSTANCE, - DataSetValuesRule.INSTANCE, - DataSetCorrelateRule.INSTANCE, - BatchTableSourceScanRule.INSTANCE - ) - - /** - * RuleSet to normalize plans for stream / DataStream execution - */ - val DATASTREAM_NORM_RULES: RuleSet = RuleSets.ofList( - // Transform window to LogicalWindowAggregate - DataStreamLogicalWindowAggregateRule.INSTANCE, - WindowPropertiesRule.INSTANCE, - WindowPropertiesHavingRule.INSTANCE, - - // simplify expressions rules - ReduceExpressionsRule.FILTER_INSTANCE, - ReduceExpressionsRule.PROJECT_INSTANCE, - ReduceExpressionsRule.CALC_INSTANCE, - ProjectToWindowRule.PROJECT - ) - - /** - * RuleSet to optimize plans for stream / DataStream execution - */ - val DATASTREAM_OPT_RULES: RuleSet = RuleSets.ofList( - // translate to DataStream nodes - DataStreamSortRule.INSTANCE, - DataStreamGroupAggregateRule.INSTANCE, - DataStreamOverAggregateRule.INSTANCE, - DataStreamGroupWindowAggregateRule.INSTANCE, - DataStreamCalcRule.INSTANCE, - DataStreamScanRule.INSTANCE, - DataStreamUnionRule.INSTANCE, - DataStreamValuesRule.INSTANCE, - DataStreamCorrelateRule.INSTANCE, - DataStreamWindowJoinRule.INSTANCE, - StreamTableSourceScanRule.INSTANCE, - DataStreamMatchRule.INSTANCE - ) - - /** - * RuleSet to decorate plans for stream / DataStream execution - */ - val DATASTREAM_DECO_RULES: RuleSet = RuleSets.ofList( - // retraction rules - DataStreamRetractionRules.DEFAULT_RETRACTION_INSTANCE, - DataStreamRetractionRules.UPDATES_AS_RETRACTION_INSTANCE, - DataStreamRetractionRules.ACCMODE_INSTANCE - ) - -} diff --git a/core/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala b/core/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala deleted file mode 100644 index 8bb052504..000000000 --- a/core/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamMatchRule.scala +++ /dev/null @@ -1,46 +0,0 @@ -package org.apache.flink.table.plan.rules.datastream - -import org.apache.calcite.plan.{RelOptRule, RelTraitSet} -import org.apache.calcite.rel.RelNode -import org.apache.calcite.rel.convert.ConverterRule -import org.apache.flink.table.plan.nodes.FlinkConventions -import org.apache.flink.table.plan.nodes.datastream.DataStreamMatch -import org.apache.flink.table.plan.nodes.logical.FlinkLogicalMatch -import org.apache.flink.table.plan.schema.RowSchema - -class DataStreamMatchRule - extends ConverterRule( - classOf[FlinkLogicalMatch], - FlinkConventions.LOGICAL, - FlinkConventions.DATASTREAM, - "DataStreamMatchRule") { - - override def convert(rel: RelNode): RelNode = { - val logicalMatch: FlinkLogicalMatch = rel.asInstanceOf[FlinkLogicalMatch] - val traitSet: RelTraitSet = rel.getTraitSet.replace(FlinkConventions.DATASTREAM) - val convertInput: RelNode = - RelOptRule.convert(logicalMatch.getInput, FlinkConventions.DATASTREAM) - - new DataStreamMatch( - rel.getCluster, - traitSet, - convertInput, - logicalMatch.getPattern, - logicalMatch.isStrictStart, - logicalMatch.isStrictEnd, - logicalMatch.getPatternDefinitions, - logicalMatch.getMeasures, - logicalMatch.getAfter, - logicalMatch.getSubsets, - logicalMatch.isAllRows, - logicalMatch.getPartitionKeys, - logicalMatch.getOrderKeys, - logicalMatch.getInterval, - new RowSchema(logicalMatch.getRowType), - new RowSchema(logicalMatch.getInput.getRowType)) - } -} - -object DataStreamMatchRule { - val INSTANCE: RelOptRule = new DataStreamMatchRule -} diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/ConvertToRow.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/ConvertToRow.scala deleted file mode 100644 index cfaf5fe81..000000000 --- a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/ConvertToRow.scala +++ /dev/null @@ -1,14 +0,0 @@ -package org.apache.flink.table.runtime.cepmatch - -import org.apache.flink.api.common.functions.MapFunction -import org.apache.flink.table.runtime.types.CRow -import org.apache.flink.types.Row - -/** - * MapFunction convert CRow to Row. - */ -class ConvertToRow extends MapFunction[CRow, Row] { - override def map(value: CRow): Row = { - value.row - } -} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/IterativeConditionRunner.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/IterativeConditionRunner.scala deleted file mode 100644 index 04a365ebb..000000000 --- a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/IterativeConditionRunner.scala +++ /dev/null @@ -1,40 +0,0 @@ -package org.apache.flink.table.runtime.cepmatch - -import org.apache.flink.cep.pattern.conditions.IterativeCondition -import org.apache.flink.table.codegen.Compiler -import org.apache.flink.types.Row -import org.slf4j.LoggerFactory - -/** - * IterativeConditionRunner with [[Row]] value. - */ -class IterativeConditionRunner( - name: String, - code: String) - extends IterativeCondition[Row] - with Compiler[IterativeCondition[Row]]{ - - val LOG = LoggerFactory.getLogger(this.getClass) - - // IterativeCondition will be serialized as part of state, - // so make function as transient to avoid ClassNotFoundException when restore state, - // see FLINK-6939 for details - @transient private var function: IterativeCondition[Row] = _ - - def init(): Unit = { - LOG.debug(s"Compiling IterativeCondition: $name \n\n Code:\n$code") - // We cannot get user's classloader currently, see FLINK-6938 for details - val clazz = compile(Thread.currentThread().getContextClassLoader, name, code) - LOG.debug("Instantiating IterativeCondition.") - function = clazz.newInstance() - } - - override def filter(value: Row, ctx: IterativeCondition.Context[Row]): Boolean = { - - if (function == null) { - init() - } - - function.filter(value, ctx) - } -} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/MatchUtil.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/MatchUtil.scala deleted file mode 100644 index e0af24e93..000000000 --- a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/MatchUtil.scala +++ /dev/null @@ -1,99 +0,0 @@ -package org.apache.flink.table.runtime.cepmatch - -import java.util - -import org.apache.calcite.rel.RelCollation -import org.apache.calcite.rex.RexNode -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.cep.{PatternFlatSelectFunction, PatternSelectFunction} -import org.apache.flink.cep.pattern.conditions.IterativeCondition -import org.apache.flink.table.api.TableConfig -import org.apache.flink.table.codegen.MatchCodeGenerator -import org.apache.flink.table.plan.schema.RowSchema -import org.apache.flink.table.runtime.types.CRow -import org.apache.flink.types.Row - -/** - * An util class to generate match functions. - * 1。IterativeCondition - * 2。PatternSelectFunction - * 3。PatternFlatSelectFunction - */ -object MatchUtil { - - private[flink] def generateIterativeCondition( - config: TableConfig, - inputType: RowSchema, - patternName: String, - patternNames: Seq[String], - patternDefinition: RexNode, - inputTypeInfo: TypeInformation[_]): IterativeCondition[Row] = { - - val generator = new MatchCodeGenerator( - config, false, inputTypeInfo, patternNames, true, Some(patternName)) - val condition = generator.generateExpression(patternDefinition) - val body = - s""" - |${condition.code} - |return ${condition.resultTerm}; - |""".stripMargin - - val genCondition = generator.generateIterativeCondition("MatchRecognizeCondition", body) - new IterativeConditionRunner(genCondition.name, genCondition.code) - } - - private[flink] def generatePatternSelectFunction( - config: TableConfig, - returnType: RowSchema, - patternNames: Seq[String], - partitionKeys: util.List[RexNode], - measures: util.Map[String, RexNode], - inputTypeInfo: TypeInformation[_]): PatternSelectFunction[Row, CRow] = { - - val generator = new MatchCodeGenerator(config, false, inputTypeInfo, patternNames, false) - - val resultExpression = generator.generateSelectOutputExpression( - partitionKeys, - measures, - returnType) - val body = - s""" - |${resultExpression.code} - |return ${resultExpression.resultTerm}; - |""".stripMargin - - generator.addReusableStatements() - val genFunction = generator.generatePatternSelectFunction( - "MatchRecognizePatternSelectFunction", - body) - new PatternSelectFunctionRunner(genFunction.name, genFunction.code) - } - - private[flink] def generatePatternFlatSelectFunction( - config: TableConfig, - returnType: RowSchema, - patternNames: Seq[String], - partitionKeys: util.List[RexNode], - orderKeys: RelCollation, - measures: util.Map[String, RexNode], - inputTypeInfo: TypeInformation[_]): PatternFlatSelectFunction[Row, CRow] = { - - val generator = new MatchCodeGenerator(config, false, inputTypeInfo, patternNames, false) - - val resultExpression = generator.generateFlatSelectOutputExpression( - partitionKeys, - orderKeys, - measures, - returnType) - val body = - s""" - |${resultExpression.code} - |""".stripMargin - - generator.addReusableStatements() - val genFunction = generator.generatePatternFlatSelectFunction( - "MatchRecognizePatternFlatSelectFunction", - body) - new PatternFlatSelectFunctionRunner(genFunction.name, genFunction.code) - } -} diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternFlatSelectFunctionRunner.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternFlatSelectFunctionRunner.scala deleted file mode 100644 index f72296324..000000000 --- a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternFlatSelectFunctionRunner.scala +++ /dev/null @@ -1,47 +0,0 @@ -package org.apache.flink.table.runtime.cepmatch - -import java.util - -import org.apache.flink.cep.PatternFlatSelectFunction -import org.apache.flink.table.codegen.Compiler -import org.apache.flink.table.runtime.CRowWrappingCollector -import org.apache.flink.table.runtime.types.CRow -import org.apache.flink.types.Row -import org.apache.flink.util.Collector -import org.slf4j.LoggerFactory - -/** - * PatternFlatSelectFunctionRunner with [[Row]] input and [[CRow]] output. - */ -class PatternFlatSelectFunctionRunner( - name: String, - code: String) - extends PatternFlatSelectFunction[Row, CRow] - with Compiler[PatternFlatSelectFunction[Row, Row]] { - - val LOG = LoggerFactory.getLogger(this.getClass) - - private var cRowWrapper: CRowWrappingCollector = _ - - private var function: PatternFlatSelectFunction[Row, Row] = _ - - def init(): Unit = { - LOG.debug(s"Compiling PatternFlatSelectFunction: $name \n\n Code:\n$code") - val clazz = compile(Thread.currentThread().getContextClassLoader, name, code) - LOG.debug("Instantiating PatternFlatSelectFunction.") - function = clazz.newInstance() - - this.cRowWrapper = new CRowWrappingCollector() - } - - override def flatSelect( - pattern: util.Map[String, util.List[Row]], - out: Collector[CRow]): Unit = { - if (function == null) { - init() - } - - cRowWrapper.out = out - function.flatSelect(pattern, cRowWrapper) - } -} diff --git a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternSelectFunctionRunner.scala b/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternSelectFunctionRunner.scala deleted file mode 100644 index d84c9e61d..000000000 --- a/core/src/main/scala/org/apache/flink/table/runtime/cepmatch/PatternSelectFunctionRunner.scala +++ /dev/null @@ -1,45 +0,0 @@ -package org.apache.flink.table.runtime.cepmatch - -import java.util - -import org.apache.flink.cep.PatternSelectFunction -import org.apache.flink.table.codegen.Compiler -import org.apache.flink.table.runtime.types.CRow -import org.apache.flink.types.Row -import org.slf4j.LoggerFactory - -/** - * PatternSelectFunctionRunner with [[Row]] input and [[CRow]] output. - */ -class PatternSelectFunctionRunner( - name: String, - code: String) - extends PatternSelectFunction[Row, CRow] - with Compiler[PatternSelectFunction[Row, Row]] { - - val LOG = LoggerFactory.getLogger(this.getClass) - - private var outCRow: CRow = _ - - private var function: PatternSelectFunction[Row, Row] = _ - - def init(): Unit = { - LOG.debug(s"Compiling PatternSelectFunction: $name \n\n Code:\n$code") - val clazz = compile(Thread.currentThread().getContextClassLoader, name, code) - LOG.debug("Instantiating PatternSelectFunction.") - function = clazz.newInstance() - } - - override def select(pattern: util.Map[String, util.List[Row]]): CRow = { - if (outCRow == null) { - outCRow = new CRow(null, true) - } - - if (function == null) { - init() - } - - outCRow.row = function.select(pattern) - outCRow - } -} diff --git a/core/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala b/core/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala deleted file mode 100644 index f89c41d71..000000000 --- a/core/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala +++ /dev/null @@ -1,534 +0,0 @@ -package org.apache.flink.table.validate - -import org.apache.calcite.sql.`type`.{OperandTypes, ReturnTypes, SqlTypeTransforms} -import org.apache.calcite.sql.fun.SqlStdOperatorTable -import org.apache.calcite.sql.util.{ChainedSqlOperatorTable, ListSqlOperatorTable, ReflectiveSqlOperatorTable} -import org.apache.calcite.sql._ -import org.apache.flink.table.api._ -import org.apache.flink.table.expressions._ -import org.apache.flink.table.functions.sql.ScalarSqlFunctions -import org.apache.flink.table.functions.utils.{AggSqlFunction, ScalarSqlFunction, TableSqlFunction} -import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction} - -import _root_.scala.collection.JavaConversions._ -import _root_.scala.collection.mutable -import _root_.scala.util.{Failure, Success, Try} - -/** - * A catalog for looking up (user-defined) functions, used during validation phases - * of both Table API and SQL API. - */ -class FunctionCatalog { - - private val functionBuilders = mutable.HashMap.empty[String, Class[_]] - private val sqlFunctions = mutable.ListBuffer[SqlFunction]() - - def registerFunction(name: String, builder: Class[_]): Unit = - functionBuilders.put(name.toLowerCase, builder) - - def registerSqlFunction(sqlFunction: SqlFunction): Unit = { - sqlFunctions --= sqlFunctions.filter(_.getName == sqlFunction.getName) - sqlFunctions += sqlFunction - } - - def getUserDefinedFunctions: Seq[String] = { - sqlFunctions.map(_.getName) - } - - def getSqlOperatorTable: SqlOperatorTable = - ChainedSqlOperatorTable.of( - new BasicOperatorTable(), - new ListSqlOperatorTable(sqlFunctions) - ) - - /** - * Lookup and create an expression if we find a match. - */ - def lookupFunction(name: String, children: Seq[Expression]): Expression = { - val funcClass = functionBuilders - .getOrElse(name.toLowerCase, throw ValidationException(s"Undefined function: $name")) - - // Instantiate a function using the provided `children` - funcClass match { - - // user-defined scalar function call - case sf if classOf[ScalarFunction].isAssignableFrom(sf) => - val scalarSqlFunction = sqlFunctions - .find(f => f.getName.equalsIgnoreCase(name) && f.isInstanceOf[ScalarSqlFunction]) - .getOrElse(throw ValidationException(s"Undefined scalar function: $name")) - .asInstanceOf[ScalarSqlFunction] - ScalarFunctionCall(scalarSqlFunction.getScalarFunction, children) - - // user-defined table function call - case tf if classOf[TableFunction[_]].isAssignableFrom(tf) => - val tableSqlFunction = sqlFunctions - .find(f => f.getName.equalsIgnoreCase(name) && f.isInstanceOf[TableSqlFunction]) - .getOrElse(throw ValidationException(s"Undefined table function: $name")) - .asInstanceOf[TableSqlFunction] - val typeInfo = tableSqlFunction.getRowTypeInfo - val function = tableSqlFunction.getTableFunction - TableFunctionCall(name, function, children, typeInfo) - - // user-defined aggregate function call - case af if classOf[AggregateFunction[_, _]].isAssignableFrom(af) => - val aggregateFunction = sqlFunctions - .find(f => f.getName.equalsIgnoreCase(name) && f.isInstanceOf[AggSqlFunction]) - .getOrElse(throw ValidationException(s"Undefined table function: $name")) - .asInstanceOf[AggSqlFunction] - val function = aggregateFunction.getFunction - val returnType = aggregateFunction.returnType - val accType = aggregateFunction.accType - AggFunctionCall(function, returnType, accType, children) - - // general expression call - case expression if classOf[Expression].isAssignableFrom(expression) => - // try to find a constructor accepts `Seq[Expression]` - Try(funcClass.getDeclaredConstructor(classOf[Seq[_]])) match { - case Success(seqCtor) => - Try(seqCtor.newInstance(children).asInstanceOf[Expression]) match { - case Success(expr) => expr - case Failure(e) => throw new ValidationException(e.getMessage) - } - case Failure(_) => - Try(funcClass.getDeclaredConstructor(classOf[Expression], classOf[Seq[_]])) match { - case Success(ctor) => - Try(ctor.newInstance(children.head, children.tail).asInstanceOf[Expression]) match { - case Success(expr) => expr - case Failure(e) => throw new ValidationException(e.getMessage) - } - case Failure(_) => - val childrenClass = Seq.fill(children.length)(classOf[Expression]) - // try to find a constructor matching the exact number of children - Try(funcClass.getDeclaredConstructor(childrenClass: _*)) match { - case Success(ctor) => - Try(ctor.newInstance(children: _*).asInstanceOf[Expression]) match { - case Success(expr) => expr - case Failure(exception) => throw ValidationException(exception.getMessage) - } - case Failure(_) => - throw ValidationException( - s"Invalid number of arguments for function $funcClass") - } - } - } - case _ => - throw ValidationException("Unsupported function.") - } - } - - /** - * Drop a function and return if the function existed. - */ - def dropFunction(name: String): Boolean = - functionBuilders.remove(name.toLowerCase).isDefined - - /** - * Drop all registered functions. - */ - def clear(): Unit = functionBuilders.clear() -} - -object FunctionCatalog { - - val builtInFunctions: Map[String, Class[_]] = Map( - - // logic - "and" -> classOf[And], - "or" -> classOf[Or], - "not" -> classOf[Not], - "equals" -> classOf[EqualTo], - "greaterThan" -> classOf[GreaterThan], - "greaterThanOrEqual" -> classOf[GreaterThanOrEqual], - "lessThan" -> classOf[LessThan], - "lessThanOrEqual" -> classOf[LessThanOrEqual], - "notEquals" -> classOf[NotEqualTo], - "in" -> classOf[In], - "isNull" -> classOf[IsNull], - "isNotNull" -> classOf[IsNotNull], - "isTrue" -> classOf[IsTrue], - "isFalse" -> classOf[IsFalse], - "isNotTrue" -> classOf[IsNotTrue], - "isNotFalse" -> classOf[IsNotFalse], - "if" -> classOf[If], - - // aggregate functions - "avg" -> classOf[Avg], - "count" -> classOf[Count], - "max" -> classOf[Max], - "min" -> classOf[Min], - "sum" -> classOf[Sum], - "sum0" -> classOf[Sum0], - "stddevPop" -> classOf[StddevPop], - "stddevSamp" -> classOf[StddevSamp], - "varPop" -> classOf[VarPop], - "varSamp" -> classOf[VarSamp], - "collect" -> classOf[Collect], - - // string functions - "charLength" -> classOf[CharLength], - "initCap" -> classOf[InitCap], - "like" -> classOf[Like], - "concat" -> classOf[Plus], - "lower" -> classOf[Lower], - "lowerCase" -> classOf[Lower], - "similar" -> classOf[Similar], - "substring" -> classOf[Substring], - "trim" -> classOf[Trim], - "upper" -> classOf[Upper], - "upperCase" -> classOf[Upper], - "position" -> classOf[Position], - "overlay" -> classOf[Overlay], - "concat" -> classOf[Concat], - "concat_ws" -> classOf[ConcatWs], - "lpad" -> classOf[Lpad], - "rpad" -> classOf[Rpad], - - // math functions - "plus" -> classOf[Plus], - "minus" -> classOf[Minus], - "divide" -> classOf[Div], - "times" -> classOf[Mul], - "abs" -> classOf[Abs], - "ceil" -> classOf[Ceil], - "exp" -> classOf[Exp], - "floor" -> classOf[Floor], - "log10" -> classOf[Log10], - "ln" -> classOf[Ln], - "power" -> classOf[Power], - "mod" -> classOf[Mod], - "sqrt" -> classOf[Sqrt], - "minusPrefix" -> classOf[UnaryMinus], - "sin" -> classOf[Sin], - "cos" -> classOf[Cos], - "tan" -> classOf[Tan], - "cot" -> classOf[Cot], - "asin" -> classOf[Asin], - "acos" -> classOf[Acos], - "atan" -> classOf[Atan], - "degrees" -> classOf[Degrees], - "radians" -> classOf[Radians], - "sign" -> classOf[Sign], - "round" -> classOf[Round], - "pi" -> classOf[Pi], - "e" -> classOf[E], - "rand" -> classOf[Rand], - "randInteger" -> classOf[RandInteger], - "bin" -> classOf[Bin], - - // temporal functions - "extract" -> classOf[Extract], - "currentDate" -> classOf[CurrentDate], - "currentTime" -> classOf[CurrentTime], - "currentTimestamp" -> classOf[CurrentTimestamp], - "localTime" -> classOf[LocalTime], - "localTimestamp" -> classOf[LocalTimestamp], - "quarter" -> classOf[Quarter], - "temporalOverlaps" -> classOf[TemporalOverlaps], - "dateTimePlus" -> classOf[Plus], - "dateFormat" -> classOf[DateFormat], - - // item - "at" -> classOf[ItemAt], - - // cardinality - "cardinality" -> classOf[Cardinality], - - // array - "array" -> classOf[ArrayConstructor], - "element" -> classOf[ArrayElement], - - // map - "map" -> classOf[MapConstructor], - - // row - "row" -> classOf[RowConstructor], - - // window properties - "start" -> classOf[WindowStart], - "end" -> classOf[WindowEnd], - - // ordering - "asc" -> classOf[Asc], - "desc" -> classOf[Desc], - - // crypto hash - "md5" -> classOf[Md5], - "sha1" -> classOf[Sha1], - "sha256" -> classOf[Sha256] - ) - - /** - * Create a new function catalog with built-in functions. - */ - def withBuiltIns: FunctionCatalog = { - val catalog = new FunctionCatalog() - builtInFunctions.foreach { case (n, c) => catalog.registerFunction(n, c) } - catalog - } -} - -class BasicOperatorTable extends ReflectiveSqlOperatorTable { - - /** - * List of supported SQL operators / functions. - * - * This list should be kept in sync with [[SqlStdOperatorTable]]. - */ - private val builtInSqlOperators: Seq[SqlOperator] = Seq( - // SET OPERATORS - SqlStdOperatorTable.UNION, - SqlStdOperatorTable.UNION_ALL, - SqlStdOperatorTable.EXCEPT, - SqlStdOperatorTable.EXCEPT_ALL, - SqlStdOperatorTable.INTERSECT, - SqlStdOperatorTable.INTERSECT_ALL, - // BINARY OPERATORS - SqlStdOperatorTable.AND, - SqlStdOperatorTable.AS, - SqlStdOperatorTable.CONCAT, - SqlStdOperatorTable.DIVIDE, - SqlStdOperatorTable.DIVIDE_INTEGER, - SqlStdOperatorTable.DOT, - SqlStdOperatorTable.EQUALS, - SqlStdOperatorTable.GREATER_THAN, - SqlStdOperatorTable.IS_DISTINCT_FROM, - SqlStdOperatorTable.IS_NOT_DISTINCT_FROM, - SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, - SqlStdOperatorTable.LESS_THAN, - SqlStdOperatorTable.LESS_THAN_OR_EQUAL, - SqlStdOperatorTable.MINUS, - SqlStdOperatorTable.MULTIPLY, - SqlStdOperatorTable.NOT_EQUALS, - SqlStdOperatorTable.OR, - SqlStdOperatorTable.PLUS, - SqlStdOperatorTable.DATETIME_PLUS, - // POSTFIX OPERATORS - SqlStdOperatorTable.DESC, - SqlStdOperatorTable.NULLS_FIRST, - SqlStdOperatorTable.IS_NOT_NULL, - SqlStdOperatorTable.IS_NULL, - SqlStdOperatorTable.IS_NOT_TRUE, - SqlStdOperatorTable.IS_TRUE, - SqlStdOperatorTable.IS_NOT_FALSE, - SqlStdOperatorTable.IS_FALSE, - SqlStdOperatorTable.IS_NOT_UNKNOWN, - SqlStdOperatorTable.IS_UNKNOWN, - // PREFIX OPERATORS - SqlStdOperatorTable.NOT, - SqlStdOperatorTable.UNARY_MINUS, - SqlStdOperatorTable.UNARY_PLUS, - // GROUPING FUNCTIONS - SqlStdOperatorTable.GROUP_ID, - SqlStdOperatorTable.GROUPING, - SqlStdOperatorTable.GROUPING_ID, - // AGGREGATE OPERATORS - SqlStdOperatorTable.SUM, - SqlStdOperatorTable.SUM0, - SqlStdOperatorTable.COUNT, - SqlStdOperatorTable.COLLECT, - SqlStdOperatorTable.MIN, - SqlStdOperatorTable.MAX, - SqlStdOperatorTable.AVG, - SqlStdOperatorTable.STDDEV_POP, - SqlStdOperatorTable.STDDEV_SAMP, - SqlStdOperatorTable.VAR_POP, - SqlStdOperatorTable.VAR_SAMP, - // ARRAY OPERATORS - SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR, - SqlStdOperatorTable.ELEMENT, - // MAP OPERATORS - SqlStdOperatorTable.MAP_VALUE_CONSTRUCTOR, - // ARRAY MAP SHARED OPERATORS - SqlStdOperatorTable.ITEM, - SqlStdOperatorTable.CARDINALITY, - // SPECIAL OPERATORS - SqlStdOperatorTable.ROW, - SqlStdOperatorTable.OVERLAPS, - SqlStdOperatorTable.LITERAL_CHAIN, - SqlStdOperatorTable.BETWEEN, - SqlStdOperatorTable.SYMMETRIC_BETWEEN, - SqlStdOperatorTable.NOT_BETWEEN, - SqlStdOperatorTable.SYMMETRIC_NOT_BETWEEN, - SqlStdOperatorTable.NOT_LIKE, - SqlStdOperatorTable.LIKE, - SqlStdOperatorTable.NOT_SIMILAR_TO, - SqlStdOperatorTable.SIMILAR_TO, - SqlStdOperatorTable.CASE, - SqlStdOperatorTable.REINTERPRET, - SqlStdOperatorTable.EXTRACT, - SqlStdOperatorTable.IN, - // FUNCTIONS - SqlStdOperatorTable.SUBSTRING, - SqlStdOperatorTable.OVERLAY, - SqlStdOperatorTable.TRIM, - SqlStdOperatorTable.POSITION, - SqlStdOperatorTable.CHAR_LENGTH, - SqlStdOperatorTable.CHARACTER_LENGTH, - SqlStdOperatorTable.UPPER, - SqlStdOperatorTable.LOWER, - SqlStdOperatorTable.INITCAP, - SqlStdOperatorTable.POWER, - SqlStdOperatorTable.SQRT, - SqlStdOperatorTable.MOD, - SqlStdOperatorTable.LN, - SqlStdOperatorTable.LOG10, - SqlStdOperatorTable.ABS, - SqlStdOperatorTable.EXP, - SqlStdOperatorTable.NULLIF, - SqlStdOperatorTable.COALESCE, - SqlStdOperatorTable.FLOOR, - SqlStdOperatorTable.CEIL, - SqlStdOperatorTable.LOCALTIME, - SqlStdOperatorTable.LOCALTIMESTAMP, - SqlStdOperatorTable.CURRENT_TIME, - SqlStdOperatorTable.CURRENT_TIMESTAMP, - SqlStdOperatorTable.CURRENT_DATE, - ScalarSqlFunctions.DATE_FORMAT, - SqlStdOperatorTable.CAST, - SqlStdOperatorTable.EXTRACT, - SqlStdOperatorTable.QUARTER, - SqlStdOperatorTable.SCALAR_QUERY, - SqlStdOperatorTable.EXISTS, - SqlStdOperatorTable.SIN, - SqlStdOperatorTable.COS, - SqlStdOperatorTable.TAN, - SqlStdOperatorTable.COT, - SqlStdOperatorTable.ASIN, - SqlStdOperatorTable.ACOS, - SqlStdOperatorTable.ATAN, - SqlStdOperatorTable.DEGREES, - SqlStdOperatorTable.RADIANS, - SqlStdOperatorTable.SIGN, - SqlStdOperatorTable.ROUND, - SqlStdOperatorTable.PI, - ScalarSqlFunctions.E, - SqlStdOperatorTable.RAND, - SqlStdOperatorTable.RAND_INTEGER, - ScalarSqlFunctions.CONCAT, - ScalarSqlFunctions.CONCAT_WS, - ScalarSqlFunctions.BIN, - SqlStdOperatorTable.TIMESTAMP_ADD, - ScalarSqlFunctions.LOG, - ScalarSqlFunctions.LPAD, - ScalarSqlFunctions.RPAD, - ScalarSqlFunctions.MD5, - ScalarSqlFunctions.SHA1, - ScalarSqlFunctions.SHA256, - - // EXTENSIONS - BasicOperatorTable.TUMBLE, - BasicOperatorTable.HOP, - BasicOperatorTable.SESSION, - BasicOperatorTable.TUMBLE_START, - BasicOperatorTable.TUMBLE_END, - BasicOperatorTable.HOP_START, - BasicOperatorTable.HOP_END, - BasicOperatorTable.SESSION_START, - BasicOperatorTable.SESSION_END, - BasicOperatorTable.TUMBLE_PROCTIME, - BasicOperatorTable.TUMBLE_ROWTIME, - BasicOperatorTable.HOP_PROCTIME, - BasicOperatorTable.HOP_ROWTIME, - BasicOperatorTable.SESSION_PROCTIME, - BasicOperatorTable.SESSION_ROWTIME, - - // MATCH_RECOGNIZE - SqlStdOperatorTable.FIRST, - SqlStdOperatorTable.LAST, - SqlStdOperatorTable.PREV, - SqlStdOperatorTable.NEXT, - SqlStdOperatorTable.CLASSIFIER, - SqlStdOperatorTable.MATCH_NUMBER, - SqlStdOperatorTable.FINAL, - SqlStdOperatorTable.RUNNING - ) - - builtInSqlOperators.foreach(register) -} - -object BasicOperatorTable { - - /** - * We need custom group auxiliary functions in order to support nested windows. - */ - - val TUMBLE: SqlGroupedWindowFunction = new SqlGroupedWindowFunction( - SqlKind.TUMBLE, - null, - OperandTypes.or(OperandTypes.DATETIME_INTERVAL, OperandTypes.DATETIME_INTERVAL_TIME)) { - override def getAuxiliaryFunctions: _root_.java.util.List[SqlGroupedWindowFunction] = - Seq( - TUMBLE_START, - TUMBLE_END, - TUMBLE_ROWTIME, - TUMBLE_PROCTIME) - } - val TUMBLE_START: SqlGroupedWindowFunction = TUMBLE.auxiliary(SqlKind.TUMBLE_START) - val TUMBLE_END: SqlGroupedWindowFunction = TUMBLE.auxiliary(SqlKind.TUMBLE_END) - val TUMBLE_ROWTIME: SqlGroupedWindowFunction = - new SqlGroupedWindowFunction( - "TUMBLE_ROWTIME", - SqlKind.OTHER_FUNCTION, - TUMBLE, - // ensure that returned rowtime is always NOT_NULLABLE - ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NOT_NULLABLE), - null, - TUMBLE.getOperandTypeChecker, - SqlFunctionCategory.SYSTEM) - val TUMBLE_PROCTIME: SqlGroupedWindowFunction = - TUMBLE.auxiliary("TUMBLE_PROCTIME", SqlKind.OTHER_FUNCTION) - - val HOP: SqlGroupedWindowFunction = new SqlGroupedWindowFunction( - SqlKind.HOP, - null, - OperandTypes.or( - OperandTypes.DATETIME_INTERVAL_INTERVAL, - OperandTypes.DATETIME_INTERVAL_INTERVAL_TIME)) { - override def getAuxiliaryFunctions: _root_.java.util.List[SqlGroupedWindowFunction] = - Seq( - HOP_START, - HOP_END, - HOP_ROWTIME, - HOP_PROCTIME) - } - val HOP_START: SqlGroupedWindowFunction = HOP.auxiliary(SqlKind.HOP_START) - val HOP_END: SqlGroupedWindowFunction = HOP.auxiliary(SqlKind.HOP_END) - val HOP_ROWTIME: SqlGroupedWindowFunction = - new SqlGroupedWindowFunction( - "HOP_ROWTIME", - SqlKind.OTHER_FUNCTION, - HOP, - // ensure that returned rowtime is always NOT_NULLABLE - ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NOT_NULLABLE), - null, - HOP.getOperandTypeChecker, - SqlFunctionCategory.SYSTEM) - val HOP_PROCTIME: SqlGroupedWindowFunction = HOP.auxiliary("HOP_PROCTIME", SqlKind.OTHER_FUNCTION) - - val SESSION: SqlGroupedWindowFunction = new SqlGroupedWindowFunction( - SqlKind.SESSION, - null, - OperandTypes.or(OperandTypes.DATETIME_INTERVAL, OperandTypes.DATETIME_INTERVAL_TIME)) { - override def getAuxiliaryFunctions: _root_.java.util.List[SqlGroupedWindowFunction] = - Seq( - SESSION_START, - SESSION_END, - SESSION_ROWTIME, - SESSION_PROCTIME) - } - val SESSION_START: SqlGroupedWindowFunction = SESSION.auxiliary(SqlKind.SESSION_START) - val SESSION_END: SqlGroupedWindowFunction = SESSION.auxiliary(SqlKind.SESSION_END) - val SESSION_ROWTIME: SqlGroupedWindowFunction = - new SqlGroupedWindowFunction( - "SESSION_ROWTIME", - SqlKind.OTHER_FUNCTION, - SESSION, - // ensure that returned rowtime is always NOT_NULLABLE - ReturnTypes.cascade(ReturnTypes.ARG0, SqlTypeTransforms.TO_NOT_NULLABLE), - null, - SESSION.getOperandTypeChecker, - SqlFunctionCategory.SYSTEM) - val SESSION_PROCTIME: SqlGroupedWindowFunction = - SESSION.auxiliary("SESSION_PROCTIME", SqlKind.OTHER_FUNCTION) - -} diff --git a/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java b/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java index 5abb66e2b..8eb09ee18 100644 --- a/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java +++ b/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java @@ -41,51 +41,6 @@ public class SideSqlExecTest { - /** - * 参考阿里Blink的cep sql语法,文档https://help.aliyun.com/document_detail/73845.html?spm=a2c4g.11186623.6.637.5cba27efFHjOSs - * @throws Exception - */ - @Test - public void testCepSql() throws Exception { - List paramList = Lists.newArrayList(); - paramList.add("-sql"); - String sqlContext = "CREATE table source(" + - "name varchar, " + - "price float, " + - "tax float, " + - "tstamp timestamp) " + - "with (" + - " type = 'kafka09',bootstrapServers = 'kudu1:9092',zookeeperQuorum = '172.16.8.107:2181/kafka', offsetReset = 'latest',topic = 'tranflow_input',parallelism = '1' " + - ");" - + "CREATE table sink(" + - "start_tstamp timestamp, " + - "bottom_tstamp timestamp, " + - "end_tstamp timestamp, " + - "bottom_total float, " + - "end_total float ) " + - "with (" + - " type = 'mysql',url = 'jdbc:mysql://172.16.8.104:3306/bank_test?charset=utf8',userName = 'dtstack',password = 'abc123',tableName = 'max_deposit_acct_base',cache = 'LRU',cacheSize = '10000',cacheTTLMs = '60000',parallelism = '1' " + - ");" - + "insert into sink " + - "select * from source " + - "MATCH_RECOGNIZE (\n" + - " MEASURES\n" + - " STRT.tstamp AS start_tstamp,\n" + - " LAST(DOWN.tstamp) AS bottom_tstamp,\n" + - " LAST(UP.tstamp) AS end_tstamp,\n" + - " FIRST(DOWN.price + DOWN.tax + 1) AS bottom_total,\n" + - " FIRST(UP.price + UP.tax) AS end_total" + - " ONE ROW PER MATCH\n" + - " PATTERN (STRT DOWN+ UP+)\n" + - " DEFINE\n" + - " DOWN AS DOWN.price < PREV(DOWN.price),\n" + - " UP AS UP.price > PREV(UP.price) AND UP.tax > LAST(DOWN.tax)\n" + - ") AS T" - ; - test(sqlContext); - } - - @Test public void testRunSideSql() throws Exception { //String runParam = "-sql CREATE+TABLE+MyTable(channel+STRING%2c+pv+INT%2c+xctime+bigint%2c+timeLeng+as+CHARACTER_LENGTH(channel)%2c++WATERMARK+FOR+xctime+AS+withOffset(xctime%2c+1000))+WITH+(+type%3d%27kafka09%27%2c+bootstrapServers%3d%27172.16.8.198%3a9092%27%2c+offsetReset%3d%27latest%27%2ctopic%3d%27nbTest1%27)%3bCREATE+TABLE+MyResult(channel+STRING%2c+pv+INT)+WITH+(+type%3d%27mysql%27%2c+url%3d%27jdbc%3amysql%3a%2f%2f172.16.8.104%3a3306%2ftest%3fcharset%3dutf8%27%2cuserName%3d%27dtstack%27%2cpassword%3d%27abc123%27%2c+tableName%3d%27pv%27)%3bcreate+table+sideTable(channel+String%2c+count+int%2c+PERIOD+FOR+SYSTEM_TIME)+WITH+(+type%3d%27mysql%27%2c+url%3d%27jdbc%3amysql%3a%2f%2f172.16.8.104%3a3306%2ftest%3fcharset%3dutf8%27%2cuserName%3d%27dtstack%27%2cpassword%3d%27abc123%27%2c+tableName%3d%27pv%27)%3binsert+into+MyResult+select+a.channel%2cb.pv+from+MyTable+a+join+sideTable+b+on+a.channel%3db.channel%3b -name xc -localSqlPluginPath D:\\gitspace\\flink-sql-plugin\\plugins -mode local -remoteSqlPluginPath /opt/dtstack/flinkplugin -confProp %7b%22time.characteristic%22%3a%22EventTime%22%7d -addjar %5b%22D%3a%5c%5cgitspace%5c%5crdos-execution-engine%5c%5c..%5c%5ctmp140%5c%5cflink14Test-1.0-SNAPSHOT.jar%22%5d"; diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 53febb240..095b950cb 100644 --- a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -1,5 +1,6 @@ package com.dtstack.flink.sql.sink.kafka; +import org.apache.commons.lang3.StringEscapeUtils; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -7,7 +8,6 @@ import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.table.shaded.org.apache.commons.lang.StringEscapeUtils; import org.apache.flink.types.Row; import org.apache.flink.types.StringValue; diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 4aa7f49fa..e5a7b9300 100644 --- a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -25,14 +25,13 @@ import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.Kafka08TableSink; -import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.TableSchemaBuilder; import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; @@ -85,7 +84,7 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { } this.fieldTypes = types; - TableSchemaBuilder schemaBuilder = TableSchema.builder(); + TableSchema.Builder schemaBuilder = TableSchema.builder(); for (int i=0;i dataStream) { - KafkaTableSink kafkaTableSink = new Kafka08TableSink( + KafkaTableSinkBase kafkaTableSink = new Kafka08TableSink( schema, topic, properties, diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index 5dae21742..61acfa2d6 100644 --- a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.sink.kafka.table; import com.dtstack.flink.sql.table.TargetTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; import java.util.HashMap; import java.util.Map; diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 88a10a293..354840ea4 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -1,13 +1,12 @@ package com.dtstack.flink.sql.sink.kafka; -import org.apache.flink.annotation.Internal; +import org.apache.commons.lang3.StringEscapeUtils; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.table.shaded.org.apache.commons.lang.StringEscapeUtils; import org.apache.flink.types.Row; import org.apache.flink.types.StringValue; @@ -22,7 +21,6 @@ * @modifyer maqi * */ -@Internal public final class CustomerCsvSerialization extends TypeSerializerSingleton { private static final long serialVersionUID = 1L; diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 106c3ab5c..db4cf3d1b 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -25,14 +25,13 @@ import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.Kafka09TableSink; -import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.TableSchemaBuilder; import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; @@ -86,7 +85,7 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { } this.fieldTypes = types; - TableSchemaBuilder schemaBuilder = TableSchema.builder(); + TableSchema.Builder schemaBuilder = TableSchema.builder(); for (int i=0;i dataStream) { - KafkaTableSink kafkaTableSink = new Kafka09TableSink( + KafkaTableSinkBase kafkaTableSink = new Kafka09TableSink( schema, topic, properties, diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index 48daf57dd..f29def338 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.sink.kafka.table; import com.dtstack.flink.sql.table.TargetTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; import java.util.HashMap; import java.util.Map; diff --git a/kafka09/pom.xml b/kafka09/pom.xml index 1ae761e9b..86613de70 100644 --- a/kafka09/pom.xml +++ b/kafka09/pom.xml @@ -6,7 +6,7 @@ flink.sql com.dtstack.flink 1.0-SNAPSHOT - + ../pom.xml 4.0.0 diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 825703332..09377e376 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -1,13 +1,12 @@ package com.dtstack.flink.sql.sink.kafka; -import org.apache.flink.annotation.Internal; +import org.apache.commons.lang3.StringEscapeUtils; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.table.shaded.org.apache.commons.lang.StringEscapeUtils; import org.apache.flink.types.Row; import org.apache.flink.types.StringValue; @@ -24,7 +23,6 @@ * @modifyer maqi * */ -@Internal public final class CustomerCsvSerialization extends TypeSerializerSingleton { private static final long serialVersionUID = 1L; diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 7d3748441..52214594c 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -25,14 +25,13 @@ import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.Kafka010TableSink; -import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.TableSchemaBuilder; import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; @@ -88,7 +87,7 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { } this.fieldTypes = types; - TableSchemaBuilder schemaBuilder = TableSchema.builder(); + TableSchema.Builder schemaBuilder = TableSchema.builder(); for (int i=0;i dataStream) { - KafkaTableSink kafkaTableSink = new Kafka010TableSink( + KafkaTableSinkBase kafkaTableSink = new Kafka010TableSink( schema, topic, properties, diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index 0235547d2..a641894db 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -19,7 +19,8 @@ package com.dtstack.flink.sql.sink.kafka.table; import com.dtstack.flink.sql.table.TargetTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; + import java.util.HashMap; import java.util.Map; diff --git a/kafka10/kafka10-source/pom.xml b/kafka10/kafka10-source/pom.xml index 0f0c55c36..0e949beeb 100644 --- a/kafka10/kafka10-source/pom.xml +++ b/kafka10/kafka10-source/pom.xml @@ -4,6 +4,7 @@ sql.kafka10 com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 5184eba4f..480133662 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -17,14 +17,13 @@ */ package com.dtstack.flink.sql.sink.kafka; -import org.apache.flink.annotation.Internal; +import org.apache.commons.lang3.StringEscapeUtils; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.table.shaded.org.apache.commons.lang.StringEscapeUtils; import org.apache.flink.types.Row; import org.apache.flink.types.StringValue; @@ -40,7 +39,6 @@ * @modifyer maqi * */ -@Internal public final class CustomerCsvSerialization extends TypeSerializerSingleton { private static final long serialVersionUID = 1L; diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 44383ec9b..3d55088d0 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -25,15 +25,14 @@ import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.Kafka011TableSink; -import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.TableSchemaBuilder; import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; @@ -90,7 +89,7 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { } this.fieldTypes = types; - TableSchemaBuilder schemaBuilder = TableSchema.builder(); + TableSchema.Builder schemaBuilder = TableSchema.builder(); for (int i=0;i dataStream) { - KafkaTableSink kafkaTableSink = new Kafka011TableSink( + KafkaTableSinkBase kafkaTableSink = new Kafka011TableSink( schema, topic, properties, diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index 78edf17b4..9a7834f4d 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.sink.kafka.table; import com.dtstack.flink.sql.table.TargetTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; import java.util.HashMap; import java.util.Map; diff --git a/kafka11/kafka11-source/pom.xml b/kafka11/kafka11-source/pom.xml index ae6997f7e..cf540b15e 100644 --- a/kafka11/kafka11-source/pom.xml +++ b/kafka11/kafka11-source/pom.xml @@ -4,6 +4,7 @@ sql.kafka11 com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 0f538aa1d..f6df31607 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -35,7 +35,6 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.SocketTextStreamFunction; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011; import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; import org.apache.flink.table.api.Table; diff --git a/kafka11/pom.xml b/kafka11/pom.xml index f1f57aa03..1f2be9c18 100644 --- a/kafka11/pom.xml +++ b/kafka11/pom.xml @@ -6,6 +6,7 @@ flink.sql com.dtstack.flink 1.0-SNAPSHOT + ../pom.xml 4.0.0 diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 47718581f..687ec80ff 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -25,6 +25,8 @@ import com.alibaba.fastjson.TypeReference; import com.dtstack.flink.sql.Main; import com.dtstack.flink.sql.launcher.perjob.PerJobSubmitter; +import org.apache.commons.lang3.BooleanUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; @@ -37,9 +39,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.table.shaded.org.apache.commons.lang.StringUtils; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; -import org.apache.flink.table.shaded.org.apache.commons.lang.BooleanUtils; import com.dtstack.flink.sql.options.*; /** diff --git a/pom.xml b/pom.xml index c3ad4d24b..eaf679c2d 100644 --- a/pom.xml +++ b/pom.xml @@ -31,7 +31,7 @@ UTF-8 - 1.6.2 + 1.7.0 diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java index 284d458b3..03b5b5a5b 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java @@ -223,7 +223,7 @@ public static Date getDate(Object obj) { return null; } if (obj instanceof String) { - SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd"); + SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); try { return new Date(format.parse((String) obj).getTime()); } catch (ParseException e) { From 15cd1d918058b8fad86d2a61bf362de3595ba0ed Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 2 Jan 2019 22:07:15 +0800 Subject: [PATCH 240/470] add nested json format parsing --- .../flink/sql/table/AbsTableParser.java | 33 ++++- .../dtstack/flink/sql/table/TableInfo.java | 20 +++ docs/serverSocketSource.md | 38 +----- .../flink/sql/source/kafka/KafkaSource.java | 4 +- .../CustomerJsonDeserialization.java | 117 +++++++++++++++--- 5 files changed, 154 insertions(+), 58 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index c3ccdd789..f8c02e22b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -22,6 +22,7 @@ import com.dtstack.flink.sql.util.ClassUtil; import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.collect.Maps; @@ -83,17 +84,43 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ for(String fieldRow : fieldRows){ fieldRow = fieldRow.trim(); - boolean isMatcherKey = dealKeyPattern(fieldRow, tableInfo); + String[] filedInfoArr = fieldRow.split("\\s+"); + boolean isMatcherKey = dealKeyPattern(fieldRow, tableInfo); if(isMatcherKey){ continue; } - String[] filedInfoArr = fieldRow.split("\\s+"); - if(filedInfoArr.length < 2){ + if(filedInfoArr.length < 2 ){ throw new RuntimeException(String.format("table [%s] field [%s] format error.", tableInfo.getName(), fieldRow)); } + if(filedInfoArr.length > 2 ){ + throw new RuntimeException("mapping field can't contain spaces."); + } + + String physicalField=null; + + if (filedInfoArr[0].contains("(")){ + String first=filedInfoArr[0]; + int leftIndex=first.indexOf("("); + int rightIndex=first.indexOf(")"); + + String newFirst=first.substring(0,leftIndex).trim(); + filedInfoArr[0]=newFirst; + + physicalField=first.substring(leftIndex+1,rightIndex).trim(); + } + + if (StringUtils.isNotBlank(physicalField)){ + tableInfo.addPhysicalMappings(filedInfoArr[0],physicalField); + }else { + tableInfo.addPhysicalMappings(filedInfoArr[0],filedInfoArr[0]); + } + + + + //Compatible situation may arise in space in the fieldName String[] filedNameArr = new String[filedInfoArr.length - 1]; System.arraycopy(filedInfoArr, 0, filedNameArr, 0, filedInfoArr.length - 1); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java index d57b3061a..889d60398 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java @@ -20,10 +20,12 @@ package com.dtstack.flink.sql.table; +import com.google.common.collect.Maps; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import java.io.Serializable; import java.util.List; +import java.util.Map; /** * Reason: @@ -56,6 +58,9 @@ public abstract class TableInfo implements Serializable { private final List fieldClassList = Lists.newArrayList(); + /** handling nested data structures **/ + private Map physicalFields = Maps.newHashMap(); + private List primaryKeys; private Integer parallelism = 1; @@ -154,7 +159,22 @@ public List getFieldClassList() { return fieldClassList; } + public Map getPhysicalFields() { + return physicalFields; + } + public void setPhysicalFields(Map physicalFields) { + this.physicalFields = physicalFields; + } + + /** + * + * @param key row field + * @param value physical field + */ + public void addPhysicalMappings(String key,String value){ + physicalFields.put(key,value); + } public String getFieldDelimiter() { return fieldDelimiter; diff --git a/docs/serverSocketSource.md b/docs/serverSocketSource.md index cffdc8dd2..4f889d829 100644 --- a/docs/serverSocketSource.md +++ b/docs/serverSocketSource.md @@ -32,42 +32,6 @@ CREATE TABLE MyTable( ## 3.Server端样例: ``` -String str = "{\"CHANNEL\":\"xc3\",\"pv\":1234567,\"xdate\":\"2018-12-07\",\"xtime\":\"2018-12-15\"};"; +String JsonStr = "{\"CHANNEL\":\"xc3\",\"pv\":1234567,\"xdate\":\"2018-12-07\",\"xtime\":\"2018-12-15\"};"; - -public class TimeServerHandler implements Runnable { - Socket socket; - - String str = "{\"CHANNEL\":\"xc3\",\"pv\":1234567,\"xdate\":\"2018-12-07\",\"xtime\":\"2018-12-15\"};"; - - public TimeServerHandler(Socket socket) { - this.socket = socket; - } - - public void run() { - PrintWriter out = null; - try { - out = new PrintWriter(this.socket.getOutputStream(), true); - while (true) { - Thread.sleep(3000); - out.println(str); - } - } catch (IOException e) { - e.printStackTrace(); - - if (out != null) { - out.close(); - } - if (socket != null) { - try { - socket.close(); - } catch (IOException e1) { - e1.printStackTrace(); - } - } - } catch (InterruptedException e) { - e.printStackTrace(); - } - } -} ``` diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index f6df31607..2be3b332f 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -92,10 +92,10 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv if ("json".equalsIgnoreCase(kafka011SourceTableInfo.getSourceDataType())) { if (topicIsPattern) { kafkaSrc = new CustomerJsonConsumer(Pattern.compile(topicName), - new CustomerJsonDeserialization(typeInformation), props); + new CustomerJsonDeserialization(typeInformation,sourceTableInfo.getPhysicalFields()), props); } else { kafkaSrc = new CustomerJsonConsumer(topicName, - new CustomerJsonDeserialization(typeInformation), props); + new CustomerJsonDeserialization(typeInformation,sourceTableInfo.getPhysicalFields()), props); } } else if ("csv".equalsIgnoreCase(kafka011SourceTableInfo.getSourceDataType())) { if (topicIsPattern) { diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java index e72fd0303..730ff6529 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java @@ -23,9 +23,15 @@ import com.dtstack.flink.sql.source.AbsDeserialization; import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import com.google.common.collect.Maps; +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; @@ -39,7 +45,11 @@ import java.io.IOException; import java.lang.reflect.Field; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; import java.util.Iterator; +import java.util.Map; import java.util.Set; import static com.dtstack.flink.sql.metric.MetricConstant.DT_PARTITION_GROUP; @@ -77,16 +87,22 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private boolean firstMsg = true; - public CustomerJsonDeserialization(TypeInformation typeInfo){ + private Map rowAndFieldMapping = Maps.newHashMap(); + + private Map nodeAndJsonnodeMapping = Maps.newHashMap(); + + public CustomerJsonDeserialization(TypeInformation typeInfo,Map rowAndFieldMapping){ this.typeInfo = typeInfo; this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + + this.rowAndFieldMapping=rowAndFieldMapping; } @Override - public Row deserialize(byte[] message) throws IOException { + public Row deserialize(byte[] message) { if(firstMsg){ try { @@ -103,7 +119,10 @@ public Row deserialize(byte[] message) throws IOException { numInBytes.inc(message.length); JsonNode root = objectMapper.readTree(message); + parseTree(root,""); + Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { JsonNode node = getIgnoreCase(root, fieldNames[i]); @@ -115,9 +134,7 @@ public Row deserialize(byte[] message) throws IOException { row.setField(i, null); } } else { - // Read the value as specified type - Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); - row.setField(i, value); + row.setField(i, convert(node, fieldTypes[i])); } } @@ -131,22 +148,29 @@ public Row deserialize(byte[] message) throws IOException { } } + public void parseTree(JsonNode jsonNode, String prefix){ + nodeAndJsonnodeMapping.clear(); + + Iterator iterator = jsonNode.fieldNames(); + while (iterator.hasNext()){ + String next = iterator.next(); + JsonNode child = jsonNode.get(next); + if (child.isObject()){ + parseTree(child,next+"."+prefix); + }else { + nodeAndJsonnodeMapping.put(prefix+next,child); + } + } + } + public void setFailOnMissingField(boolean failOnMissingField) { this.failOnMissingField = failOnMissingField; } - public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { - - Iterator iter = jsonNode.fieldNames(); - while (iter.hasNext()) { - String key1 = iter.next(); - if (key1.equalsIgnoreCase(key)) { - return jsonNode.get(key1); - } - } - - return null; + public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { + String nodeMappingKey = rowAndFieldMapping.get(key); + return nodeAndJsonnodeMapping.get(nodeMappingKey); } public void setFetcher(AbstractFetcher fetcher) { @@ -192,4 +216,65 @@ protected void registerPtMetric(AbstractFetcher fetcher) throws Exceptio private static String partitionLagMetricName(TopicPartition tp) { return tp + ".records-lag"; } + + // -------------------------------------------------------------------------------------------- + + private Object convert(JsonNode node, TypeInformation info) { + if (info.getTypeClass().equals(Types.BOOLEAN.getTypeClass())) { + return node.asBoolean(); + } else if (info.getTypeClass().equals(Types.STRING.getTypeClass())) { + return node.asText(); + } else if (info.getTypeClass().equals(Types.BIG_DEC.getTypeClass())) { + return node.decimalValue(); + } else if (info.getTypeClass().equals(Types.BIG_INT.getTypeClass())) { + return node.bigIntegerValue(); + } else if (info.getTypeClass().equals(Types.SQL_DATE.getTypeClass())) { + return Date.valueOf(node.asText()); + } else if (info.getTypeClass().equals(Types.SQL_TIME.getTypeClass())) { + // according to RFC 3339 every full-time must have a timezone; + // until we have full timezone support, we only support UTC; + // users can parse their time as string as a workaround + final String time = node.asText(); + if (time.indexOf('Z') < 0 || time.indexOf('.') >= 0) { + throw new IllegalStateException( + "Invalid time format. Only a time in UTC timezone without milliseconds is supported yet. " + + "Format: HH:mm:ss'Z'"); + } + return Time.valueOf(time.substring(0, time.length() - 1)); + } else if (info.getTypeClass().equals(Types.SQL_TIMESTAMP.getTypeClass())) { + // according to RFC 3339 every date-time must have a timezone; + // until we have full timezone support, we only support UTC; + // users can parse their time as string as a workaround + final String timestamp = node.asText(); + if (timestamp.indexOf('Z') < 0) { + throw new IllegalStateException( + "Invalid timestamp format. Only a timestamp in UTC timezone is supported yet. " + + "Format: yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); + } + return Timestamp.valueOf(timestamp.substring(0, timestamp.length() - 1).replace('T', ' ')); + } else if (info instanceof ObjectArrayTypeInfo) { + throw new IllegalStateException("Unsupported type information '" + info + "' for node: " + node); + } else if (info instanceof BasicArrayTypeInfo) { + throw new IllegalStateException("Unsupported type information '" + info + "' for node: " + node); + } else if (info instanceof PrimitiveArrayTypeInfo && + ((PrimitiveArrayTypeInfo) info).getComponentType() == Types.BYTE) { + return convertByteArray(node); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return objectMapper.treeToValue(node, info.getTypeClass()); + } catch (JsonProcessingException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for node: " + node); + } + } + } + + private Object convertByteArray(JsonNode node) { + try { + return node.binaryValue(); + } catch (IOException e) { + throw new RuntimeException("Unable to deserialize byte array.", e); + } + } } From d28c31246438e45d2cc6bd64b53ea3dc94419d3a Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 3 Jan 2019 15:57:05 +0800 Subject: [PATCH 241/470] kafka11 read nest json field --- .../flink/sql/table/AbsTableParser.java | 34 +++---------------- .../CustomerJsonDeserialization.java | 28 ++++----------- .../source/kafka/table/KafkaSourceParser.java | 26 ++++++++++++++ pom.xml | 30 ++++++++-------- 4 files changed, 51 insertions(+), 67 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index f8c02e22b..051f2f250 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -85,42 +85,15 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ fieldRow = fieldRow.trim(); String[] filedInfoArr = fieldRow.split("\\s+"); - - boolean isMatcherKey = dealKeyPattern(fieldRow, tableInfo); - if(isMatcherKey){ - continue; - } - if(filedInfoArr.length < 2 ){ throw new RuntimeException(String.format("table [%s] field [%s] format error.", tableInfo.getName(), fieldRow)); } - if(filedInfoArr.length > 2 ){ - throw new RuntimeException("mapping field can't contain spaces."); - } - - String physicalField=null; - - if (filedInfoArr[0].contains("(")){ - String first=filedInfoArr[0]; - int leftIndex=first.indexOf("("); - int rightIndex=first.indexOf(")"); - - String newFirst=first.substring(0,leftIndex).trim(); - filedInfoArr[0]=newFirst; - - physicalField=first.substring(leftIndex+1,rightIndex).trim(); - } - - if (StringUtils.isNotBlank(physicalField)){ - tableInfo.addPhysicalMappings(filedInfoArr[0],physicalField); - }else { - tableInfo.addPhysicalMappings(filedInfoArr[0],filedInfoArr[0]); + boolean isMatcherKey = dealKeyPattern(fieldRow, tableInfo); + if(isMatcherKey){ + continue; } - - - //Compatible situation may arise in space in the fieldName String[] filedNameArr = new String[filedInfoArr.length - 1]; System.arraycopy(filedInfoArr, 0, filedNameArr, 0, filedInfoArr.length - 1); @@ -128,6 +101,7 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ String fieldType = filedInfoArr[filedInfoArr.length - 1 ].trim(); Class fieldClass = ClassUtil.stringConvertClass(fieldType); + tableInfo.addPhysicalMappings(filedInfoArr[0],filedInfoArr[0]); tableInfo.addField(fieldName); tableInfo.addFieldClass(fieldClass); tableInfo.addFieldType(fieldType); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java index 730ff6529..c2baed6ce 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java @@ -140,9 +140,9 @@ public Row deserialize(byte[] message) { numInResolveRecord.inc(); return row; - } catch (Throwable t) { + } catch (Exception e) { //add metric of dirty data - LOG.error(t.getMessage()); + LOG.error(e.getMessage()); dirtyDataCounter.inc(); return null; } @@ -231,27 +231,11 @@ private Object convert(JsonNode node, TypeInformation info) { } else if (info.getTypeClass().equals(Types.SQL_DATE.getTypeClass())) { return Date.valueOf(node.asText()); } else if (info.getTypeClass().equals(Types.SQL_TIME.getTypeClass())) { - // according to RFC 3339 every full-time must have a timezone; - // until we have full timezone support, we only support UTC; - // users can parse their time as string as a workaround - final String time = node.asText(); - if (time.indexOf('Z') < 0 || time.indexOf('.') >= 0) { - throw new IllegalStateException( - "Invalid time format. Only a time in UTC timezone without milliseconds is supported yet. " + - "Format: HH:mm:ss'Z'"); - } - return Time.valueOf(time.substring(0, time.length() - 1)); + // local zone + return Time.valueOf(node.asText()); } else if (info.getTypeClass().equals(Types.SQL_TIMESTAMP.getTypeClass())) { - // according to RFC 3339 every date-time must have a timezone; - // until we have full timezone support, we only support UTC; - // users can parse their time as string as a workaround - final String timestamp = node.asText(); - if (timestamp.indexOf('Z') < 0) { - throw new IllegalStateException( - "Invalid timestamp format. Only a timestamp in UTC timezone is supported yet. " + - "Format: yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); - } - return Timestamp.valueOf(timestamp.substring(0, timestamp.length() - 1).replace('T', ' ')); + // local zone + return Timestamp.valueOf(node.asText()); } else if (info instanceof ObjectArrayTypeInfo) { throw new IllegalStateException("Unsupported type information '" + info + "' for node: " + node); } else if (info instanceof BasicArrayTypeInfo) { diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index f2297f9c7..43b25b47a 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -20,10 +20,14 @@ package com.dtstack.flink.sql.source.kafka.table; import com.dtstack.flink.sql.table.AbsSourceParser; +import com.dtstack.flink.sql.table.SourceTableInfo; import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.ClassUtil; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; /** * Reason: @@ -35,6 +39,28 @@ public class KafkaSourceParser extends AbsSourceParser { + private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; + + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + + static { + keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); + + keyHandlerMap.put(KAFKA_NEST_FIELD_KEY, KafkaSourceParser::dealNestField); + } + + static void dealNestField(Matcher matcher, TableInfo tableInfo) { + SourceTableInfo sourceTableInfo = (SourceTableInfo) tableInfo; + String physicalField = matcher.group(1); + String fieldType = matcher.group(3); + String mappingField = matcher.group(4); + Class fieldClass=ClassUtil.stringConvertClass(fieldType); + + tableInfo.addPhysicalMappings(mappingField,physicalField); + tableInfo.addField(mappingField); + tableInfo.addFieldClass(fieldClass); + tableInfo.addFieldType(fieldType); + } @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { diff --git a/pom.xml b/pom.xml index eaf679c2d..e350e03ab 100644 --- a/pom.xml +++ b/pom.xml @@ -10,22 +10,22 @@ http://maven.apache.org core - kafka09 - kafka10 + + kafka11 - mysql - hbase - elasticsearch5 - mongo - redis5 - launcher - rdb - sqlserver - oracle - cassandra - kafka08 - serversocket - console + + + + + + + + + + + + + From 2da4973da3dc48a03934965f09ff9fb9ee59ed7c Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 3 Jan 2019 21:28:41 +0800 Subject: [PATCH 242/470] recover pom --- README.md | 3 +-- pom.xml | 30 +++++++++++++++--------------- 2 files changed, 16 insertions(+), 17 deletions(-) diff --git a/README.md b/README.md index 808b40432..22f463efb 100644 --- a/README.md +++ b/README.md @@ -14,11 +14,10 @@ * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra,console # 后续开发计划 - * 增加kafka结果表功能 * 增加SQL支持CEP * 维表快照 * sql优化(谓词下移等) - + ## 1 快速起步 ### 1.1 运行模式 diff --git a/pom.xml b/pom.xml index e350e03ab..eaf679c2d 100644 --- a/pom.xml +++ b/pom.xml @@ -10,22 +10,22 @@ http://maven.apache.org core - - + kafka09 + kafka10 kafka11 - - - - - - - - - - - - - + mysql + hbase + elasticsearch5 + mongo + redis5 + launcher + rdb + sqlserver + oracle + cassandra + kafka08 + serversocket + console From fd55dc030f974d29b32fe3f950bcd2a5c850c4b2 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Fri, 4 Jan 2019 10:31:59 +0800 Subject: [PATCH 243/470] handle json array type when get node --- .../CustomerJsonDeserialization.java | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java index c2baed6ce..df994132c 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java @@ -34,6 +34,7 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.types.Row; @@ -124,7 +125,7 @@ public Row deserialize(byte[] message) { Row row = new Row(fieldNames.length); for (int i = 0; i < fieldNames.length; i++) { - JsonNode node = getIgnoreCase(root, fieldNames[i]); + JsonNode node = getIgnoreCase(fieldNames[i]); if (node == null) { if (failOnMissingField) { @@ -168,9 +169,16 @@ public void setFailOnMissingField(boolean failOnMissingField) { } - public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { + public JsonNode getIgnoreCase(String key) { String nodeMappingKey = rowAndFieldMapping.get(key); - return nodeAndJsonnodeMapping.get(nodeMappingKey); + JsonNode node = nodeAndJsonnodeMapping.get(nodeMappingKey); + JsonNodeType nodeType = node.getNodeType(); + + if (nodeType==JsonNodeType.ARRAY){ + throw new IllegalStateException("Unsupported type information array .") ; + } + + return node; } public void setFetcher(AbstractFetcher fetcher) { @@ -236,12 +244,7 @@ private Object convert(JsonNode node, TypeInformation info) { } else if (info.getTypeClass().equals(Types.SQL_TIMESTAMP.getTypeClass())) { // local zone return Timestamp.valueOf(node.asText()); - } else if (info instanceof ObjectArrayTypeInfo) { - throw new IllegalStateException("Unsupported type information '" + info + "' for node: " + node); - } else if (info instanceof BasicArrayTypeInfo) { - throw new IllegalStateException("Unsupported type information '" + info + "' for node: " + node); - } else if (info instanceof PrimitiveArrayTypeInfo && - ((PrimitiveArrayTypeInfo) info).getComponentType() == Types.BYTE) { + } else if (info.getTypeClass().equals(Types.BYTE.getTypeClass())){ return convertByteArray(node); } else { // for types that were specified without JSON schema From 25c78ffafff07a25347722f6b64b2c9f0ea542d7 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 8 Jan 2019 20:24:29 +0800 Subject: [PATCH 244/470] remove primary key field space --- .../main/java/com/dtstack/flink/sql/table/AbsTableParser.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index a43769918..4fa4f0597 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -109,7 +109,7 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ } public static void dealPrimaryKey(Matcher matcher, TableInfo tableInfo){ - String primaryFields = matcher.group(1); + String primaryFields = matcher.group(1).trim(); String[] splitArry = primaryFields.split(","); List primaryKes = Lists.newArrayList(splitArry); tableInfo.setPrimaryKeys(primaryKes); From 02f57cac947b7aa020ec6ad7f636272c966b85a1 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 8 Jan 2019 20:27:40 +0800 Subject: [PATCH 245/470] remove primary key field space --- .../main/java/com/dtstack/flink/sql/table/AbsTableParser.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index 051f2f250..f11a646eb 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -111,7 +111,7 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ } public static void dealPrimaryKey(Matcher matcher, TableInfo tableInfo){ - String primaryFields = matcher.group(1); + String primaryFields = matcher.group(1).trim(); String[] splitArry = primaryFields.split(","); List primaryKes = Lists.newArrayList(splitArry); tableInfo.setPrimaryKeys(primaryKes); From 3d20e61268cb29fe93e3858c3e4f925640329075 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 9 Jan 2019 11:09:24 +0800 Subject: [PATCH 246/470] hbase remove field to upper --- .../java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java | 2 +- .../side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java | 4 ++-- .../sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java index de89f337e..c9adc87b9 100644 --- a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java @@ -170,7 +170,7 @@ private void loadData(Map> tmpCache) throws SQLExcep StringBuilder key = new StringBuilder(); key.append(family).append(":").append(qualifier); - kv.put(aliasNameInversion.get(key.toString().toUpperCase()), value); + kv.put(aliasNameInversion.get(key.toString()), value); } tmpCache.put(new String(r.getRow()), kv); } diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java index d077f3493..d79f67ec5 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java @@ -98,9 +98,9 @@ private String dealOneRow(ArrayList> args, String rowKeyStr, String mapKey = cf + ":" + col; //The table format defined using different data type conversion byte - String colType = colRefType.get(mapKey.toUpperCase()); + String colType = colRefType.get(mapKey); Object val = HbaseUtils.convertByte(keyValue.value(), colType); - sideMap.put(mapKey.toUpperCase(), val); + sideMap.put(mapKey, val); } if (oneRow.size() > 0) { diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java index a85f58c4b..4a4b60ff6 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java @@ -73,9 +73,9 @@ public void asyncGetData(String tableName, String rowKeyStr, Row input, ResultFu String col = new String(keyValue.qualifier()); String mapKey = cf + ":" + col; //The table format defined using different data type conversion byte - String colType = colRefType.get(mapKey.toUpperCase()); + String colType = colRefType.get(mapKey); Object val = HbaseUtils.convertByte(keyValue.value(), colType); - sideMap.put(mapKey.toUpperCase(), val); + sideMap.put(mapKey, val); } if(arg.size() > 0){ From 3594509f534b25f20a6ec34b8457850b371e34d4 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 10 Jan 2019 13:54:21 +0800 Subject: [PATCH 247/470] fix mongo read asyncside info bug --- .../com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index d376bcdde..d8a22772b 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -142,12 +142,12 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except dealMissKey(input, resultFuture); return; } else if (ECacheContentType.MultiLine == val.getType()) { - + List rowList = Lists.newArrayList(); for (Object jsonArray : (List) val.getContent()) { Row row = fillData(input, jsonArray); - resultFuture.complete(Collections.singleton(row)); + rowList.add(row); } - + resultFuture.complete(rowList); } else { throw new RuntimeException("not support cache obj type " + val.getType()); } From 4dc651bbf294492bb18e473628fc0729858fb1a9 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 10 Jan 2019 13:57:13 +0800 Subject: [PATCH 248/470] fix mongo sync read bug --- .../com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index d376bcdde..d8a22772b 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -142,12 +142,12 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except dealMissKey(input, resultFuture); return; } else if (ECacheContentType.MultiLine == val.getType()) { - + List rowList = Lists.newArrayList(); for (Object jsonArray : (List) val.getContent()) { Row row = fillData(input, jsonArray); - resultFuture.complete(Collections.singleton(row)); + rowList.add(row); } - + resultFuture.complete(rowList); } else { throw new RuntimeException("not support cache obj type " + val.getType()); } From 9d39c607111b569d3879b6993306a344143fa7b7 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 10 Jan 2019 14:16:22 +0800 Subject: [PATCH 249/470] fix redis asyn read info bug --- .../dtstack/flink/sql/side/redis/RedisAsyncReqRow.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index 1a6aa0425..251107287 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -141,8 +141,12 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except dealMissKey(input, resultFuture); return; }else if(ECacheContentType.MultiLine == val.getType()){ - Row row = fillData(input, val.getContent()); - resultFuture.complete(Collections.singleton(row)); + List rowList = Lists.newArrayList(); + for (Object jsonArray : (List) val.getContent()) { + Row row = fillData(input, val.getContent()); + rowList.add(row); + } + resultFuture.complete(rowList); }else{ throw new RuntimeException("not support cache obj type " + val.getType()); } From 3c8cc0ce4f9ced671837fde5c4d0d5d78c78ef52 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 10 Jan 2019 14:18:33 +0800 Subject: [PATCH 250/470] fix redis async read info bug --- .../dtstack/flink/sql/side/redis/RedisAsyncReqRow.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index 1a6aa0425..251107287 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -141,8 +141,12 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except dealMissKey(input, resultFuture); return; }else if(ECacheContentType.MultiLine == val.getType()){ - Row row = fillData(input, val.getContent()); - resultFuture.complete(Collections.singleton(row)); + List rowList = Lists.newArrayList(); + for (Object jsonArray : (List) val.getContent()) { + Row row = fillData(input, val.getContent()); + rowList.add(row); + } + resultFuture.complete(rowList); }else{ throw new RuntimeException("not support cache obj type " + val.getType()); } From 70a5b46fccb4cb1165a48df70bb9c936c95fbf66 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 10 Jan 2019 17:53:36 +0800 Subject: [PATCH 251/470] mongo rdb syncside bug fix --- .../dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java | 1 + .../dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java | 8 ++++---- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index d8a22772b..226885ed7 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -130,6 +130,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except Object equalObj = input.getField(conValIndex); if (equalObj == null) { resultFuture.complete(null); + return; } basicDBObject.put(sideInfo.getEqualFieldList().get(i), equalObj); } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index 73fab0e9f..35f300c26 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -75,8 +75,8 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except Object equalObj = input.getField(conValIndex); if (equalObj == null) { resultFuture.complete(null); + return; } - inputParams.add(equalObj); } @@ -89,12 +89,12 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except dealMissKey(input, resultFuture); return; } else if (ECacheContentType.MultiLine == val.getType()) { - + List rowList = Lists.newArrayList(); for (Object jsonArray : (List) val.getContent()) { Row row = fillData(input, jsonArray); - resultFuture.complete(Collections.singleton(row)); + rowList.add(row); } - + resultFuture.complete(rowList); } else { throw new RuntimeException("not support cache obj type " + val.getType()); } From c9a56594094e2be93178f47c40f8b8b29d78258d Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 10 Jan 2019 17:58:59 +0800 Subject: [PATCH 252/470] fix asyncside bug --- .../java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java | 1 + .../java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java | 1 + .../com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java | 2 +- .../java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java | 1 + 4 files changed, 4 insertions(+), 1 deletion(-) diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java index 6e82e4109..ed5cfa558 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java @@ -129,6 +129,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except Object equalObj = input.getField(conValIndex); if(equalObj == null){ resultFuture.complete(null); + return; } refData.put(sideInfo.getEqualFieldList().get(i), equalObj); diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index d8a22772b..226885ed7 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -130,6 +130,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except Object equalObj = input.getField(conValIndex); if (equalObj == null) { resultFuture.complete(null); + return; } basicDBObject.put(sideInfo.getEqualFieldList().get(i), equalObj); } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index 6909b312a..2e3df783c 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -75,8 +75,8 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except Object equalObj = input.getField(conValIndex); if (equalObj == null) { resultFuture.complete(null); + return; } - inputParams.add(equalObj); } diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index 251107287..59ef330c9 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -125,6 +125,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except Object equalObj = input.getField(conValIndex); if(equalObj == null){ resultFuture.complete(null); + return; } keyData.add(sideInfo.getEqualFieldList().get(i)); From 2bc7159c445e7e6b5b7949b867390ab6c0826737 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 11 Jan 2019 17:24:33 +0800 Subject: [PATCH 253/470] =?UTF-8?q?=E5=B0=86=E6=89=80=E6=9C=89=E5=8F=98?= =?UTF-8?q?=E9=87=8F=E6=B7=BB=E5=8A=A0=E5=88=B0env=E4=B8=8A?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/Main.java | 26 ++++++++++++++++++- .../flink/sql/launcher/LauncherMain.java | 19 ++++++++------ 2 files changed, 36 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 415b6e2a9..37ffa4df4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -42,6 +42,7 @@ import org.apache.commons.cli.DefaultParser; import org.apache.commons.cli.Options; import org.apache.commons.io.Charsets; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -52,6 +53,7 @@ import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; import org.apache.flink.calcite.shaded.com.google.common.collect.Sets; import org.apache.flink.client.program.ContextEnvironment; +import org.apache.flink.configuration.Configuration; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamContextEnvironment; @@ -67,6 +69,7 @@ import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.net.URL; import java.net.URLClassLoader; import java.net.URLDecoder; @@ -316,12 +319,33 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en } } - private static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws IOException { + private static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws IOException, NoSuchMethodException { StreamExecutionEnvironment env = !ClusterMode.local.name().equals(deployMode) ? StreamExecutionEnvironment.getExecutionEnvironment() : new MyLocalStreamEnvironment(); env.setParallelism(FlinkUtil.getEnvParallelism(confProperties)); + Configuration globalJobParameters = new Configuration(); + Method method = Configuration.class.getDeclaredMethod("setValueInternal", String.class, Object.class); + method.setAccessible(true); + + confProperties.forEach((key,val) -> { + try { + method.invoke(globalJobParameters, key, val); + } catch (IllegalAccessException e) { + e.printStackTrace(); + } catch (InvocationTargetException e) { + e.printStackTrace(); + } + }); + + ExecutionConfig exeConfig = env.getConfig(); + if(exeConfig.getGlobalJobParameters() == null){ + exeConfig.setGlobalJobParameters(globalJobParameters); + }else if(exeConfig.getGlobalJobParameters() instanceof Configuration){ + ((Configuration) exeConfig.getGlobalJobParameters()).addAll(globalJobParameters); + } + if(FlinkUtil.getMaxEnvParallelism(confProperties) > 0){ env.setMaxParallelism(FlinkUtil.getMaxEnvParallelism(confProperties)); diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index eea52e0af..32813393d 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -23,24 +23,27 @@ import avro.shaded.com.google.common.collect.Lists; import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.TypeReference; +import com.dtstack.flink.sql.ClusterMode; import com.dtstack.flink.sql.Main; import com.dtstack.flink.sql.launcher.perjob.PerJobSubmitter; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; - -import java.io.*; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; - -import com.dtstack.flink.sql.ClusterMode; import org.apache.flink.client.program.PackagedProgramUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.table.shaded.org.apache.commons.lang.StringUtils; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.table.shaded.org.apache.commons.lang.BooleanUtils; +import org.apache.flink.table.shaded.org.apache.commons.lang.StringUtils; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; /** * Date: 2017/2/20 From 5084503088d4b6e2c3112e5c13b20aa506210677 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Fri, 11 Jan 2019 20:57:21 +0800 Subject: [PATCH 254/470] fix rdb asynside bug --- .../flink/sql/side/rdb/async/RdbAsyncReqRow.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index 35f300c26..6989dd64f 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -26,19 +26,15 @@ import io.vertx.core.json.JsonArray; import io.vertx.ext.sql.SQLClient; import io.vertx.ext.sql.SQLConnection; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.math.BigInteger; import java.sql.Timestamp; -import java.util.Collections; import java.util.List; import java.util.Map; @@ -122,18 +118,21 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except int resultSize = rs.result().getResults().size(); if (resultSize > 0) { - for (JsonArray line : rs.result().getResults()) { + List rowList = Lists.newArrayList(); + for (JsonArray line : rs.result().getResults()) { Row row = fillData(input, line); if (openCache()) { cacheContent.add(line); } - resultFuture.complete(Collections.singleton(row)); + rowList.add(row); } if (openCache()) { putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); } + + resultFuture.complete(rowList); } else { dealMissKey(input, resultFuture); if (openCache()) { From 09067d10024845c22052e97a6cffbf1fd8d02a3d Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Fri, 18 Jan 2019 16:58:46 +0800 Subject: [PATCH 255/470] remove kafka offset check,control by front page --- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 4 ---- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 4 ---- .../flink/sql/source/kafka/table/KafkaSourceTableInfo.java | 4 ---- 3 files changed, 12 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 339cf6264..9081b956b 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -121,10 +121,6 @@ public void setOffset(String offset) { public boolean check() { Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); Preconditions.checkNotNull(topic, "kafka of topic is required"); - //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); - Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); - return false; } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 6ce03a19b..113159450 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -121,10 +121,6 @@ public void setTopicIsPattern(Boolean topicIsPattern) { public boolean check() { Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); Preconditions.checkNotNull(topic, "kafka of topic is required"); - //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); - Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); - return false; } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 693b8668d..97b360573 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -122,10 +122,6 @@ public void setOffset(String offset) { public boolean check() { Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); Preconditions.checkNotNull(topic, "kafka of topic is required"); - //Preconditions.checkNotNull(groupId, "kafka of groupId is required"); - Preconditions.checkState(offsetReset.equalsIgnoreCase("latest") - || offsetReset.equalsIgnoreCase("earliest"), "kafka of offsetReset set fail"); - return false; } From d74071cfc9c9aa59fa5c7d7d5784da9bfae698dd Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 23 Mar 2019 14:29:13 +0800 Subject: [PATCH 256/470] =?UTF-8?q?=E6=B7=BB=E5=8A=A0=E5=AF=B9=E8=A7=A3?= =?UTF-8?q?=E6=9E=90json=E5=B5=8C=E5=A5=97=E7=BB=93=E6=9E=84=E7=9A=84?= =?UTF-8?q?=E6=94=AF=E6=8C=81?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../kafka/CustomerJsonDeserialization.java | 36 ++++++++++++++----- .../kafka/CustomerJsonDeserialization.java | 35 +++++++++++++----- .../kafka/CustomerJsonDeserialization.java | 34 +++++++++++++----- 3 files changed, 81 insertions(+), 24 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index ec369b28f..de1918aa9 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -25,7 +25,9 @@ import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; @@ -40,6 +42,7 @@ import java.io.IOException; import java.lang.reflect.Field; import java.util.Iterator; +import java.util.Map; import java.util.Set; import static com.dtstack.flink.sql.metric.MetricConstant.*; @@ -75,6 +78,9 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private boolean firstMsg = true; + private Map nodeAndJsonNodeMapping = Maps.newHashMap(); + + public CustomerJsonDeserialization(TypeInformation typeInfo){ this.typeInfo = typeInfo; @@ -101,9 +107,11 @@ public Row deserialize(byte[] message) throws IOException { numInBytes.inc(message.length); JsonNode root = objectMapper.readTree(message); + parseTree(root, null); Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { - JsonNode node = getIgnoreCase(root, fieldNames[i]); + JsonNode node = nodeAndJsonNodeMapping.get(fieldNames[i]); if (node == null) { if (failOnMissingField) { @@ -132,18 +140,30 @@ public void setFailOnMissingField(boolean failOnMissingField) { this.failOnMissingField = failOnMissingField; } - public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { - Iterator iter = jsonNode.fieldNames(); - while (iter.hasNext()) { - String key1 = iter.next(); - if (key1.equalsIgnoreCase(key)) { - return jsonNode.get(key1); + private void parseTree(JsonNode jsonNode, String prefix){ + nodeAndJsonNodeMapping.clear(); + + Iterator iterator = jsonNode.fieldNames(); + while (iterator.hasNext()){ + String next = iterator.next(); + JsonNode child = jsonNode.get(next); + String nodeKey = getNodeKey(prefix, next); + + if (child.isValueNode()){ + nodeAndJsonNodeMapping.put(nodeKey, child); + }else { + parseTree(child, nodeKey); } } + } - return null; + private String getNodeKey(String prefix, String nodeName){ + if(Strings.isNullOrEmpty(prefix)){ + return nodeName; + } + return prefix + "." + nodeName; } public void setFetcher(AbstractFetcher fetcher) { diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index bcb54159a..09cdc8fec 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -25,7 +25,9 @@ import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; @@ -40,6 +42,7 @@ import java.io.IOException; import java.lang.reflect.Field; import java.util.Iterator; +import java.util.Map; import java.util.Set; import static com.dtstack.flink.sql.metric.MetricConstant.*; @@ -75,6 +78,9 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private boolean firstMsg = true; + private Map nodeAndJsonNodeMapping = Maps.newHashMap(); + + public CustomerJsonDeserialization(TypeInformation typeInfo){ this.typeInfo = typeInfo; @@ -101,9 +107,11 @@ public Row deserialize(byte[] message) throws IOException { numInBytes.inc(message.length); JsonNode root = objectMapper.readTree(message); + parseTree(root, null); Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { - JsonNode node = getIgnoreCase(root, fieldNames[i]); + JsonNode node = nodeAndJsonNodeMapping.get(fieldNames[i]); if (node == null) { if (failOnMissingField) { @@ -132,18 +140,29 @@ public void setFailOnMissingField(boolean failOnMissingField) { this.failOnMissingField = failOnMissingField; } - public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { + private void parseTree(JsonNode jsonNode, String prefix){ + nodeAndJsonNodeMapping.clear(); - Iterator iter = jsonNode.fieldNames(); - while (iter.hasNext()) { - String key1 = iter.next(); - if (key1.equalsIgnoreCase(key)) { - return jsonNode.get(key1); + Iterator iterator = jsonNode.fieldNames(); + while (iterator.hasNext()){ + String next = iterator.next(); + JsonNode child = jsonNode.get(next); + String nodeKey = getNodeKey(prefix, next); + + if (child.isValueNode()){ + nodeAndJsonNodeMapping.put(nodeKey, child); + }else { + parseTree(child, nodeKey); } } + } - return null; + private String getNodeKey(String prefix, String nodeName){ + if(Strings.isNullOrEmpty(prefix)){ + return nodeName; + } + return prefix + "." + nodeName; } public void setFetcher(AbstractFetcher fetcher) { diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 78f1c17ff..c95f23260 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -25,7 +25,9 @@ import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; @@ -40,6 +42,7 @@ import java.io.IOException; import java.lang.reflect.Field; import java.util.Iterator; +import java.util.Map; import java.util.Set; import static com.dtstack.flink.sql.metric.MetricConstant.DT_PARTITION_GROUP; @@ -77,6 +80,8 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private boolean firstMsg = true; + private Map nodeAndJsonNodeMapping = Maps.newHashMap(); + public CustomerJsonDeserialization(TypeInformation typeInfo){ this.typeInfo = typeInfo; @@ -103,9 +108,11 @@ public Row deserialize(byte[] message) throws IOException { numInBytes.inc(message.length); JsonNode root = objectMapper.readTree(message); + parseTree(root, null); Row row = new Row(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { - JsonNode node = getIgnoreCase(root, fieldNames[i]); + JsonNode node = nodeAndJsonNodeMapping.get(fieldNames[i]); if (node == null) { if (failOnMissingField) { @@ -134,18 +141,29 @@ public void setFailOnMissingField(boolean failOnMissingField) { this.failOnMissingField = failOnMissingField; } - public JsonNode getIgnoreCase(JsonNode jsonNode, String key) { + private void parseTree(JsonNode jsonNode, String prefix){ + nodeAndJsonNodeMapping.clear(); + + Iterator iterator = jsonNode.fieldNames(); + while (iterator.hasNext()){ + String next = iterator.next(); + JsonNode child = jsonNode.get(next); + String nodeKey = getNodeKey(prefix, next); - Iterator iter = jsonNode.fieldNames(); - while (iter.hasNext()) { - String key1 = iter.next(); - if (key1.equalsIgnoreCase(key)) { - return jsonNode.get(key1); + if (child.isValueNode()){ + nodeAndJsonNodeMapping.put(nodeKey, child); + }else { + parseTree(child, nodeKey); } } + } - return null; + private String getNodeKey(String prefix, String nodeName){ + if(Strings.isNullOrEmpty(prefix)){ + return nodeName; + } + return prefix + "." + nodeName; } public void setFetcher(AbstractFetcher fetcher) { From 801a40cf20a1087cc758d06d0e2723606e3bfc40 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 23 Mar 2019 15:33:41 +0800 Subject: [PATCH 257/470] =?UTF-8?q?=E6=B7=BB=E5=8A=A0=E5=AF=B9=E8=A7=A3?= =?UTF-8?q?=E6=9E=90json=E5=B5=8C=E5=A5=97=E7=BB=93=E6=9E=84=E7=9A=84?= =?UTF-8?q?=E6=94=AF=E6=8C=81=EF=BC=9A=20=E5=88=97=E6=A0=BC=E5=BC=8F=20a.b?= =?UTF-8?q?=20int=20as=20newfield?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flink/sql/table/TableInfo.java | 17 +++++++++++ .../kafka/CustomerJsonDeserialization.java | 24 +++++++++++++-- .../flink/sql/source/kafka/KafkaSource.java | 4 +-- .../source/kafka/table/KafkaSourceParser.java | 26 ++++++++++++++++ .../kafka/CustomerJsonDeserialization.java | 24 +++++++++++++-- .../flink/sql/source/kafka/KafkaSource.java | 4 +-- .../source/kafka/table/KafkaSourceParser.java | 30 +++++++++++++++++++ .../kafka/CustomerJsonDeserialization.java | 25 ++++++++++++++-- .../flink/sql/source/kafka/KafkaSource.java | 4 +-- .../source/kafka/table/KafkaSourceParser.java | 30 +++++++++++++++++++ 10 files changed, 173 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java index b56b790ea..57f94e2e0 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java @@ -21,9 +21,11 @@ package com.dtstack.flink.sql.table; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import java.io.Serializable; import java.util.List; +import java.util.Map; /** * Reason: @@ -48,6 +50,9 @@ public abstract class TableInfo implements Serializable { private final List fieldList = Lists.newArrayList(); + /**key:别名, value: realField */ + private Map physicalFields = Maps.newHashMap(); + private final List fieldTypeList = Lists.newArrayList(); private final List fieldClassList = Lists.newArrayList(); @@ -114,6 +119,10 @@ public void addField(String fieldName){ fieldList.add(fieldName); } + public void addPhysicalMappings(String aliasName, String physicalFieldName){ + physicalFields.put(aliasName, physicalFieldName); + } + public void addFieldClass(Class fieldClass){ fieldClassList.add(fieldClass); } @@ -146,6 +155,14 @@ public List getFieldClassList() { return fieldClassList; } + public Map getPhysicalFields() { + return physicalFields; + } + + public void setPhysicalFields(Map physicalFields) { + this.physicalFields = physicalFields; + } + public void finish(){ this.fields = fieldList.toArray(new String[fieldList.size()]); this.fieldClasses = fieldClassList.toArray(new Class[fieldClassList.size()]); diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index de1918aa9..9ee70af04 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -30,6 +30,7 @@ import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.types.Row; @@ -80,13 +81,17 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private Map nodeAndJsonNodeMapping = Maps.newHashMap(); + private Map rowAndFieldMapping; - public CustomerJsonDeserialization(TypeInformation typeInfo){ + + public CustomerJsonDeserialization(TypeInformation typeInfo, Map rowAndFieldMapping){ this.typeInfo = typeInfo; this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + + this.rowAndFieldMapping= rowAndFieldMapping; } @Override @@ -111,7 +116,7 @@ public Row deserialize(byte[] message) throws IOException { Row row = new Row(fieldNames.length); for (int i = 0; i < fieldNames.length; i++) { - JsonNode node = nodeAndJsonNodeMapping.get(fieldNames[i]); + JsonNode node = getIgnoreCase(fieldNames[i]); if (node == null) { if (failOnMissingField) { @@ -133,6 +138,8 @@ public Row deserialize(byte[] message) throws IOException { //add metric of dirty data dirtyDataCounter.inc(); return null; + }finally { + nodeAndJsonNodeMapping.clear(); } } @@ -140,9 +147,20 @@ public void setFailOnMissingField(boolean failOnMissingField) { this.failOnMissingField = failOnMissingField; } + private JsonNode getIgnoreCase(String key) { + String nodeMappingKey = rowAndFieldMapping.getOrDefault(key, key); + JsonNode node = nodeAndJsonNodeMapping.get(nodeMappingKey); + JsonNodeType nodeType = node.getNodeType(); + + if (nodeType == JsonNodeType.ARRAY){ + throw new IllegalStateException("Unsupported type information array .") ; + } + + return node; + } + private void parseTree(JsonNode jsonNode, String prefix){ - nodeAndJsonNodeMapping.clear(); Iterator iterator = jsonNode.fieldNames(); while (iterator.hasNext()){ diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index b08485e0b..3cf6fdf90 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -84,10 +84,10 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv FlinkKafkaConsumer09 kafkaSrc; if (BooleanUtils.isTrue(kafka09SourceTableInfo.getTopicIsPattern())) { kafkaSrc = new CustomerKafka09Consumer(Pattern.compile(topicName), - new CustomerJsonDeserialization(typeInformation), props); + new CustomerJsonDeserialization(typeInformation, kafka09SourceTableInfo.getPhysicalFields()), props); } else { kafkaSrc = new CustomerKafka09Consumer(topicName, - new CustomerJsonDeserialization(typeInformation), props); + new CustomerJsonDeserialization(typeInformation, kafka09SourceTableInfo.getPhysicalFields()), props); } //earliest,latest diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 664e8c4b1..4c34ea1b9 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -21,10 +21,14 @@ package com.dtstack.flink.sql.source.kafka.table; import com.dtstack.flink.sql.table.AbsSourceParser; +import com.dtstack.flink.sql.table.SourceTableInfo; import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.ClassUtil; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; /** * Reason: @@ -35,6 +39,28 @@ public class KafkaSourceParser extends AbsSourceParser { + private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; + + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + + static { + keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); + + keyHandlerMap.put(KAFKA_NEST_FIELD_KEY, KafkaSourceParser::dealNestField); + } + + static void dealNestField(Matcher matcher, TableInfo tableInfo) { + String physicalField = matcher.group(1); + String fieldType = matcher.group(3); + String mappingField = matcher.group(4); + Class fieldClass= ClassUtil.stringConvertClass(fieldType); + + tableInfo.addPhysicalMappings(mappingField, physicalField); + tableInfo.addField(mappingField); + tableInfo.addFieldClass(fieldClass); + tableInfo.addFieldType(fieldType); + } + @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 09cdc8fec..d72083762 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -30,6 +30,7 @@ import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.types.Row; @@ -80,13 +81,16 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private Map nodeAndJsonNodeMapping = Maps.newHashMap(); + private Map rowAndFieldMapping; - public CustomerJsonDeserialization(TypeInformation typeInfo){ + public CustomerJsonDeserialization(TypeInformation typeInfo, Map rowAndFieldMapping){ this.typeInfo = typeInfo; this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + + this.rowAndFieldMapping= rowAndFieldMapping; } @Override @@ -111,7 +115,7 @@ public Row deserialize(byte[] message) throws IOException { Row row = new Row(fieldNames.length); for (int i = 0; i < fieldNames.length; i++) { - JsonNode node = nodeAndJsonNodeMapping.get(fieldNames[i]); + JsonNode node = getIgnoreCase(fieldNames[i]); if (node == null) { if (failOnMissingField) { @@ -133,15 +137,29 @@ public Row deserialize(byte[] message) throws IOException { //add metric of dirty data dirtyDataCounter.inc(); return null; + }finally { + nodeAndJsonNodeMapping.clear(); } } + public JsonNode getIgnoreCase(String key) { + String nodeMappingKey = rowAndFieldMapping.getOrDefault(key, key); + JsonNode node = nodeAndJsonNodeMapping.get(nodeMappingKey); + JsonNodeType nodeType = node.getNodeType(); + + if (nodeType==JsonNodeType.ARRAY){ + throw new IllegalStateException("Unsupported type information array .") ; + } + + return node; + } + + public void setFailOnMissingField(boolean failOnMissingField) { this.failOnMissingField = failOnMissingField; } private void parseTree(JsonNode jsonNode, String prefix){ - nodeAndJsonNodeMapping.clear(); Iterator iterator = jsonNode.fieldNames(); while (iterator.hasNext()){ diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 523eb25dc..9a3c63c9a 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -85,10 +85,10 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv FlinkKafkaConsumer010 kafkaSrc; if (BooleanUtils.isTrue(kafka010SourceTableInfo.getTopicIsPattern())) { kafkaSrc = new CustomerKafka010Consumer(Pattern.compile(topicName), - new CustomerJsonDeserialization(typeInformation), props); + new CustomerJsonDeserialization(typeInformation, kafka010SourceTableInfo.getPhysicalFields()), props); } else { kafkaSrc = new CustomerKafka010Consumer(topicName, - new CustomerJsonDeserialization(typeInformation), props); + new CustomerJsonDeserialization(typeInformation, kafka010SourceTableInfo.getPhysicalFields()), props); } //earliest,latest diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 4a2590536..f5077c8ee 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -22,9 +22,12 @@ import com.dtstack.flink.sql.table.AbsSourceParser; import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.ClassUtil; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; /** * Reason: @@ -35,6 +38,33 @@ public class KafkaSourceParser extends AbsSourceParser { + private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; + + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + + static { + keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); + + keyHandlerMap.put(KAFKA_NEST_FIELD_KEY, KafkaSourceParser::dealNestField); + } + + /** + * add parser for alias field + * @param matcher + * @param tableInfo + */ + static void dealNestField(Matcher matcher, TableInfo tableInfo) { + String physicalField = matcher.group(1); + String fieldType = matcher.group(3); + String mappingField = matcher.group(4); + Class fieldClass= ClassUtil.stringConvertClass(fieldType); + + tableInfo.addPhysicalMappings(mappingField, physicalField); + tableInfo.addField(mappingField); + tableInfo.addFieldClass(fieldClass); + tableInfo.addFieldType(fieldType); + } + @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index c95f23260..deb600337 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -30,6 +30,7 @@ import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.types.Row; @@ -82,12 +83,17 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private Map nodeAndJsonNodeMapping = Maps.newHashMap(); - public CustomerJsonDeserialization(TypeInformation typeInfo){ + private Map rowAndFieldMapping; + + + public CustomerJsonDeserialization(TypeInformation typeInfo, Map rowAndFieldMapping){ this.typeInfo = typeInfo; this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + + this.rowAndFieldMapping= rowAndFieldMapping; } @Override @@ -112,7 +118,7 @@ public Row deserialize(byte[] message) throws IOException { Row row = new Row(fieldNames.length); for (int i = 0; i < fieldNames.length; i++) { - JsonNode node = nodeAndJsonNodeMapping.get(fieldNames[i]); + JsonNode node = getIgnoreCase(fieldNames[i]); if (node == null) { if (failOnMissingField) { @@ -134,7 +140,21 @@ public Row deserialize(byte[] message) throws IOException { //add metric of dirty data dirtyDataCounter.inc(); return null; + }finally { + nodeAndJsonNodeMapping.clear(); + } + } + + public JsonNode getIgnoreCase(String key) { + String nodeMappingKey = rowAndFieldMapping.getOrDefault(key, key); + JsonNode node = nodeAndJsonNodeMapping.get(nodeMappingKey); + JsonNodeType nodeType = node.getNodeType(); + + if (nodeType==JsonNodeType.ARRAY){ + throw new IllegalStateException("Unsupported type information array .") ; } + + return node; } public void setFailOnMissingField(boolean failOnMissingField) { @@ -142,7 +162,6 @@ public void setFailOnMissingField(boolean failOnMissingField) { } private void parseTree(JsonNode jsonNode, String prefix){ - nodeAndJsonNodeMapping.clear(); Iterator iterator = jsonNode.fieldNames(); while (iterator.hasNext()){ diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 5c9f5eb49..ec795cd4e 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -85,10 +85,10 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv FlinkKafkaConsumer011 kafkaSrc; if (BooleanUtils.isTrue(kafka011SourceTableInfo.getTopicIsPattern())) { kafkaSrc = new CustomerKafka011Consumer(Pattern.compile(topicName), - new CustomerJsonDeserialization(typeInformation), props); + new CustomerJsonDeserialization(typeInformation, kafka011SourceTableInfo.getPhysicalFields()), props); } else { kafkaSrc = new CustomerKafka011Consumer(topicName, - new CustomerJsonDeserialization(typeInformation), props); + new CustomerJsonDeserialization(typeInformation, kafka011SourceTableInfo.getPhysicalFields()), props); } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 47d453adc..f13960862 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -22,9 +22,12 @@ import com.dtstack.flink.sql.table.AbsSourceParser; import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.ClassUtil; import com.dtstack.flink.sql.util.MathUtil; import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; /** * Reason: @@ -35,6 +38,33 @@ public class KafkaSourceParser extends AbsSourceParser { + private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; + + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + + static { + keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); + + keyHandlerMap.put(KAFKA_NEST_FIELD_KEY, KafkaSourceParser::dealNestField); + } + + /** + * add parser for alias field + * @param matcher + * @param tableInfo + */ + static void dealNestField(Matcher matcher, TableInfo tableInfo) { + String physicalField = matcher.group(1); + String fieldType = matcher.group(3); + String mappingField = matcher.group(4); + Class fieldClass= ClassUtil.stringConvertClass(fieldType); + + tableInfo.addPhysicalMappings(mappingField, physicalField); + tableInfo.addField(mappingField); + tableInfo.addFieldClass(fieldClass); + tableInfo.addFieldType(fieldType); + } + @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { From 866ab8f069bab0bde9a018dcbb3d9d6d9f382b8f Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Fri, 29 Mar 2019 11:26:30 +0800 Subject: [PATCH 258/470] check kafka list --- .../com/dtstack/flink/sql/table/AbsTableParser.java | 2 +- .../sql/source/kafka/table/KafkaSourceParser.java | 9 +++++++-- .../sql/source/kafka/table/KafkaSourceParser.java | 10 ++++++++-- .../sql/source/kafka/table/KafkaSourceParser.java | 9 +++++++-- 4 files changed, 23 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index 4fa4f0597..ddbf269e9 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -56,7 +56,7 @@ protected boolean fieldNameNeedsUpperCase() { return true; } - public abstract TableInfo getTableInfo(String tableName, String fieldsInfo, Map props); + public abstract TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) throws Exception; public boolean dealKeyPattern(String fieldRow, TableInfo tableInfo){ for(Map.Entry keyPattern : keyPatternMap.entrySet()){ diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 4c34ea1b9..1bace9fda 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -62,14 +62,19 @@ static void dealNestField(Matcher matcher, TableInfo tableInfo) { } @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) throws Exception { KafkaSourceTableInfo kafka09SourceTableInfo = new KafkaSourceTableInfo(); kafka09SourceTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, kafka09SourceTableInfo); kafka09SourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSourceTableInfo.PARALLELISM_KEY.toLowerCase()))); - kafka09SourceTableInfo.setBootstrapServers(MathUtil.getString(props.get(KafkaSourceTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase()))); + String bootstrapServer = MathUtil.getString(props.get(KafkaSourceTableInfo.GROUPID_KEY.toLowerCase())); + if (bootstrapServer == null || bootstrapServer.trim().equals("")){ + throw new Exception("BootstrapServers can not be empty!"); + } else { + kafka09SourceTableInfo.setBootstrapServers(bootstrapServer); + } kafka09SourceTableInfo.setGroupId(MathUtil.getString(props.get(KafkaSourceTableInfo.GROUPID_KEY.toLowerCase()))); kafka09SourceTableInfo.setTopic(MathUtil.getString(props.get(KafkaSourceTableInfo.TOPIC_KEY.toLowerCase()))); kafka09SourceTableInfo.setOffsetReset(MathUtil.getString(props.get(KafkaSourceTableInfo.OFFSETRESET_KEY.toLowerCase()))); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index f5077c8ee..6472394a9 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -66,14 +66,20 @@ static void dealNestField(Matcher matcher, TableInfo tableInfo) { } @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) throws Exception { KafkaSourceTableInfo kafka10SourceTableInfo = new KafkaSourceTableInfo(); kafka10SourceTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, kafka10SourceTableInfo); kafka10SourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSourceTableInfo.PARALLELISM_KEY.toLowerCase()))); - kafka10SourceTableInfo.setBootstrapServers(MathUtil.getString(props.get(KafkaSourceTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase()))); + + String bootstrapServer = MathUtil.getString(props.get(KafkaSourceTableInfo.GROUPID_KEY.toLowerCase())); + if (bootstrapServer == null || bootstrapServer.trim().equals("")){ + throw new Exception("BootstrapServers can not be empty!"); + } else { + kafka10SourceTableInfo.setBootstrapServers(bootstrapServer); + } kafka10SourceTableInfo.setGroupId(MathUtil.getString(props.get(KafkaSourceTableInfo.GROUPID_KEY.toLowerCase()))); kafka10SourceTableInfo.setTopic(MathUtil.getString(props.get(KafkaSourceTableInfo.TOPIC_KEY.toLowerCase()))); kafka10SourceTableInfo.setOffsetReset(MathUtil.getString(props.get(KafkaSourceTableInfo.OFFSETRESET_KEY.toLowerCase()))); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index f13960862..ff53dfb3f 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -66,14 +66,19 @@ static void dealNestField(Matcher matcher, TableInfo tableInfo) { } @Override - public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) throws Exception { KafkaSourceTableInfo kafka11SourceTableInfo = new KafkaSourceTableInfo(); kafka11SourceTableInfo.setName(tableName); parseFieldsInfo(fieldsInfo, kafka11SourceTableInfo); kafka11SourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSourceTableInfo.PARALLELISM_KEY.toLowerCase()))); - kafka11SourceTableInfo.setBootstrapServers(MathUtil.getString(props.get(KafkaSourceTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase()))); + String bootstrapServer = MathUtil.getString(props.get(KafkaSourceTableInfo.GROUPID_KEY.toLowerCase())); + if (bootstrapServer == null || bootstrapServer.trim().equals("")){ + throw new Exception("BootstrapServers can not be empty!"); + } else { + kafka11SourceTableInfo.setBootstrapServers(bootstrapServer); + } kafka11SourceTableInfo.setGroupId(MathUtil.getString(props.get(KafkaSourceTableInfo.GROUPID_KEY.toLowerCase()))); kafka11SourceTableInfo.setTopic(MathUtil.getString(props.get(KafkaSourceTableInfo.TOPIC_KEY.toLowerCase()))); kafka11SourceTableInfo.setOffsetReset(MathUtil.getString(props.get(KafkaSourceTableInfo.OFFSETRESET_KEY.toLowerCase()))); From dcff87d7f28b00b73aef8a18f5e8c2ffba397bce Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 1 Apr 2019 14:40:58 +0800 Subject: [PATCH 259/470] check BOOTSTRAPSERVERS_KEY --- .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 +- .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 +- .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 1bace9fda..0f8963574 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -69,7 +69,7 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Wed, 3 Apr 2019 17:07:18 +0800 Subject: [PATCH 260/470] support offsetnum --- .../com/dtstack/flink/sql/source/kafka/KafkaSource.java | 6 +++++- .../com/dtstack/flink/sql/source/kafka/KafkaSource.java | 6 +++++- .../com/dtstack/flink/sql/source/kafka/KafkaSource.java | 6 +++++- 3 files changed, 15 insertions(+), 3 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 3cf6fdf90..326b2b7c1 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -68,7 +68,11 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv Properties props = new Properties(); props.setProperty("bootstrap.servers", kafka09SourceTableInfo.getBootstrapServers()); - props.setProperty("auto.offset.reset", kafka09SourceTableInfo.getOffsetReset()); + if (DtStringUtil.isJosn(kafka09SourceTableInfo.getOffsetReset())){ + props.setProperty("auto.offset.reset", "none"); + } else { + props.setProperty("auto.offset.reset", kafka09SourceTableInfo.getOffsetReset()); + } if (StringUtils.isNotBlank(kafka09SourceTableInfo.getGroupId())){ props.setProperty("group.id", kafka09SourceTableInfo.getGroupId()); } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 9a3c63c9a..052dd5339 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -68,7 +68,11 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv Properties props = new Properties(); props.setProperty("bootstrap.servers", kafka010SourceTableInfo.getBootstrapServers()); - props.setProperty("auto.offset.reset", kafka010SourceTableInfo.getOffsetReset()); + if (DtStringUtil.isJosn(kafka010SourceTableInfo.getOffsetReset())){ + props.setProperty("auto.offset.reset", "none"); + } else { + props.setProperty("auto.offset.reset", kafka010SourceTableInfo.getOffsetReset()); + } if (StringUtils.isNotBlank(kafka010SourceTableInfo.getGroupId())){ props.setProperty("group.id", kafka010SourceTableInfo.getGroupId()); } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index ec795cd4e..54e4190ac 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -68,7 +68,11 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv Properties props = new Properties(); props.setProperty("bootstrap.servers", kafka011SourceTableInfo.getBootstrapServers()); - props.setProperty("auto.offset.reset", kafka011SourceTableInfo.getOffsetReset()); + if (DtStringUtil.isJosn(kafka011SourceTableInfo.getOffsetReset())){ + props.setProperty("auto.offset.reset", "none"); + } else { + props.setProperty("auto.offset.reset", kafka011SourceTableInfo.getOffsetReset()); + } if (StringUtils.isNotBlank(kafka011SourceTableInfo.getGroupId())){ props.setProperty("group.id", kafka011SourceTableInfo.getGroupId()); } From 869390530fe2162b4c7f1e631be13773f327e1bd Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 4 Apr 2019 11:57:41 +0800 Subject: [PATCH 261/470] =?UTF-8?q?=E4=BF=AE=E6=94=B9oracle=20=E8=8E=B7?= =?UTF-8?q?=E5=8F=96=E7=B4=A2=E5=BC=95=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index f288e50c0..93391b713 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -62,7 +62,7 @@ public boolean isReplaceInsertQuery() throws SQLException { */ public void fillRealIndexes() throws SQLException { Map> map = Maps.newHashMap(); - ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, null, getTableName(), true, false); + ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, null, getTableName(), true, true); while (rs.next()) { String indexName = rs.getString("INDEX_NAME"); From e8b00c4cec0b1daf8681fe2fde703bbadc0004de Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 4 Apr 2019 14:56:01 +0800 Subject: [PATCH 262/470] =?UTF-8?q?(1)=E4=BF=AE=E6=94=B9oracle=20=E6=8B=BC?= =?UTF-8?q?=E6=8E=A5sql=EF=BC=8C=E5=AF=B9=E8=A1=A8=E5=92=8C=E5=AD=97?= =?UTF-8?q?=E6=AE=B5=20=E6=B7=BB=E5=8A=A0=20\"\"=20(2)=E4=BF=AE=E6=94=B9js?= =?UTF-8?q?on=E8=A7=A3=E6=9E=90=E7=9A=84=E6=97=B6=E5=80=99=E9=81=87?= =?UTF-8?q?=E5=88=B0=E6=9C=AA=E5=AE=9A=E4=B9=89=E7=9A=84key=E7=9A=84?= =?UTF-8?q?=E6=97=B6=E5=80=99=E4=B8=8D=E5=86=8D=E6=8A=9B=E5=BC=82=E5=B8=B8?= =?UTF-8?q?=EF=BC=8C=E8=BF=94=E5=9B=9Enull?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../sql/source/kafka/CustomerJsonDeserialization.java | 4 ++++ .../sql/source/kafka/CustomerJsonDeserialization.java | 5 +++++ .../sql/source/kafka/CustomerJsonDeserialization.java | 4 ++++ .../com/dtstack/flink/sql/sink/oracle/OracleSink.java | 10 +++++++++- 4 files changed, 22 insertions(+), 1 deletion(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 9ee70af04..574fe65cd 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -150,6 +150,10 @@ public void setFailOnMissingField(boolean failOnMissingField) { private JsonNode getIgnoreCase(String key) { String nodeMappingKey = rowAndFieldMapping.getOrDefault(key, key); JsonNode node = nodeAndJsonNodeMapping.get(nodeMappingKey); + if(node == null){ + return null; + } + JsonNodeType nodeType = node.getNodeType(); if (nodeType == JsonNodeType.ARRAY){ diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index d72083762..321cc01aa 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -145,6 +145,11 @@ public Row deserialize(byte[] message) throws IOException { public JsonNode getIgnoreCase(String key) { String nodeMappingKey = rowAndFieldMapping.getOrDefault(key, key); JsonNode node = nodeAndJsonNodeMapping.get(nodeMappingKey); + + if(node == null){ + return null; + } + JsonNodeType nodeType = node.getNodeType(); if (nodeType==JsonNodeType.ARRAY){ diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index deb600337..123d9d5cc 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -148,6 +148,10 @@ public Row deserialize(byte[] message) throws IOException { public JsonNode getIgnoreCase(String key) { String nodeMappingKey = rowAndFieldMapping.getOrDefault(key, key); JsonNode node = nodeAndJsonNodeMapping.get(nodeMappingKey); + if(node == null){ + return null; + } + JsonNodeType nodeType = node.getNodeType(); if (nodeType==JsonNodeType.ARRAY){ diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index 2ca020c54..d8669b11f 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -22,6 +22,7 @@ import com.dtstack.flink.sql.sink.rdb.format.ExtendOutputFormat; import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; import java.util.ArrayList; import java.util.Iterator; @@ -54,8 +55,14 @@ public void buildSql(String tableName, List fields) { } private void buildInsertSql(String tableName, List fields) { + + tableName = quoteTable(tableName); String sqlTmp = "insert into " + tableName + " (${fields}) values (${placeholder})"; - String fieldsStr = StringUtils.join(fields, ","); + + List adaptFields = Lists.newArrayList(); + fields.forEach(field -> adaptFields.add(quoteColumn(field))); + + String fieldsStr = StringUtils.join(adaptFields, ","); String placeholder = ""; for (String fieldName : fields) { @@ -68,6 +75,7 @@ private void buildInsertSql(String tableName, List fields) { @Override public String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField) { + tableName = quoteTable(tableName); return "MERGE INTO " + tableName + " T1 USING " + "(" + makeValues(fieldNames) + ") T2 ON (" + updateKeySql(realIndexes) + ") WHEN MATCHED THEN UPDATE SET " From 1326ef90da9de69d52be562692a49fbf7ece71f0 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Mon, 8 Apr 2019 20:14:19 +0800 Subject: [PATCH 263/470] add kafkaSink --- .../flink/sql/sink/StreamSinkFactory.java | 7 +- .../kafka/CustomerKafka09JsonTableSink.java | 78 ++++++++++++++++++ kafka09/pom.xml | 8 ++ .../kafka/CustomerKafka10JsonTableSink.java | 81 +++++++++++++++++++ kafka10/pom.xml | 6 ++ .../kafka/CustomerKafka11JsonTableSink.java | 81 +++++++++++++++++++ kafka11/pom.xml | 7 ++ 7 files changed, 265 insertions(+), 3 deletions(-) create mode 100644 kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka09JsonTableSink.java create mode 100644 kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java create mode 100644 kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java diff --git a/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java b/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java index 9ef30f97c..ebda80c8a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java @@ -51,9 +51,9 @@ public static AbsTableParser getSqlParser(String pluginType, String sqlRootDir) DtClassLoader dtClassLoader = (DtClassLoader) classLoader; String pluginJarPath = PluginUtil.getJarFileDirPath(String.format(DIR_NAME_FORMAT, pluginType), sqlRootDir); - PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); - String className = PluginUtil.getSqlParserClassName(pluginType, CURR_TYPE); + String typeNoVersion = DtStringUtil.getPluginTypeWithoutVersion(pluginType); + String className = PluginUtil.getSqlParserClassName(typeNoVersion, CURR_TYPE); Class targetParser = dtClassLoader.loadClass(className); if(!AbsTableParser.class.isAssignableFrom(targetParser)){ @@ -77,7 +77,8 @@ public static TableSink getTableSink(TargetTableInfo targetTableInfo, String loc PluginUtil.addPluginJar(pluginJarDirPath, dtClassLoader); - String className = PluginUtil.getGenerClassName(pluginType, CURR_TYPE); + String typeNoVersion = DtStringUtil.getPluginTypeWithoutVersion(pluginType); + String className = PluginUtil.getGenerClassName(typeNoVersion, CURR_TYPE); Class sinkClass = dtClassLoader.loadClass(className); if(!IStreamSinkGener.class.isAssignableFrom(sinkClass)){ diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka09JsonTableSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka09JsonTableSink.java new file mode 100644 index 000000000..998d163e2 --- /dev/null +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka09JsonTableSink.java @@ -0,0 +1,78 @@ +/** + * 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 com.dtstack.flink.sql.sink.kafka; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.connectors.kafka.*; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; +import org.apache.flink.table.util.TableConnectorUtil; +import org.apache.flink.types.Row; + +import java.util.Properties; + +/** + * Reason: add schema info + * Date: 2019/4/8 + * Company: www.dtstack.com + * + * @author maqi + */ +public class CustomerKafka09JsonTableSink extends KafkaJsonTableSink { + + + protected SerializationSchema schema; + + public CustomerKafka09JsonTableSink(String topic, Properties properties, SerializationSchema schema) { + super(topic, properties, new FlinkFixedPartitioner<>()); + this.schema = schema; + } + + public CustomerKafka09JsonTableSink(String topic, Properties properties, FlinkKafkaPartitioner partitioner, SerializationSchema schema) { + super(topic, properties, partitioner); + this.schema = schema; + } + + + @Deprecated + public CustomerKafka09JsonTableSink(String topic, Properties properties, KafkaPartitioner partitioner, SerializationSchema schema) { + super(topic, properties, new FlinkKafkaDelegatePartitioner<>(partitioner)); + this.schema = schema; + } + + @Override + protected FlinkKafkaProducerBase createKafkaProducer(String topic, Properties properties, SerializationSchema serializationSchema, FlinkKafkaPartitioner partitioner) { + return new FlinkKafkaProducer09<>(topic, serializationSchema, properties, partitioner); + } + + @Override + protected Kafka09JsonTableSink createCopy() { + return new Kafka09JsonTableSink(topic, properties, partitioner); + } + + @Override + public void emitDataStream(DataStream dataStream) { + FlinkKafkaProducerBase kafkaProducer = createKafkaProducer(topic, properties, schema, partitioner); + // always enable flush on checkpoint to achieve at-least-once if query runs with checkpointing enabled. + kafkaProducer.setFlushOnCheckpoint(true); + dataStream.addSink(kafkaProducer).name(TableConnectorUtil.generateRuntimeName(this.getClass(), fieldNames)); + } +} diff --git a/kafka09/pom.xml b/kafka09/pom.xml index 457a8d387..1ae761e9b 100644 --- a/kafka09/pom.xml +++ b/kafka09/pom.xml @@ -15,9 +15,17 @@ kafka09-source + kafka09-sink + + + org.apache.flink + flink-connector-kafka-0.9_2.11 + ${flink.version} + + junit junit diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java new file mode 100644 index 000000000..303892c1e --- /dev/null +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java @@ -0,0 +1,81 @@ +/** + * 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 com.dtstack.flink.sql.sink.kafka; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase; +import org.apache.flink.streaming.connectors.kafka.Kafka09JsonTableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaJsonTableSink; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; +import org.apache.flink.table.util.TableConnectorUtil; +import org.apache.flink.types.Row; + +import java.util.Properties; + +/** + * Reason: add schema info + * Date: 2019/4/8 + * Company: www.dtstack.com + * + * @author maqi + */ +public class CustomerKafka10JsonTableSink extends KafkaJsonTableSink { + + + protected SerializationSchema schema; + + public CustomerKafka10JsonTableSink(String topic, Properties properties, SerializationSchema schema) { + super(topic, properties, new FlinkFixedPartitioner<>()); + this.schema = schema; + } + + public CustomerKafka10JsonTableSink(String topic, Properties properties, FlinkKafkaPartitioner partitioner, SerializationSchema schema) { + super(topic, properties, partitioner); + this.schema = schema; + } + + + @Deprecated + public CustomerKafka10JsonTableSink(String topic, Properties properties, KafkaPartitioner partitioner, SerializationSchema schema) { + super(topic, properties, new FlinkKafkaDelegatePartitioner<>(partitioner)); + this.schema = schema; + } + + @Override + protected FlinkKafkaProducerBase createKafkaProducer(String topic, Properties properties, SerializationSchema serializationSchema, FlinkKafkaPartitioner partitioner) { + return new FlinkKafkaProducer010(topic, serializationSchema, properties, partitioner); + } + + @Override + protected Kafka09JsonTableSink createCopy() { + return new Kafka09JsonTableSink(topic, properties, partitioner); + } + + @Override + public void emitDataStream(DataStream dataStream) { + FlinkKafkaProducerBase kafkaProducer = createKafkaProducer(topic, properties, schema, partitioner); + // always enable flush on checkpoint to achieve at-least-once if query runs with checkpointing enabled. + kafkaProducer.setFlushOnCheckpoint(true); + dataStream.addSink(kafkaProducer).name(TableConnectorUtil.generateRuntimeName(this.getClass(), fieldNames)); + } +} diff --git a/kafka10/pom.xml b/kafka10/pom.xml index 9c8c78bba..d8ad7370f 100644 --- a/kafka10/pom.xml +++ b/kafka10/pom.xml @@ -14,9 +14,15 @@ kafka10-source + kafka10-sink + + org.apache.flink + flink-connector-kafka-0.10_2.11 + ${flink.version} + junit junit diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java new file mode 100644 index 000000000..05d15949a --- /dev/null +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java @@ -0,0 +1,81 @@ +/** + * 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 com.dtstack.flink.sql.sink.kafka; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase; +import org.apache.flink.streaming.connectors.kafka.Kafka09JsonTableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaJsonTableSink; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; +import org.apache.flink.table.util.TableConnectorUtil; +import org.apache.flink.types.Row; + +import java.util.Properties; + +/** + * Reason: add schema info + * Date: 2019/4/8 + * Company: www.dtstack.com + * + * @author maqi + */ +public class CustomerKafka11JsonTableSink extends KafkaJsonTableSink { + + + protected SerializationSchema schema; + + public CustomerKafka11JsonTableSink(String topic, Properties properties, SerializationSchema schema) { + super(topic, properties, new FlinkFixedPartitioner<>()); + this.schema = schema; + } + + public CustomerKafka11JsonTableSink(String topic, Properties properties, FlinkKafkaPartitioner partitioner, SerializationSchema schema) { + super(topic, properties, partitioner); + this.schema = schema; + } + + + @Deprecated + public CustomerKafka11JsonTableSink(String topic, Properties properties, KafkaPartitioner partitioner, SerializationSchema schema) { + super(topic, properties, new FlinkKafkaDelegatePartitioner<>(partitioner)); + this.schema = schema; + } + + @Override + protected FlinkKafkaProducerBase createKafkaProducer(String topic, Properties properties, SerializationSchema serializationSchema, FlinkKafkaPartitioner partitioner) { + return new FlinkKafkaProducer010(topic, serializationSchema, properties, partitioner); + } + + @Override + protected Kafka09JsonTableSink createCopy() { + return new Kafka09JsonTableSink(topic, properties, partitioner); + } + + @Override + public void emitDataStream(DataStream dataStream) { + FlinkKafkaProducerBase kafkaProducer = createKafkaProducer(topic, properties, schema, partitioner); + // always enable flush on checkpoint to achieve at-least-once if query runs with checkpointing enabled. + kafkaProducer.setFlushOnCheckpoint(true); + dataStream.addSink(kafkaProducer).name(TableConnectorUtil.generateRuntimeName(this.getClass(), fieldNames)); + } +} diff --git a/kafka11/pom.xml b/kafka11/pom.xml index 841e20be9..90972fa82 100644 --- a/kafka11/pom.xml +++ b/kafka11/pom.xml @@ -14,9 +14,16 @@ kafka11-source + kafka11-sink + + org.apache.flink + flink-connector-kafka-0.11_2.11 + ${flink.version} + + junit junit From ba4df87cc73eb3f9031d784ef8d96d8821e3b464 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Mon, 8 Apr 2019 20:14:59 +0800 Subject: [PATCH 264/470] add kafkaSink --- kafka09/kafka09-sink/pom.xml | 90 ++++++++++++++ .../flink/sql/sink/kafka/KafkaSink.java | 106 +++++++++++++++++ .../sql/sink/kafka/table/KafkaSinkParser.java | 48 ++++++++ .../sink/kafka/table/KafkaSinkTableInfo.java | 80 +++++++++++++ kafka10/kafka10-sink/pom.xml | 89 ++++++++++++++ .../flink/sql/sink/kafka/KafkaSink.java | 110 ++++++++++++++++++ .../sql/sink/kafka/table/KafkaSinkParser.java | 48 ++++++++ .../sink/kafka/table/KafkaSinkTableInfo.java | 82 +++++++++++++ kafka11/kafka11-sink/pom.xml | 89 ++++++++++++++ .../flink/sql/sink/kafka/KafkaSink.java | 108 +++++++++++++++++ .../sql/sink/kafka/table/KafkaSinkParser.java | 48 ++++++++ .../sink/kafka/table/KafkaSinkTableInfo.java | 80 +++++++++++++ 12 files changed, 978 insertions(+) create mode 100644 kafka09/kafka09-sink/pom.xml create mode 100644 kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java create mode 100644 kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java create mode 100644 kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java create mode 100644 kafka10/kafka10-sink/pom.xml create mode 100644 kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java create mode 100644 kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java create mode 100644 kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java create mode 100644 kafka11/kafka11-sink/pom.xml create mode 100644 kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java create mode 100644 kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java create mode 100644 kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java diff --git a/kafka09/kafka09-sink/pom.xml b/kafka09/kafka09-sink/pom.xml new file mode 100644 index 000000000..5d11d9df8 --- /dev/null +++ b/kafka09/kafka09-sink/pom.xml @@ -0,0 +1,90 @@ + + + + sql.kafka09 + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.kafka09 + jar + + kafka09-sink + http://maven.apache.org + + + + org.apache.flink + flink-json + ${flink.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java new file mode 100644 index 000000000..aeb17ac71 --- /dev/null +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -0,0 +1,106 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; +import java.util.Properties; + +/** + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author DocLi + * @modifyer maqi + */ +public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { + + protected String[] fieldNames; + + protected TypeInformation[] fieldTypes; + + protected String topic; + + protected Properties properties; + + /** Serialization schema for encoding records to Kafka. */ + protected SerializationSchema serializationSchema; + + @Override + public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { + KafkaSinkTableInfo kafka09SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; + this.topic = kafka09SinkTableInfo.getTopic(); + this.fieldNames = kafka09SinkTableInfo.getFields(); + TypeInformation[] types = new TypeInformation[kafka09SinkTableInfo.getFields().length]; + for (int i = 0; i < kafka09SinkTableInfo.getFieldClasses().length; i++) { + types[i] = TypeInformation.of(kafka09SinkTableInfo.getFieldClasses()[i]); + } + this.fieldTypes = types; + + properties = new Properties(); + properties.setProperty("bootstrap.servers", kafka09SinkTableInfo.getBootstrapServers()); + + this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); + return this; + } + + @Override + public void emitDataStream(DataStream dataStream) { + KafkaTableSink kafkaTableSink = new CustomerKafka09JsonTableSink( + topic, + properties, + serializationSchema + ); + + kafkaTableSink.emitDataStream(dataStream); + } + + @Override + public TypeInformation getOutputType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + @Override + public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + +} diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java new file mode 100644 index 000000000..dff0e6d75 --- /dev/null +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -0,0 +1,48 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka.table; + +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +/** + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ +public class KafkaSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + KafkaSinkTableInfo kafka09SinkTableInfo = new KafkaSinkTableInfo(); + kafka09SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka09SinkTableInfo); + + + kafka09SinkTableInfo.setBootstrapServers(MathUtil.getString(props.get(KafkaSinkTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase()))); + kafka09SinkTableInfo.setTopic(MathUtil.getString(props.get(KafkaSinkTableInfo.TOPIC_KEY.toLowerCase()))); + kafka09SinkTableInfo.check(); + return kafka09SinkTableInfo; + } +} diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java new file mode 100644 index 000000000..0280728ac --- /dev/null +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -0,0 +1,80 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +/** + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author DocLi + * @modifyer maqi + */ +public class KafkaSinkTableInfo extends TargetTableInfo { + //version + private static final String CURR_TYPE = "kafka09"; + + public static final String BOOTSTRAPSERVERS_KEY = "bootstrapServers"; + + public static final String TOPIC_KEY = "topic"; + + private String bootstrapServers; + + private String topic; + + public KafkaSinkTableInfo() { + super.setType(CURR_TYPE); + } + + + public String getBootstrapServers() { + return bootstrapServers; + } + + public void setBootstrapServers(String bootstrapServers) { + this.bootstrapServers = bootstrapServers; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + @Override + public boolean check() { + Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(topic, "kafka of topic is required"); + return false; + } + + @Override + public String getType() { +// return super.getType() + SOURCE_SUFFIX; + return super.getType(); + } +} diff --git a/kafka10/kafka10-sink/pom.xml b/kafka10/kafka10-sink/pom.xml new file mode 100644 index 000000000..3a8b7592e --- /dev/null +++ b/kafka10/kafka10-sink/pom.xml @@ -0,0 +1,89 @@ + + + + sql.kafka10 + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.kafka10 + jar + + kafka10-sink + http://maven.apache.org + + + + org.apache.flink + flink-json + ${flink.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java new file mode 100644 index 000000000..b8085c744 --- /dev/null +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -0,0 +1,110 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +import java.util.Optional; +import java.util.Properties; +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author DocLi + * + * @modifyer maqi + * + */ +public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { + + + protected String[] fieldNames; + + protected TypeInformation[] fieldTypes; + + protected String topic; + + protected Properties properties; + + /** Serialization schema for encoding records to Kafka. */ + protected SerializationSchema serializationSchema; + + @Override + public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { + KafkaSinkTableInfo kafka10SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; + this.topic = kafka10SinkTableInfo.getTopic(); + this.fieldNames = kafka10SinkTableInfo.getFields(); + TypeInformation[] types = new TypeInformation[kafka10SinkTableInfo.getFields().length]; + for (int i = 0; i < kafka10SinkTableInfo.getFieldClasses().length; i++) { + types[i] = TypeInformation.of(kafka10SinkTableInfo.getFieldClasses()[i]); + } + this.fieldTypes = types; + + properties = new Properties(); + properties.setProperty("bootstrap.servers", kafka10SinkTableInfo.getBootstrapServers()); + + this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); + return this; + } + + @Override + public void emitDataStream(DataStream dataStream) { + KafkaTableSink kafkaTableSink = new CustomerKafka10JsonTableSink( + topic, + properties, + serializationSchema + ); + + kafkaTableSink.emitDataStream(dataStream); + } + + @Override + public TypeInformation getOutputType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + @Override + public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } +} diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java new file mode 100644 index 000000000..312a1c16a --- /dev/null +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -0,0 +1,48 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka.table; + +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +/** + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ +public class KafkaSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + KafkaSinkTableInfo kafka10SinkTableInfo = new KafkaSinkTableInfo(); + kafka10SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka10SinkTableInfo); + + + kafka10SinkTableInfo.setBootstrapServers(MathUtil.getString(props.get(KafkaSinkTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase()))); + kafka10SinkTableInfo.setTopic(MathUtil.getString(props.get(KafkaSinkTableInfo.TOPIC_KEY.toLowerCase()))); + kafka10SinkTableInfo.check(); + return kafka10SinkTableInfo; + } +} diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java new file mode 100644 index 000000000..a33f120b5 --- /dev/null +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -0,0 +1,82 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author DocLi + * + * @modifyer maqi + * + */ +public class KafkaSinkTableInfo extends TargetTableInfo { + //version + private static final String CURR_TYPE = "kafka10"; + + public static final String BOOTSTRAPSERVERS_KEY = "bootstrapServers"; + + public static final String TOPIC_KEY = "topic"; + + private String bootstrapServers; + + private String topic; + + public KafkaSinkTableInfo() { + super.setType(CURR_TYPE); + } + + + public String getBootstrapServers() { + return bootstrapServers; + } + + public void setBootstrapServers(String bootstrapServers) { + this.bootstrapServers = bootstrapServers; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + @Override + public boolean check() { + Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(topic, "kafka of topic is required"); + return false; + } + + @Override + public String getType() { +// return super.getType() + SOURCE_SUFFIX; + return super.getType(); + } +} diff --git a/kafka11/kafka11-sink/pom.xml b/kafka11/kafka11-sink/pom.xml new file mode 100644 index 000000000..4c748c90b --- /dev/null +++ b/kafka11/kafka11-sink/pom.xml @@ -0,0 +1,89 @@ + + + + sql.kafka11 + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.kafka11 + 1.0-SNAPSHOT + kafka11-sink + jar + + + + + org.apache.flink + flink-json + ${flink.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java new file mode 100644 index 000000000..04f753612 --- /dev/null +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -0,0 +1,108 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; +import java.util.Properties; + +/** + * kafka result table + * Date: 2018/12/18 + * Company: www.dtstack.com + * + * @author DocLi + * + * @modifyer maqi + * + */ +public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { + + protected String[] fieldNames; + + protected TypeInformation[] fieldTypes; + + protected String topic; + + protected Properties properties; + + /** Serialization schema for encoding records to Kafka. */ + protected SerializationSchema serializationSchema; + + @Override + public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { + KafkaSinkTableInfo kafka10SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; + this.topic = kafka10SinkTableInfo.getTopic(); + this.fieldNames = kafka10SinkTableInfo.getFields(); + TypeInformation[] types = new TypeInformation[kafka10SinkTableInfo.getFields().length]; + for (int i = 0; i < kafka10SinkTableInfo.getFieldClasses().length; i++) { + types[i] = TypeInformation.of(kafka10SinkTableInfo.getFieldClasses()[i]); + } + this.fieldTypes = types; + + properties = new Properties(); + properties.setProperty("bootstrap.servers", kafka10SinkTableInfo.getBootstrapServers()); + + this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); + return this; + } + + @Override + public void emitDataStream(DataStream dataStream) { + KafkaTableSink kafkaTableSink = new CustomerKafka11JsonTableSink( + topic, + properties, + serializationSchema + ); + + kafkaTableSink.emitDataStream(dataStream); + } + + @Override + public TypeInformation getOutputType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + @Override + public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } +} diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java new file mode 100644 index 000000000..973c33fcf --- /dev/null +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -0,0 +1,48 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka.table; + +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ +public class KafkaSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + KafkaSinkTableInfo kafka11SinkTableInfo = new KafkaSinkTableInfo(); + kafka11SinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafka11SinkTableInfo); + + + kafka11SinkTableInfo.setBootstrapServers(MathUtil.getString(props.get(KafkaSinkTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase()))); + kafka11SinkTableInfo.setTopic(MathUtil.getString(props.get(KafkaSinkTableInfo.TOPIC_KEY.toLowerCase()))); + kafka11SinkTableInfo.check(); + return kafka11SinkTableInfo; + } +} diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java new file mode 100644 index 000000000..aa9f26d94 --- /dev/null +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -0,0 +1,80 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +/** + * + * Date: 2018/12/18 + * Company: www.dtstack.com + * @author DocLi + * + * @modifyer maqi + * + */ +public class KafkaSinkTableInfo extends TargetTableInfo { + //version + private static final String CURR_TYPE = "kafka11"; + public static final String BOOTSTRAPSERVERS_KEY = "bootstrapServers"; + + public static final String TOPIC_KEY = "topic"; + + private String bootstrapServers; + + private String topic; + + public KafkaSinkTableInfo() { + super.setType(CURR_TYPE); + } + + + public String getBootstrapServers() { + return bootstrapServers; + } + + public void setBootstrapServers(String bootstrapServers) { + this.bootstrapServers = bootstrapServers; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + @Override + public boolean check() { + Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(topic, "kafka of topic is required"); + return false; + } + + @Override + public String getType() { +// return super.getType() + SOURCE_SUFFIX; + return super.getType(); + } +} From fd873ad99ffaf2c47fbf8ef43cd2bc9b5a6333f4 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 14 Dec 2018 16:40:22 +0800 Subject: [PATCH 265/470] Update README.md --- README.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 5dfa80efb..93edde5c3 100644 --- a/README.md +++ b/README.md @@ -14,10 +14,13 @@ * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra # 后续开发计划 - * 增加kafka结果表功能 * 增加SQL支持CEP * 维表快照 * sql优化(谓词下移等) + * serverSocket 源表 + * console 结果表 + * kafka avro格式 + * topN ## 1 快速起步 ### 1.1 运行模式 From 157aa9117a0ab847eeabc5d49edaee06bb7c857f Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 10 Apr 2019 09:56:07 +0800 Subject: [PATCH 266/470] add git-branch --- .../cassandra-side/cassandra-all-side/pom.xml | 2 +- .../cassandra-async-side/pom.xml | 2 +- core/pom.xml | 2 +- elasticsearch5/elasticsearch5-sink/pom.xml | 2 +- hbase/hbase-side/hbase-all-side/pom.xml | 2 +- hbase/hbase-side/hbase-async-side/pom.xml | 2 +- kafka09/kafka09-sink/pom.xml | 2 +- kafka09/kafka09-source/pom.xml | 2 +- kafka10/kafka10-sink/pom.xml | 2 +- kafka10/kafka10-source/pom.xml | 2 +- kafka11/kafka11-sink/pom.xml | 2 +- kafka11/kafka11-source/pom.xml | 2 +- mongo/mongo-side/mongo-all-side/pom.xml | 2 +- mongo/mongo-side/mongo-async-side/pom.xml | 2 +- mongo/mongo-sink/pom.xml | 2 +- mysql/mysql-side/mysql-all-side/pom.xml | 2 +- mysql/mysql-side/mysql-async-side/pom.xml | 2 +- mysql/mysql-sink/pom.xml | 2 +- oracle/oracle-side/oracle-all-side/pom.xml | 2 +- oracle/oracle-side/oracle-async-side/pom.xml | 2 +- oracle/oracle-sink/pom.xml | 2 +- pom.xml | 23 +++++++++++++++++++ redis5/redis5-side/redis-all-side/pom.xml | 2 +- redis5/redis5-side/redis-async-side/pom.xml | 2 +- redis5/redis5-sink/pom.xml | 2 +- .../sqlserver-side/sqlserver-all-side/pom.xml | 2 +- .../sqlserver-async-side/pom.xml | 2 +- sqlserver/sqlserver-sink/pom.xml | 2 +- 28 files changed, 50 insertions(+), 27 deletions(-) diff --git a/cassandra/cassandra-side/cassandra-all-side/pom.xml b/cassandra/cassandra-side/cassandra-all-side/pom.xml index 74c62afdb..461e70beb 100644 --- a/cassandra/cassandra-side/cassandra-all-side/pom.xml +++ b/cassandra/cassandra-side/cassandra-all-side/pom.xml @@ -76,7 +76,7 @@ + tofile="${basedir}/../../../plugins/cassandraallside/${project.name}-${git.branch}.jar" /> diff --git a/cassandra/cassandra-side/cassandra-async-side/pom.xml b/cassandra/cassandra-side/cassandra-async-side/pom.xml index cd709fecd..ee24ae31a 100644 --- a/cassandra/cassandra-side/cassandra-async-side/pom.xml +++ b/cassandra/cassandra-side/cassandra-async-side/pom.xml @@ -92,7 +92,7 @@ + tofile="${basedir}/../../../plugins/cassandraasyncside/${project.name}-${git.branch}.jar" /> diff --git a/core/pom.xml b/core/pom.xml index 1040fcea6..cbcc8eaae 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -106,7 +106,7 @@ + tofile="${basedir}/../plugins/${project.name}-${git.branch}.jar" /> diff --git a/elasticsearch5/elasticsearch5-sink/pom.xml b/elasticsearch5/elasticsearch5-sink/pom.xml index a6453f6ce..459ec5fad 100644 --- a/elasticsearch5/elasticsearch5-sink/pom.xml +++ b/elasticsearch5/elasticsearch5-sink/pom.xml @@ -85,7 +85,7 @@ + tofile="${basedir}/../../plugins/elasticsearchsink/${project.name}-${git.branch}.jar" /> diff --git a/hbase/hbase-side/hbase-all-side/pom.xml b/hbase/hbase-side/hbase-all-side/pom.xml index 32850176b..4a72d0b07 100644 --- a/hbase/hbase-side/hbase-all-side/pom.xml +++ b/hbase/hbase-side/hbase-all-side/pom.xml @@ -93,7 +93,7 @@ + tofile="${basedir}/../../../plugins/hbaseallside/${project.name}-${git.branch}.jar" /> diff --git a/hbase/hbase-side/hbase-async-side/pom.xml b/hbase/hbase-side/hbase-async-side/pom.xml index 193b720a0..312d2c2ae 100644 --- a/hbase/hbase-side/hbase-async-side/pom.xml +++ b/hbase/hbase-side/hbase-async-side/pom.xml @@ -90,7 +90,7 @@ + tofile="${basedir}/../../../plugins/hbaseasyncside/${project.name}-${git.branch}.jar" /> diff --git a/kafka09/kafka09-sink/pom.xml b/kafka09/kafka09-sink/pom.xml index 5d11d9df8..41590fb85 100644 --- a/kafka09/kafka09-sink/pom.xml +++ b/kafka09/kafka09-sink/pom.xml @@ -77,7 +77,7 @@ + tofile="${basedir}/../../plugins/kafka09sink/${project.name}-${git.branch}.jar" /> diff --git a/kafka09/kafka09-source/pom.xml b/kafka09/kafka09-source/pom.xml index 59e05c020..1b008bd3e 100644 --- a/kafka09/kafka09-source/pom.xml +++ b/kafka09/kafka09-source/pom.xml @@ -78,7 +78,7 @@ + tofile="${basedir}/../../plugins/kafka09source/${project.name}-${git.branch}.jar" /> diff --git a/kafka10/kafka10-sink/pom.xml b/kafka10/kafka10-sink/pom.xml index 3a8b7592e..9e2d0dd20 100644 --- a/kafka10/kafka10-sink/pom.xml +++ b/kafka10/kafka10-sink/pom.xml @@ -77,7 +77,7 @@ + tofile="${basedir}/../../plugins/kafka10sink/${project.name}-${git.branch}.jar" /> diff --git a/kafka10/kafka10-source/pom.xml b/kafka10/kafka10-source/pom.xml index 365f98048..0555cdad6 100644 --- a/kafka10/kafka10-source/pom.xml +++ b/kafka10/kafka10-source/pom.xml @@ -77,7 +77,7 @@ + tofile="${basedir}/../../plugins/kafka10source/${project.name}-${git.branch}.jar" /> diff --git a/kafka11/kafka11-sink/pom.xml b/kafka11/kafka11-sink/pom.xml index 4c748c90b..a67b38738 100644 --- a/kafka11/kafka11-sink/pom.xml +++ b/kafka11/kafka11-sink/pom.xml @@ -77,7 +77,7 @@ + tofile="${basedir}/../../plugins/kafka11sink/${project.name}-${git.branch}.jar"/> diff --git a/kafka11/kafka11-source/pom.xml b/kafka11/kafka11-source/pom.xml index 98182aa64..130e66344 100644 --- a/kafka11/kafka11-source/pom.xml +++ b/kafka11/kafka11-source/pom.xml @@ -77,7 +77,7 @@ + tofile="${basedir}/../../plugins/kafka11source/${project.name}-${git.branch}.jar" /> diff --git a/mongo/mongo-side/mongo-all-side/pom.xml b/mongo/mongo-side/mongo-all-side/pom.xml index 28b15bd7c..87709ec4e 100644 --- a/mongo/mongo-side/mongo-all-side/pom.xml +++ b/mongo/mongo-side/mongo-all-side/pom.xml @@ -76,7 +76,7 @@ + tofile="${basedir}/../../../plugins/mongoallside/${project.name}-${git.branch}.jar" /> diff --git a/mongo/mongo-side/mongo-async-side/pom.xml b/mongo/mongo-side/mongo-async-side/pom.xml index 68df02fd0..e800be51c 100644 --- a/mongo/mongo-side/mongo-async-side/pom.xml +++ b/mongo/mongo-side/mongo-async-side/pom.xml @@ -92,7 +92,7 @@ + tofile="${basedir}/../../../plugins/mongoasyncside/${project.name}-${git.branch}.jar" /> diff --git a/mongo/mongo-sink/pom.xml b/mongo/mongo-sink/pom.xml index 79a770684..48bb3bef9 100644 --- a/mongo/mongo-sink/pom.xml +++ b/mongo/mongo-sink/pom.xml @@ -66,7 +66,7 @@ + tofile="${basedir}/../../plugins/mongosink/${project.name}-${git.branch}.jar" /> diff --git a/mysql/mysql-side/mysql-all-side/pom.xml b/mysql/mysql-side/mysql-all-side/pom.xml index 92ce13a85..cc3df111d 100644 --- a/mysql/mysql-side/mysql-all-side/pom.xml +++ b/mysql/mysql-side/mysql-all-side/pom.xml @@ -80,7 +80,7 @@ + tofile="${basedir}/../../../plugins/mysqlallside/${project.name}-${git.branch}.jar"/> diff --git a/mysql/mysql-side/mysql-async-side/pom.xml b/mysql/mysql-side/mysql-async-side/pom.xml index a5588ea97..55110ad3b 100644 --- a/mysql/mysql-side/mysql-async-side/pom.xml +++ b/mysql/mysql-side/mysql-async-side/pom.xml @@ -81,7 +81,7 @@ + tofile="${basedir}/../../../plugins/mysqlasyncside/${project.name}-${git.branch}.jar"/> diff --git a/mysql/mysql-sink/pom.xml b/mysql/mysql-sink/pom.xml index 01b964da8..139a3ea53 100644 --- a/mysql/mysql-sink/pom.xml +++ b/mysql/mysql-sink/pom.xml @@ -79,7 +79,7 @@ + tofile="${basedir}/../../plugins/mysqlsink/${project.name}-${git.branch}.jar"/> diff --git a/oracle/oracle-side/oracle-all-side/pom.xml b/oracle/oracle-side/oracle-all-side/pom.xml index 458077376..0685549c4 100644 --- a/oracle/oracle-side/oracle-all-side/pom.xml +++ b/oracle/oracle-side/oracle-all-side/pom.xml @@ -80,7 +80,7 @@ + tofile="${basedir}/../../../plugins/oracleallside/${project.name}-${git.branch}.jar"/> diff --git a/oracle/oracle-side/oracle-async-side/pom.xml b/oracle/oracle-side/oracle-async-side/pom.xml index 3a4b36559..ba227fd1e 100644 --- a/oracle/oracle-side/oracle-async-side/pom.xml +++ b/oracle/oracle-side/oracle-async-side/pom.xml @@ -79,7 +79,7 @@ + tofile="${basedir}/../../../plugins/oracleasyncside/${project.name}-${git.branch}.jar"/> diff --git a/oracle/oracle-sink/pom.xml b/oracle/oracle-sink/pom.xml index c5807f806..7bd52ec72 100644 --- a/oracle/oracle-sink/pom.xml +++ b/oracle/oracle-sink/pom.xml @@ -79,7 +79,7 @@ + tofile="${basedir}/../../plugins/oraclesink/${project.name}-${git.branch}.jar"/> diff --git a/pom.xml b/pom.xml index 7ef6c1fdd..0e2d9e709 100644 --- a/pom.xml +++ b/pom.xml @@ -41,6 +41,29 @@ 1.8 + + + pl.project13.maven + git-commit-id-plugin + 2.2.6 + + + + revision + + + + + yyyy.MM.dd HH:mm:ss + true + true + + false + -dirty + false + + + diff --git a/redis5/redis5-side/redis-all-side/pom.xml b/redis5/redis5-side/redis-all-side/pom.xml index 7dd752b96..5d8a4a7d0 100644 --- a/redis5/redis5-side/redis-all-side/pom.xml +++ b/redis5/redis5-side/redis-all-side/pom.xml @@ -73,7 +73,7 @@ + tofile="${basedir}/../../../plugins/redisallside/${project.name}-${git.branch}.jar" /> diff --git a/redis5/redis5-side/redis-async-side/pom.xml b/redis5/redis5-side/redis-async-side/pom.xml index e0e573904..39a6280cc 100644 --- a/redis5/redis5-side/redis-async-side/pom.xml +++ b/redis5/redis5-side/redis-async-side/pom.xml @@ -83,7 +83,7 @@ + tofile="${basedir}/../../../plugins/redisasyncside/${project.name}-${git.branch}.jar" /> diff --git a/redis5/redis5-sink/pom.xml b/redis5/redis5-sink/pom.xml index 2919d8685..be6a445ed 100644 --- a/redis5/redis5-sink/pom.xml +++ b/redis5/redis5-sink/pom.xml @@ -83,7 +83,7 @@ + tofile="${basedir}/../../plugins/redissink/${project.name}-${git.branch}.jar" /> diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml b/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml index 7e3db7ecd..136a2af3d 100644 --- a/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml +++ b/sqlserver/sqlserver-side/sqlserver-all-side/pom.xml @@ -80,7 +80,7 @@ + tofile="${basedir}/../../../plugins/sqlserverallside/${project.name}-${git.branch}.jar"/> diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml b/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml index 902228d01..fc2d000f3 100644 --- a/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml +++ b/sqlserver/sqlserver-side/sqlserver-async-side/pom.xml @@ -80,7 +80,7 @@ + tofile="${basedir}/../../../plugins/sqlserverasyncside/${project.name}-${git.branch}.jar"/> diff --git a/sqlserver/sqlserver-sink/pom.xml b/sqlserver/sqlserver-sink/pom.xml index 1180f1ca8..2066877ff 100644 --- a/sqlserver/sqlserver-sink/pom.xml +++ b/sqlserver/sqlserver-sink/pom.xml @@ -81,7 +81,7 @@ + tofile="${basedir}/../../plugins/sqlserversink/${project.name}-${git.branch}.jar"/> From 219168dea9da4d6cfe55c8bfa23c30eabdcd29d7 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 10 Apr 2019 16:08:56 +0800 Subject: [PATCH 267/470] convert rowtime timezone --- .../flink/sql/table/SourceTableInfo.java | 26 +++++++++++++++++++ .../CustomerWaterMarkerForLong.java | 26 ++++++++++++++----- .../CustomerWaterMarkerForTimeStamp.java | 20 ++++++++++---- .../sql/watermarker/WaterMarkerAssigner.java | 6 +++-- .../source/kafka/table/KafkaSourceParser.java | 1 + .../source/kafka/table/KafkaSourceParser.java | 1 + .../source/kafka/table/KafkaSourceParser.java | 1 + 7 files changed, 67 insertions(+), 14 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java index ed342e002..f59b3927b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java @@ -21,9 +21,12 @@ package com.dtstack.flink.sql.table; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import java.util.ArrayList; import java.util.Map; +import java.util.TimeZone; /** * Reason: @@ -36,6 +39,10 @@ public abstract class SourceTableInfo extends TableInfo { public static final String SOURCE_SUFFIX = "Source"; + public static final String TIME_ZONE_KEY="timezone"; + + private String timeZone="Asia/Shanghai"; + private String eventTimeField; private Integer maxOutOrderness = 10; @@ -101,4 +108,23 @@ public String getAdaptSelectSql(){ public String getAdaptName(){ return getName() + "_adapt"; } + + public String getTimeZone() { + return timeZone; + } + + public void setTimeZone(String timeZone) { + if (timeZone==null){ + return; + } + timeZoneCheck(timeZone); + this.timeZone = timeZone; + } + + private void timeZoneCheck(String timeZone) { + ArrayList zones = Lists.newArrayList(TimeZone.getAvailableIDs()); + if (!zones.contains(timeZone)){ + throw new IllegalArgumentException(" timezone is Incorrect!"); + } + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java index 0a6874a81..26058a330 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java @@ -16,17 +16,18 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.watermarker; import com.dtstack.flink.sql.util.MathUtil; -import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.TimeZone; + /** * Custom watermark --- for eventtime * Date: 2017/12/28 @@ -44,9 +45,12 @@ public class CustomerWaterMarkerForLong extends AbsCustomerWaterMarker { private long lastTime = 0; - public CustomerWaterMarkerForLong(Time maxOutOfOrderness, int pos) { + private TimeZone timezone; + + public CustomerWaterMarkerForLong(Time maxOutOfOrderness, int pos,String timezone) { super(maxOutOfOrderness); this.pos = pos; + this.timezone= TimeZone.getTimeZone(timezone); } @Override @@ -54,13 +58,21 @@ public long extractTimestamp(Row row) { try{ Long eveTime = MathUtil.getLongVal(row.getField(pos)); - lastTime = eveTime; - eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - eveTime)/1000)); - return eveTime; + Long extractTime=eveTime; + + lastTime = extractTime + timezone.getOffset(extractTime); + + eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(extractTime))/1000)); + + return lastTime; }catch (Exception e){ logger.error("", e); } - return lastTime; } + + public long convertTimeZone(long evenTime){ + long res = evenTime - timezone.getOffset(evenTime) + TimeZone.getDefault().getOffset(evenTime); + return res; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java index b4aee0302..48172ff6d 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java @@ -27,6 +27,7 @@ import org.slf4j.LoggerFactory; import java.sql.Timestamp; +import java.util.TimeZone; /** * Custom watermark --- for eventtime @@ -45,25 +46,34 @@ public class CustomerWaterMarkerForTimeStamp extends AbsCustomerWaterMarker private long lastTime = 0; + private TimeZone timezone; - public CustomerWaterMarkerForTimeStamp(Time maxOutOfOrderness, int pos) { + public CustomerWaterMarkerForTimeStamp(Time maxOutOfOrderness, int pos,String timezone) { super(maxOutOfOrderness); this.pos = pos; + this.timezone= TimeZone.getTimeZone(timezone); } @Override public long extractTimestamp(Row row) { try { Timestamp time = (Timestamp) row.getField(pos); - lastTime = time.getTime(); - eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - time.getTime())/1000)); - return time.getTime(); + long extractTime=time.getTime(); + + lastTime = extractTime + timezone.getOffset(extractTime); + + eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(extractTime))/1000)); + + return lastTime; } catch (RuntimeException e) { logger.error("", e); } return lastTime; } - + public long convertTimeZone(long evenTime){ + long res = evenTime - timezone.getOffset(evenTime) + TimeZone.getDefault().getOffset(evenTime); + return res; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java index a29e8391b..9288ecb87 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java @@ -54,6 +54,8 @@ public DataStream assignWaterMarker(DataStream dataStream, RowTypeInfo type int maxOutOrderness = sourceTableInfo.getMaxOutOrderness(); + String timeZone=sourceTableInfo.getTimeZone(); + String[] fieldNames = typeInfo.getFieldNames(); TypeInformation[] fieldTypes = typeInfo.getFieldTypes(); @@ -75,9 +77,9 @@ public DataStream assignWaterMarker(DataStream dataStream, RowTypeInfo type AbsCustomerWaterMarker waterMarker = null; if(fieldType.getTypeClass().getTypeName().equalsIgnoreCase("java.sql.Timestamp")){ - waterMarker = new CustomerWaterMarkerForTimeStamp(Time.milliseconds(maxOutOrderness), pos); + waterMarker = new CustomerWaterMarkerForTimeStamp(Time.milliseconds(maxOutOrderness), pos,timeZone); }else if(fieldType.getTypeClass().getTypeName().equalsIgnoreCase("java.lang.Long")){ - waterMarker = new CustomerWaterMarkerForLong(Time.milliseconds(maxOutOrderness), pos); + waterMarker = new CustomerWaterMarkerForLong(Time.milliseconds(maxOutOrderness), pos,timeZone); }else{ throw new IllegalArgumentException("not support type of " + fieldType + ", current only support(timestamp, long)."); } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 664e8c4b1..b292ba181 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -48,6 +48,7 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Wed, 10 Apr 2019 18:56:44 +0800 Subject: [PATCH 268/470] support kafka parameter --- .../flink/sql/sink/kafka/KafkaSink.java | 3 +++ .../sql/sink/kafka/table/KafkaSinkParser.java | 5 +++++ .../sink/kafka/table/KafkaSinkTableInfo.java | 13 +++++++++++++ .../flink/sql/sink/kafka/KafkaSink.java | 3 +++ .../sql/sink/kafka/table/KafkaSinkParser.java | 5 +++++ .../sink/kafka/table/KafkaSinkTableInfo.java | 13 +++++++++++++ .../flink/sql/sink/kafka/KafkaSink.java | 18 ++++++++++-------- .../sql/sink/kafka/table/KafkaSinkParser.java | 5 +++++ .../sink/kafka/table/KafkaSinkTableInfo.java | 15 +++++++++++++++ 9 files changed, 72 insertions(+), 8 deletions(-) diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index aeb17ac71..2ffc03740 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -64,6 +64,9 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { this.fieldTypes = types; properties = new Properties(); + for (String key:kafka09SinkTableInfo.getKafkaParamKeys()) { + properties.setProperty(key, kafka09SinkTableInfo.getKafkaParam(key)); + } properties.setProperty("bootstrap.servers", kafka09SinkTableInfo.getBootstrapServers()); this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index dff0e6d75..2f460099d 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -42,6 +42,11 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map kafkaParam = new HashMap(); + public KafkaSinkTableInfo() { super.setType(CURR_TYPE); } + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); + } + + public String getKafkaParam(String key){ + return kafkaParam.get(key); + } + + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); + } public String getBootstrapServers() { return bootstrapServers; diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index b8085c744..55a647676 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -69,6 +69,9 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { this.fieldTypes = types; properties = new Properties(); + for (String key:kafka10SinkTableInfo.getKafkaParamKeys()) { + properties.setProperty(key, kafka10SinkTableInfo.getKafkaParam(key)); + } properties.setProperty("bootstrap.servers", kafka10SinkTableInfo.getBootstrapServers()); this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index 312a1c16a..2487ca11e 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -42,6 +42,11 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map kafkaParam = new HashMap(); + private String topic; public KafkaSinkTableInfo() { super.setType(CURR_TYPE); } + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); + } + + public String getKafkaParam(String key){ + return kafkaParam.get(key); + } + + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); + } public String getBootstrapServers() { diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 04f753612..de575bfd7 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -57,18 +57,20 @@ public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener< @Override public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { - KafkaSinkTableInfo kafka10SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; - this.topic = kafka10SinkTableInfo.getTopic(); - this.fieldNames = kafka10SinkTableInfo.getFields(); - TypeInformation[] types = new TypeInformation[kafka10SinkTableInfo.getFields().length]; - for (int i = 0; i < kafka10SinkTableInfo.getFieldClasses().length; i++) { - types[i] = TypeInformation.of(kafka10SinkTableInfo.getFieldClasses()[i]); + KafkaSinkTableInfo kafka11SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; + this.topic = kafka11SinkTableInfo.getTopic(); + this.fieldNames = kafka11SinkTableInfo.getFields(); + TypeInformation[] types = new TypeInformation[kafka11SinkTableInfo.getFields().length]; + for (int i = 0; i < kafka11SinkTableInfo.getFieldClasses().length; i++) { + types[i] = TypeInformation.of(kafka11SinkTableInfo.getFieldClasses()[i]); } this.fieldTypes = types; properties = new Properties(); - properties.setProperty("bootstrap.servers", kafka10SinkTableInfo.getBootstrapServers()); - + for (String key:kafka11SinkTableInfo.getKafkaParamKeys()) { + properties.setProperty(key, kafka11SinkTableInfo.getKafkaParam(key)); + } + properties.setProperty("bootstrap.servers", kafka11SinkTableInfo.getBootstrapServers()); this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); return this; } diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index 973c33fcf..ca4aa48ef 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -42,6 +42,11 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map kafkaParam = new HashMap(); + public KafkaSinkTableInfo() { super.setType(CURR_TYPE); } + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); + } + + public String getKafkaParam(String key){ + return kafkaParam.get(key); + } + + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); + } + + public String getBootstrapServers() { return bootstrapServers; From 25799ecae92288c240e888759b6905ef50cb8d81 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 11 Apr 2019 14:34:51 +0800 Subject: [PATCH 269/470] timeZone not null check --- .../java/com/dtstack/flink/sql/table/SourceTableInfo.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java index f59b3927b..a92aa9fb1 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java @@ -23,6 +23,7 @@ import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.util.StringUtils; import java.util.ArrayList; import java.util.Map; @@ -41,7 +42,7 @@ public abstract class SourceTableInfo extends TableInfo { public static final String TIME_ZONE_KEY="timezone"; - private String timeZone="Asia/Shanghai"; + private String timeZone=TimeZone.getDefault().getID(); private String eventTimeField; @@ -70,7 +71,6 @@ public void setMaxOutOrderness(Integer maxOutOrderness) { if(maxOutOrderness == null){ return; } - this.maxOutOrderness = maxOutOrderness; } @@ -114,7 +114,7 @@ public String getTimeZone() { } public void setTimeZone(String timeZone) { - if (timeZone==null){ + if (StringUtils.isNullOrWhitespaceOnly(timeZone)){ return; } timeZoneCheck(timeZone); From d2fd08c821861753833bcfe490181d928f5caf4a Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 11 Apr 2019 17:43:26 +0800 Subject: [PATCH 270/470] modify customerWaterMark --- .../sql/watermarker/CustomerWaterMarkerForLong.java | 9 ++------- .../sql/watermarker/CustomerWaterMarkerForTimeStamp.java | 6 +----- 2 files changed, 3 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java index 26058a330..00ae8501b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForLong.java @@ -57,12 +57,11 @@ public CustomerWaterMarkerForLong(Time maxOutOfOrderness, int pos,String timezon public long extractTimestamp(Row row) { try{ - Long eveTime = MathUtil.getLongVal(row.getField(pos)); - Long extractTime=eveTime; + Long extractTime = MathUtil.getLongVal(row.getField(pos)); lastTime = extractTime + timezone.getOffset(extractTime); - eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(extractTime))/1000)); + eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - extractTime)/1000)); return lastTime; }catch (Exception e){ @@ -71,8 +70,4 @@ public long extractTimestamp(Row row) { return lastTime; } - public long convertTimeZone(long evenTime){ - long res = evenTime - timezone.getOffset(evenTime) + TimeZone.getDefault().getOffset(evenTime); - return res; - } } diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java index 48172ff6d..1b52b0d88 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/CustomerWaterMarkerForTimeStamp.java @@ -63,7 +63,7 @@ public long extractTimestamp(Row row) { lastTime = extractTime + timezone.getOffset(extractTime); - eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - convertTimeZone(extractTime))/1000)); + eventDelayGauge.setDelayTime(MathUtil.getIntegerVal((System.currentTimeMillis() - extractTime)/1000)); return lastTime; } catch (RuntimeException e) { @@ -72,8 +72,4 @@ public long extractTimestamp(Row row) { return lastTime; } - public long convertTimeZone(long evenTime){ - long res = evenTime - timezone.getOffset(evenTime) + TimeZone.getDefault().getOffset(evenTime); - return res; - } } From 103a9ad49f8b866fc943d68f01e02a6661421f82 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 11 Apr 2019 17:53:38 +0800 Subject: [PATCH 271/470] code format --- .../main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java | 2 +- .../com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java | 2 +- .../main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java | 2 +- .../com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java | 2 +- .../main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java | 2 +- .../com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 2ffc03740..c5970150e 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -64,7 +64,7 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { this.fieldTypes = types; properties = new Properties(); - for (String key:kafka09SinkTableInfo.getKafkaParamKeys()) { + for (String key : kafka09SinkTableInfo.getKafkaParamKeys()) { properties.setProperty(key, kafka09SinkTableInfo.getKafkaParam(key)); } properties.setProperty("bootstrap.servers", kafka09SinkTableInfo.getBootstrapServers()); diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index 2f460099d..3e1f707e2 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -42,7 +42,7 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Wed, 17 Apr 2019 16:17:42 +0800 Subject: [PATCH 272/470] find jar name --- .../dtstack/flink/sql/util/PluginUtil.java | 37 +++++++++++++++++-- .../flink/sql/launcher/LauncherMain.java | 8 ++-- 2 files changed, 38 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java index 611cef37f..1f448b20a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.util; import com.dtstack.flink.sql.classloader.DtClassLoader; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerationException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParseException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonMappingException; @@ -28,6 +29,7 @@ import java.io.ByteArrayInputStream; import java.io.File; +import java.io.FilenameFilter; import java.io.IOException; import java.net.MalformedURLException; import java.net.URL; @@ -105,15 +107,19 @@ public static Properties stringToProperties(String str) throws IOException{ return properties; } - public static URL getRemoteJarFilePath(String pluginType, String tableType, String remoteSqlRootDir) throws MalformedURLException { + public static URL getRemoteJarFilePath(String pluginType, String tableType, String remoteSqlRootDir) throws Exception { String dirName = pluginType + tableType.toLowerCase(); - String jarName = String.format("%s-%s.jar", pluginType, tableType.toLowerCase()); + String prefix = String.format("%s-%s", pluginType, tableType.toLowerCase()); + String jarPath = remoteSqlRootDir + SP + dirName; + String jarName = getCoreJarFileName(jarPath, prefix); return new URL("file:" + remoteSqlRootDir + SP + dirName + SP + jarName); } - public static URL getRemoteSideJarFilePath(String pluginType, String sideOperator, String tableType, String remoteSqlRootDir) throws MalformedURLException { + public static URL getRemoteSideJarFilePath(String pluginType, String sideOperator, String tableType, String remoteSqlRootDir) throws Exception { String dirName = pluginType + sideOperator + tableType.toLowerCase(); - String jarName = String.format("%s-%s-%s.jar", pluginType, sideOperator, tableType.toLowerCase()); + String prefix = String.format("%s-%s-%s", pluginType, sideOperator, tableType.toLowerCase()); + String jarPath = remoteSqlRootDir + SP + dirName; + String jarName = getCoreJarFileName(jarPath, prefix); return new URL("file:" + remoteSqlRootDir + SP + dirName + SP + jarName); } @@ -138,4 +144,27 @@ public static void addPluginJar(String pluginDir, DtClassLoader classLoader) thr } } + public static String getCoreJarFileName (String path, String prefix) throws Exception { + String coreJarFileName = null; + File pluginDir = new File(path); + if (pluginDir.exists() && pluginDir.isDirectory()){ + File[] jarFiles = pluginDir.listFiles(new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + return name.toLowerCase().startsWith(prefix) && name.toLowerCase().endsWith(".jar"); + } + }); + + if (jarFiles != null && jarFiles.length > 0){ + coreJarFileName = jarFiles[0].getName(); + } + } + + if (StringUtils.isEmpty(coreJarFileName)){ + throw new Exception("Can not find core jar file in path:" + path); + } + + return coreJarFileName; + } + } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 32813393d..0917b9eda 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -26,6 +26,7 @@ import com.dtstack.flink.sql.ClusterMode; import com.dtstack.flink.sql.Main; import com.dtstack.flink.sql.launcher.perjob.PerJobSubmitter; +import com.dtstack.flink.sql.util.PluginUtil; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.PackagedProgramUtils; @@ -52,13 +53,14 @@ */ public class LauncherMain { - private static final String CORE_JAR = "core.jar"; + private static final String CORE_JAR = "core"; private static String SP = File.separator; - private static String getLocalCoreJarPath(String localSqlRootJar){ - return localSqlRootJar + SP + CORE_JAR; + private static String getLocalCoreJarPath(String localSqlRootJar) throws Exception { + String jarPath = PluginUtil.getCoreJarFileName(localSqlRootJar, CORE_JAR); + return jarPath; } public static void main(String[] args) throws Exception { From 7086e9d176044196cc31b189e648231d479f3ecd Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Sat, 20 Apr 2019 17:02:25 +0800 Subject: [PATCH 273/470] remove kafkasink slf4 --- kafka11/kafka11-sink/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka11/kafka11-sink/pom.xml b/kafka11/kafka11-sink/pom.xml index a67b38738..c27ebb791 100644 --- a/kafka11/kafka11-sink/pom.xml +++ b/kafka11/kafka11-sink/pom.xml @@ -39,7 +39,7 @@ - + org.slf4j From 88d43ee56071fc94fa9f547b211e58b437e67031 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 22 Apr 2019 16:46:02 +0800 Subject: [PATCH 274/470] =?UTF-8?q?=E4=BF=AE=E5=A4=8D=20yarn=20perjob?= =?UTF-8?q?=E6=A8=A1=E5=BC=8F=E4=B8=8B=E5=8A=A0=E8=BD=BD=20udf=20jar=20?= =?UTF-8?q?=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flink/sql/util/FlinkUtil.java | 9 ++- .../flink/sql/util/ReflectionUtils.java | 79 +++++++++++++++++++ 2 files changed, 86 insertions(+), 2 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/util/ReflectionUtils.java diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java index 181f6de5c..0ba59ace6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java @@ -267,8 +267,13 @@ public static URLClassLoader loadExtraJar(List jarURLList, URLClassLoader c return classLoader; } - private static void urlClassLoaderAddUrl(URLClassLoader classLoader, URL url) throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { - Method method = classLoader.getClass().getDeclaredMethod("addURL", URL.class); + private static void urlClassLoaderAddUrl(URLClassLoader classLoader, URL url) throws InvocationTargetException, IllegalAccessException { + Method method = ReflectionUtils.getDeclaredMethod(classLoader, "addURL", URL.class); + + if(method == null){ + throw new RuntimeException("can't not find declared method addURL, curr classLoader is " + classLoader.getClass()); + } + method.setAccessible(true); method.invoke(classLoader, url); } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ReflectionUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/ReflectionUtils.java new file mode 100644 index 000000000..b67993e05 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/util/ReflectionUtils.java @@ -0,0 +1,79 @@ +/* + * 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 com.dtstack.flink.sql.util; + +import java.lang.reflect.Field; +import java.lang.reflect.Method; + +/** + * + * Date: 2019/4/22 + * Company: www.dtstack.com + * @author xuchao + */ + +public class ReflectionUtils { + + /** + * get declaredMethod util find + * @param object + * @param methodName + * @param parameterTypes + * @return + */ + public static Method getDeclaredMethod(Object object, String methodName, Class ... parameterTypes){ + Method method = null ; + + for(Class clazz = object.getClass() ; clazz != Object.class ; clazz = clazz.getSuperclass()) { + try { + method = clazz.getDeclaredMethod(methodName, parameterTypes) ; + return method ; + } catch (Exception e) { + //do nothing then can get method from super Class + } + } + + return null; + } + + /** + * 循环向上转型, 获取对象的 DeclaredField + * @param object : 子类对象 + * @param fieldName : 父类中的属性名 + * @return 父类中的属性对象 + */ + + public static Field getDeclaredField(Object object, String fieldName){ + Field field = null ; + + Class clazz = object.getClass() ; + + for(; clazz != Object.class ; clazz = clazz.getSuperclass()) { + try { + field = clazz.getDeclaredField(fieldName) ; + return field ; + } catch (Exception e) { + //do nothing then can get field from super Class + } + } + + return null; + } + +} From 6fbf16bcbbb1c532a83b9a18541a35959c79a09a Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 24 Apr 2019 14:13:11 +0800 Subject: [PATCH 275/470] fix --- .../sql/sink/kafka/CustomerKafka10JsonTableSink.java | 6 +++--- .../sql/sink/kafka/CustomerKafka11JsonTableSink.java | 9 +++++---- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java index 303892c1e..8c9f31659 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java @@ -21,7 +21,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase; -import org.apache.flink.streaming.connectors.kafka.Kafka09JsonTableSink; +import org.apache.flink.streaming.connectors.kafka.Kafka010JsonTableSink; import org.apache.flink.streaming.connectors.kafka.KafkaJsonTableSink; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; @@ -67,8 +67,8 @@ protected FlinkKafkaProducerBase createKafkaProducer(String topic, Properti } @Override - protected Kafka09JsonTableSink createCopy() { - return new Kafka09JsonTableSink(topic, properties, partitioner); + protected Kafka010JsonTableSink createCopy() { + return new Kafka010JsonTableSink(topic, properties, partitioner); } @Override diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java index 05d15949a..1b4bbdd95 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java @@ -21,7 +21,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase; -import org.apache.flink.streaming.connectors.kafka.Kafka09JsonTableSink; +import org.apache.flink.streaming.connectors.kafka.Kafka010JsonTableSink; import org.apache.flink.streaming.connectors.kafka.KafkaJsonTableSink; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; @@ -33,6 +33,7 @@ import java.util.Properties; /** + * * Reason: add schema info * Date: 2019/4/8 * Company: www.dtstack.com @@ -60,15 +61,15 @@ public CustomerKafka11JsonTableSink(String topic, Properties properties, KafkaPa super(topic, properties, new FlinkKafkaDelegatePartitioner<>(partitioner)); this.schema = schema; } - + //TODO 暂时使用010 @Override protected FlinkKafkaProducerBase createKafkaProducer(String topic, Properties properties, SerializationSchema serializationSchema, FlinkKafkaPartitioner partitioner) { return new FlinkKafkaProducer010(topic, serializationSchema, properties, partitioner); } @Override - protected Kafka09JsonTableSink createCopy() { - return new Kafka09JsonTableSink(topic, properties, partitioner); + protected Kafka010JsonTableSink createCopy() { + return new Kafka010JsonTableSink(topic, properties, partitioner); } @Override From 6670e1b432107c8c361ee748abbded46461ccd4f Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 24 Apr 2019 21:03:43 +0800 Subject: [PATCH 276/470] kafkasink metric --- kafka09/kafka09-sink/pom.xml | 2 +- .../kafka/CustomerFlinkKafkaProducer09.java | 124 ++++++++++ .../CustomerJsonRowSerializationSchema.java | 219 +++++++++++++++++ .../kafka/CustomerKafka09JsonTableSink.java | 2 +- .../flink/sql/sink/kafka/KafkaSink.java | 3 +- kafka10/kafka10-sink/pom.xml | 2 +- .../kafka/CustomerFlinkKafkaProducer010.java | 124 ++++++++++ .../CustomerJsonRowSerializationSchema.java | 219 +++++++++++++++++ .../kafka/CustomerKafka10JsonTableSink.java | 3 +- .../flink/sql/sink/kafka/KafkaSink.java | 3 +- .../kafka/CustomerFlinkKafkaProducer011.java | 124 ++++++++++ .../CustomerJsonRowSerializationSchema.java | 225 ++++++++++++++++++ .../kafka/CustomerKafka11JsonTableSink.java | 3 +- .../flink/sql/sink/kafka/KafkaSink.java | 2 +- .../sql/launcher/LauncherOptionParser.java | 6 +- pom.xml | 20 +- 16 files changed, 1056 insertions(+), 25 deletions(-) create mode 100644 kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer09.java create mode 100644 kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java create mode 100644 kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer010.java create mode 100644 kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java create mode 100644 kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer011.java create mode 100644 kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java diff --git a/kafka09/kafka09-sink/pom.xml b/kafka09/kafka09-sink/pom.xml index 41590fb85..7b16fade8 100644 --- a/kafka09/kafka09-sink/pom.xml +++ b/kafka09/kafka09-sink/pom.xml @@ -39,7 +39,7 @@ - + org.slf4j diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer09.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer09.java new file mode 100644 index 000000000..09886b7e9 --- /dev/null +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer09.java @@ -0,0 +1,124 @@ +/** + * 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 com.dtstack.flink.sql.sink.kafka; + +import com.dtstack.flink.sql.metric.MetricConstant; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer09; +import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricWrapper; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; + +import java.util.Map; +import java.util.Properties; + +/** + * Reason: + * Date: 2019/4/24 + * Company: www.dtstack.com + * + * @author maqi + */ +public class CustomerFlinkKafkaProducer09 extends FlinkKafkaProducer09 { + + CustomerJsonRowSerializationSchema schema; + + public CustomerFlinkKafkaProducer09(String topicId, SerializationSchema serializationSchema, Properties producerConfig) { + super(topicId, serializationSchema, producerConfig); + this.schema = (CustomerJsonRowSerializationSchema) serializationSchema; + } + + @Override + public void open(Configuration configuration) { + producer = getKafkaProducer(this.producerConfig); + + RuntimeContext ctx = getRuntimeContext(); + Counter counter = ctx.getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + MeterView meter = ctx.getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(counter, 20)); + + schema.setCounter(counter); + + if (null != flinkKafkaPartitioner) { + if (flinkKafkaPartitioner instanceof FlinkKafkaDelegatePartitioner) { + ((FlinkKafkaDelegatePartitioner) flinkKafkaPartitioner).setPartitions( + getPartitionsByTopic(this.defaultTopicId, this.producer)); + } + flinkKafkaPartitioner.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks()); + } + + + // register Kafka metrics to Flink accumulators + if (!Boolean.parseBoolean(producerConfig.getProperty(KEY_DISABLE_METRICS, "false"))) { + Map metrics = this.producer.metrics(); + + if (metrics == null) { + // MapR's Kafka implementation returns null here. + } else { + final MetricGroup kafkaMetricGroup = getRuntimeContext().getMetricGroup().addGroup("KafkaProducer"); + for (Map.Entry metric : metrics.entrySet()) { + kafkaMetricGroup.gauge(metric.getKey().name(), new KafkaMetricWrapper(metric.getValue())); + } + } + } + + if (flushOnCheckpoint && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) { + flushOnCheckpoint = false; + } + + if (logFailuresOnly) { + callback = new Callback() { + @Override + public void onCompletion(RecordMetadata metadata, Exception e) { + if (e != null) { + } + acknowledgeMessage(); + } + }; + } else { + callback = new Callback() { + @Override + public void onCompletion(RecordMetadata metadata, Exception exception) { + if (exception != null && asyncException == null) { + asyncException = exception; + } + acknowledgeMessage(); + } + }; + } + } + + private void acknowledgeMessage() { + if (flushOnCheckpoint) { + synchronized (pendingRecordsLock) { + pendingRecords--; + if (pendingRecords == 0) { + pendingRecordsLock.notifyAll(); + } + } + } + } +} diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java new file mode 100644 index 000000000..05c934b27 --- /dev/null +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java @@ -0,0 +1,219 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowDeserializationSchema; +import org.apache.flink.formats.json.JsonSchemaConverter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ContainerNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Time; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; + +/** + * Serialization schema that serializes an object of Flink types into a JSON bytes. + * + *

Serializes the input Flink object into a JSON string and + * converts it into byte[]. + * + *

Result byte[] messages can be deserialized using {@link JsonRowDeserializationSchema}. + */ +public class CustomerJsonRowSerializationSchema implements SerializationSchema { + + private static final long serialVersionUID = -2885556750743978636L; + + /** Type information describing the input type. */ + private final TypeInformation typeInfo; + + /** Object mapper that is used to create output JSON objects. */ + private final ObjectMapper mapper = new ObjectMapper(); + + /** Formatter for RFC 3339-compliant string representation of a time value (with UTC timezone, without milliseconds). */ + private SimpleDateFormat timeFormat = new SimpleDateFormat("HH:mm:ss'Z'"); + + /** Formatter for RFC 3339-compliant string representation of a time value (with UTC timezone). */ + private SimpleDateFormat timeFormatWithMillis = new SimpleDateFormat("HH:mm:ss.SSS'Z'"); + + /** Formatter for RFC 3339-compliant string representation of a timestamp value (with UTC timezone). */ + private SimpleDateFormat timestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); + + /** Reusable object node. */ + private transient ObjectNode node; + + private Counter counter; + + /** + * Creates a JSON serialization schema for the given type information. + * + * @param typeInfo The field names of {@link Row} are used to map to JSON properties. + */ + public CustomerJsonRowSerializationSchema(TypeInformation typeInfo) { + Preconditions.checkNotNull(typeInfo, "Type information"); + this.typeInfo = typeInfo; + } + + /** + * Creates a JSON serialization schema for the given JSON schema. + * + * @param jsonSchema JSON schema describing the result type + * + * @see http://json-schema.org/ + */ + public CustomerJsonRowSerializationSchema(String jsonSchema) { + this(JsonSchemaConverter.convert(jsonSchema)); + } + + @Override + public byte[] serialize(Row row) { + if (node == null) { + node = mapper.createObjectNode(); + } + + try { + convertRow(node, (RowTypeInfo) typeInfo, row); + counter.inc(); + return mapper.writeValueAsBytes(node); + } catch (Throwable t) { + throw new RuntimeException("Could not serialize row '" + row + "'. " + + "Make sure that the schema matches the input.", t); + } + } + + // -------------------------------------------------------------------------------------------- + + private ObjectNode convertRow(ObjectNode reuse, RowTypeInfo info, Row row) { + if (reuse == null) { + reuse = mapper.createObjectNode(); + } + final String[] fieldNames = info.getFieldNames(); + final TypeInformation[] fieldTypes = info.getFieldTypes(); + + // validate the row + if (row.getArity() != fieldNames.length) { + throw new IllegalStateException(String.format( + "Number of elements in the row '%s' is different from number of field names: %d", row, fieldNames.length)); + } + + for (int i = 0; i < fieldNames.length; i++) { + final String name = fieldNames[i]; + + final JsonNode fieldConverted = convert(reuse, reuse.get(name), fieldTypes[i], row.getField(i)); + reuse.set(name, fieldConverted); + } + + return reuse; + } + + private JsonNode convert(ContainerNode container, JsonNode reuse, TypeInformation info, Object object) { + if (info == Types.VOID || object == null) { + return container.nullNode(); + } else if (info == Types.BOOLEAN) { + return container.booleanNode((Boolean) object); + } else if (info == Types.STRING) { + return container.textNode((String) object); + } else if (info == Types.BIG_DEC) { + // convert decimal if necessary + if (object instanceof BigDecimal) { + return container.numberNode((BigDecimal) object); + } + return container.numberNode(BigDecimal.valueOf(((Number) object).doubleValue())); + } else if (info == Types.BIG_INT) { + // convert integer if necessary + if (object instanceof BigInteger) { + return container.numberNode((BigInteger) object); + } + return container.numberNode(BigInteger.valueOf(((Number) object).longValue())); + } else if (info == Types.SQL_DATE) { + return container.textNode(object.toString()); + } else if (info == Types.SQL_TIME) { + final Time time = (Time) object; + // strip milliseconds if possible + if (time.getTime() % 1000 > 0) { + return container.textNode(timeFormatWithMillis.format(time)); + } + return container.textNode(timeFormat.format(time)); + } else if (info == Types.SQL_TIMESTAMP) { + return container.textNode(timestampFormat.format((Timestamp) object)); + } else if (info instanceof RowTypeInfo) { + if (reuse != null && reuse instanceof ObjectNode) { + return convertRow((ObjectNode) reuse, (RowTypeInfo) info, (Row) object); + } else { + return convertRow(null, (RowTypeInfo) info, (Row) object); + } + } else if (info instanceof ObjectArrayTypeInfo) { + if (reuse != null && reuse instanceof ArrayNode) { + return convertObjectArray((ArrayNode) reuse, ((ObjectArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } else { + return convertObjectArray(null, ((ObjectArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } + } else if (info instanceof BasicArrayTypeInfo) { + if (reuse != null && reuse instanceof ArrayNode) { + return convertObjectArray((ArrayNode) reuse, ((BasicArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } else { + return convertObjectArray(null, ((BasicArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } + } else if (info instanceof PrimitiveArrayTypeInfo && ((PrimitiveArrayTypeInfo) info).getComponentType() == Types.BYTE) { + return container.binaryNode((byte[]) object); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return mapper.valueToTree(object); + } catch (IllegalArgumentException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for object: " + object, e); + } + } + } + + private ArrayNode convertObjectArray(ArrayNode reuse, TypeInformation info, Object[] array) { + if (reuse == null) { + reuse = mapper.createArrayNode(); + } else { + reuse.removeAll(); + } + + for (Object object : array) { + reuse.add(convert(reuse, null, info, object)); + } + return reuse; + } + + public Counter getCounter() { + return counter; + } + + public void setCounter(Counter counter) { + this.counter = counter; + } +} diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka09JsonTableSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka09JsonTableSink.java index 998d163e2..508366435 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka09JsonTableSink.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka09JsonTableSink.java @@ -60,7 +60,7 @@ public CustomerKafka09JsonTableSink(String topic, Properties properties, KafkaPa @Override protected FlinkKafkaProducerBase createKafkaProducer(String topic, Properties properties, SerializationSchema serializationSchema, FlinkKafkaPartitioner partitioner) { - return new FlinkKafkaProducer09<>(topic, serializationSchema, properties, partitioner); + return new CustomerFlinkKafkaProducer09(topic, serializationSchema, properties); } @Override diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index c5970150e..864bf997f 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -24,7 +24,6 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; import org.apache.flink.table.sinks.AppendStreamTableSink; @@ -69,7 +68,7 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { } properties.setProperty("bootstrap.servers", kafka09SinkTableInfo.getBootstrapServers()); - this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); + this.serializationSchema = new CustomerJsonRowSerializationSchema(getOutputType()); return this; } diff --git a/kafka10/kafka10-sink/pom.xml b/kafka10/kafka10-sink/pom.xml index 9e2d0dd20..9893d386f 100644 --- a/kafka10/kafka10-sink/pom.xml +++ b/kafka10/kafka10-sink/pom.xml @@ -39,7 +39,7 @@ - + org.slf4j diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer010.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer010.java new file mode 100644 index 000000000..dcda22f39 --- /dev/null +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer010.java @@ -0,0 +1,124 @@ +/** + * 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 com.dtstack.flink.sql.sink.kafka; + +import com.dtstack.flink.sql.metric.MetricConstant; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; +import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricWrapper; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; + +import java.util.Map; +import java.util.Properties; + +/** + * Reason: + * Date: 2019/4/24 + * Company: www.dtstack.com + * + * @author maqi + */ +public class CustomerFlinkKafkaProducer010 extends FlinkKafkaProducer010 { + + CustomerJsonRowSerializationSchema schema; + + public CustomerFlinkKafkaProducer010(String topicId, SerializationSchema serializationSchema, Properties producerConfig) { + super(topicId, serializationSchema, producerConfig); + this.schema = (CustomerJsonRowSerializationSchema) serializationSchema; + } + + @Override + public void open(Configuration configuration) { + producer = getKafkaProducer(this.producerConfig); + + RuntimeContext ctx = getRuntimeContext(); + Counter counter = ctx.getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + MeterView meter = ctx.getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(counter, 20)); + + schema.setCounter(counter); + + if (null != flinkKafkaPartitioner) { + if (flinkKafkaPartitioner instanceof FlinkKafkaDelegatePartitioner) { + ((FlinkKafkaDelegatePartitioner) flinkKafkaPartitioner).setPartitions( + getPartitionsByTopic(this.defaultTopicId, this.producer)); + } + flinkKafkaPartitioner.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks()); + } + + + // register Kafka metrics to Flink accumulators + if (!Boolean.parseBoolean(producerConfig.getProperty(KEY_DISABLE_METRICS, "false"))) { + Map metrics = this.producer.metrics(); + + if (metrics == null) { + // MapR's Kafka implementation returns null here. + } else { + final MetricGroup kafkaMetricGroup = getRuntimeContext().getMetricGroup().addGroup("KafkaProducer"); + for (Map.Entry metric : metrics.entrySet()) { + kafkaMetricGroup.gauge(metric.getKey().name(), new KafkaMetricWrapper(metric.getValue())); + } + } + } + + if (flushOnCheckpoint && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) { + flushOnCheckpoint = false; + } + + if (logFailuresOnly) { + callback = new Callback() { + @Override + public void onCompletion(RecordMetadata metadata, Exception e) { + if (e != null) { + } + acknowledgeMessage(); + } + }; + } else { + callback = new Callback() { + @Override + public void onCompletion(RecordMetadata metadata, Exception exception) { + if (exception != null && asyncException == null) { + asyncException = exception; + } + acknowledgeMessage(); + } + }; + } + } + + private void acknowledgeMessage() { + if (flushOnCheckpoint) { + synchronized (pendingRecordsLock) { + pendingRecords--; + if (pendingRecords == 0) { + pendingRecordsLock.notifyAll(); + } + } + } + } +} diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java new file mode 100644 index 000000000..54c346fe5 --- /dev/null +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java @@ -0,0 +1,219 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowDeserializationSchema; +import org.apache.flink.formats.json.JsonSchemaConverter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ContainerNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Time; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; + +/** + * Serialization schema that serializes an object of Flink types into a JSON bytes. + * + *

Serializes the input Flink object into a JSON string and + * converts it into byte[]. + * + *

Result byte[] messages can be deserialized using {@link JsonRowDeserializationSchema}. + */ +public class CustomerJsonRowSerializationSchema implements SerializationSchema { + + private static final long serialVersionUID = -2885556750743978636L; + + /** Type information describing the input type. */ + private final TypeInformation typeInfo; + + /** Object mapper that is used to create output JSON objects. */ + private final ObjectMapper mapper = new ObjectMapper(); + + /** Formatter for RFC 3339-compliant string representation of a time value (with UTC timezone, without milliseconds). */ + private SimpleDateFormat timeFormat = new SimpleDateFormat("HH:mm:ss'Z'"); + + /** Formatter for RFC 3339-compliant string representation of a time value (with UTC timezone). */ + private SimpleDateFormat timeFormatWithMillis = new SimpleDateFormat("HH:mm:ss.SSS'Z'"); + + /** Formatter for RFC 3339-compliant string representation of a timestamp value (with UTC timezone). */ + private SimpleDateFormat timestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); + + /** Reusable object node. */ + private transient ObjectNode node; + + private Counter counter; + + /** + * Creates a JSON serialization schema for the given type information. + * + * @param typeInfo The field names of {@link Row} are used to map to JSON properties. + */ + public CustomerJsonRowSerializationSchema(TypeInformation typeInfo) { + Preconditions.checkNotNull(typeInfo, "Type information"); + this.typeInfo = typeInfo; + } + + /** + * Creates a JSON serialization schema for the given JSON schema. + * + * @param jsonSchema JSON schema describing the result type + * + * @see http://json-schema.org/ + */ + public CustomerJsonRowSerializationSchema(String jsonSchema) { + this(JsonSchemaConverter.convert(jsonSchema)); + } + + @Override + public byte[] serialize(Row row) { + if (node == null) { + node = mapper.createObjectNode(); + } + + try { + convertRow(node, (RowTypeInfo) typeInfo, row); + counter.inc(); + return mapper.writeValueAsBytes(node); + } catch (Throwable t) { + throw new RuntimeException("Could not serialize row '" + row + "'. " + + "Make sure that the schema matches the input.", t); + } + } + + // -------------------------------------------------------------------------------------------- + + private ObjectNode convertRow(ObjectNode reuse, RowTypeInfo info, Row row) { + if (reuse == null) { + reuse = mapper.createObjectNode(); + } + final String[] fieldNames = info.getFieldNames(); + final TypeInformation[] fieldTypes = info.getFieldTypes(); + + // validate the row + if (row.getArity() != fieldNames.length) { + throw new IllegalStateException(String.format( + "Number of elements in the row '%s' is different from number of field names: %d", row, fieldNames.length)); + } + + for (int i = 0; i < fieldNames.length; i++) { + final String name = fieldNames[i]; + + final JsonNode fieldConverted = convert(reuse, reuse.get(name), fieldTypes[i], row.getField(i)); + reuse.set(name, fieldConverted); + } + + return reuse; + } + + private JsonNode convert(ContainerNode container, JsonNode reuse, TypeInformation info, Object object) { + if (info == Types.VOID || object == null) { + return container.nullNode(); + } else if (info == Types.BOOLEAN) { + return container.booleanNode((Boolean) object); + } else if (info == Types.STRING) { + return container.textNode((String) object); + } else if (info == Types.BIG_DEC) { + // convert decimal if necessary + if (object instanceof BigDecimal) { + return container.numberNode((BigDecimal) object); + } + return container.numberNode(BigDecimal.valueOf(((Number) object).doubleValue())); + } else if (info == Types.BIG_INT) { + // convert integer if necessary + if (object instanceof BigInteger) { + return container.numberNode((BigInteger) object); + } + return container.numberNode(BigInteger.valueOf(((Number) object).longValue())); + } else if (info == Types.SQL_DATE) { + return container.textNode(object.toString()); + } else if (info == Types.SQL_TIME) { + final Time time = (Time) object; + // strip milliseconds if possible + if (time.getTime() % 1000 > 0) { + return container.textNode(timeFormatWithMillis.format(time)); + } + return container.textNode(timeFormat.format(time)); + } else if (info == Types.SQL_TIMESTAMP) { + return container.textNode(timestampFormat.format((Timestamp) object)); + } else if (info instanceof RowTypeInfo) { + if (reuse != null && reuse instanceof ObjectNode) { + return convertRow((ObjectNode) reuse, (RowTypeInfo) info, (Row) object); + } else { + return convertRow(null, (RowTypeInfo) info, (Row) object); + } + } else if (info instanceof ObjectArrayTypeInfo) { + if (reuse != null && reuse instanceof ArrayNode) { + return convertObjectArray((ArrayNode) reuse, ((ObjectArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } else { + return convertObjectArray(null, ((ObjectArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } + } else if (info instanceof BasicArrayTypeInfo) { + if (reuse != null && reuse instanceof ArrayNode) { + return convertObjectArray((ArrayNode) reuse, ((BasicArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } else { + return convertObjectArray(null, ((BasicArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } + } else if (info instanceof PrimitiveArrayTypeInfo && ((PrimitiveArrayTypeInfo) info).getComponentType() == Types.BYTE) { + return container.binaryNode((byte[]) object); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return mapper.valueToTree(object); + } catch (IllegalArgumentException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for object: " + object, e); + } + } + } + + private ArrayNode convertObjectArray(ArrayNode reuse, TypeInformation info, Object[] array) { + if (reuse == null) { + reuse = mapper.createArrayNode(); + } else { + reuse.removeAll(); + } + + for (Object object : array) { + reuse.add(convert(reuse, null, info, object)); + } + return reuse; + } + + public Counter getCounter() { + return counter; + } + + public void setCounter(Counter counter) { + this.counter = counter; + } +} diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java index 8c9f31659..45dc6a331 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java @@ -19,7 +19,6 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase; import org.apache.flink.streaming.connectors.kafka.Kafka010JsonTableSink; import org.apache.flink.streaming.connectors.kafka.KafkaJsonTableSink; @@ -63,7 +62,7 @@ public CustomerKafka10JsonTableSink(String topic, Properties properties, KafkaPa @Override protected FlinkKafkaProducerBase createKafkaProducer(String topic, Properties properties, SerializationSchema serializationSchema, FlinkKafkaPartitioner partitioner) { - return new FlinkKafkaProducer010(topic, serializationSchema, properties, partitioner); + return new CustomerFlinkKafkaProducer010(topic, serializationSchema, properties); } @Override diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index a260aaa0d..427bf14cc 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -24,7 +24,6 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; import org.apache.flink.table.sinks.AppendStreamTableSink; @@ -74,7 +73,7 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { } properties.setProperty("bootstrap.servers", kafka10SinkTableInfo.getBootstrapServers()); - this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); + this.serializationSchema = new CustomerJsonRowSerializationSchema(getOutputType()); return this; } diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer011.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer011.java new file mode 100644 index 000000000..2b3424e42 --- /dev/null +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer011.java @@ -0,0 +1,124 @@ +/** + * 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 com.dtstack.flink.sql.sink.kafka; + +import com.dtstack.flink.sql.metric.MetricConstant; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; +import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricWrapper; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.Metric; +import org.apache.kafka.common.MetricName; + +import java.util.Map; +import java.util.Properties; + +/** + * Reason: + * Date: 2019/4/24 + * Company: www.dtstack.com + * + * @author maqi + */ +public class CustomerFlinkKafkaProducer011 extends FlinkKafkaProducer010 { + + CustomerJsonRowSerializationSchema schema; + + public CustomerFlinkKafkaProducer011(String topicId, SerializationSchema serializationSchema, Properties producerConfig) { + super(topicId, serializationSchema, producerConfig); + this.schema = (CustomerJsonRowSerializationSchema) serializationSchema; + } + + @Override + public void open(Configuration configuration) { + producer = getKafkaProducer(this.producerConfig); + + RuntimeContext ctx = getRuntimeContext(); + Counter counter = ctx.getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + MeterView meter = ctx.getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(counter, 20)); + + schema.setCounter(counter); + + if (null != flinkKafkaPartitioner) { + if (flinkKafkaPartitioner instanceof FlinkKafkaDelegatePartitioner) { + ((FlinkKafkaDelegatePartitioner) flinkKafkaPartitioner).setPartitions( + getPartitionsByTopic(this.defaultTopicId, this.producer)); + } + flinkKafkaPartitioner.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks()); + } + + + // register Kafka metrics to Flink accumulators + if (!Boolean.parseBoolean(producerConfig.getProperty(KEY_DISABLE_METRICS, "false"))) { + Map metrics = this.producer.metrics(); + + if (metrics == null) { + // MapR's Kafka implementation returns null here. + } else { + final MetricGroup kafkaMetricGroup = getRuntimeContext().getMetricGroup().addGroup("KafkaProducer"); + for (Map.Entry metric : metrics.entrySet()) { + kafkaMetricGroup.gauge(metric.getKey().name(), new KafkaMetricWrapper(metric.getValue())); + } + } + } + + if (flushOnCheckpoint && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) { + flushOnCheckpoint = false; + } + + if (logFailuresOnly) { + callback = new Callback() { + @Override + public void onCompletion(RecordMetadata metadata, Exception e) { + if (e != null) { + } + acknowledgeMessage(); + } + }; + } else { + callback = new Callback() { + @Override + public void onCompletion(RecordMetadata metadata, Exception exception) { + if (exception != null && asyncException == null) { + asyncException = exception; + } + acknowledgeMessage(); + } + }; + } + } + + private void acknowledgeMessage() { + if (flushOnCheckpoint) { + synchronized (pendingRecordsLock) { + pendingRecords--; + if (pendingRecords == 0) { + pendingRecordsLock.notifyAll(); + } + } + } + } +} diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java new file mode 100644 index 000000000..487d938ce --- /dev/null +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java @@ -0,0 +1,225 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka; + +import com.dtstack.flink.sql.sink.MetricOutputFormat; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.formats.json.JsonRowDeserializationSchema; +import org.apache.flink.formats.json.JsonSchemaConverter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ContainerNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Time; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; + +/** + * Serialization schema that serializes an object of Flink types into a JSON bytes. + * + *

Serializes the input Flink object into a JSON string and + * converts it into byte[]. + * + *

Result byte[] messages can be deserialized using {@link JsonRowDeserializationSchema}. + */ +@PublicEvolving +public class CustomerJsonRowSerializationSchema implements SerializationSchema { + + private static final long serialVersionUID = -2885556750743978636L; + + /** Type information describing the input type. */ + private final TypeInformation typeInfo; + + /** Object mapper that is used to create output JSON objects. */ + private final ObjectMapper mapper = new ObjectMapper(); + + /** Formatter for RFC 3339-compliant string representation of a time value (with UTC timezone, without milliseconds). */ + private SimpleDateFormat timeFormat = new SimpleDateFormat("HH:mm:ss'Z'"); + + /** Formatter for RFC 3339-compliant string representation of a time value (with UTC timezone). */ + private SimpleDateFormat timeFormatWithMillis = new SimpleDateFormat("HH:mm:ss.SSS'Z'"); + + /** Formatter for RFC 3339-compliant string representation of a timestamp value (with UTC timezone). */ + private SimpleDateFormat timestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); + + /** Reusable object node. */ + private transient ObjectNode node; + + private Counter counter; + + /** + * Creates a JSON serialization schema for the given type information. + * + * @param typeInfo The field names of {@link Row} are used to map to JSON properties. + */ + public CustomerJsonRowSerializationSchema(TypeInformation typeInfo) { + Preconditions.checkNotNull(typeInfo, "Type information"); + this.typeInfo = typeInfo; + } + + /** + * Creates a JSON serialization schema for the given JSON schema. + * + * @param jsonSchema JSON schema describing the result type + * + * @see http://json-schema.org/ + */ + public CustomerJsonRowSerializationSchema(String jsonSchema) { + this(JsonSchemaConverter.convert(jsonSchema)); + } + + @Override + public byte[] serialize(Row row) { + if (node == null) { + node = mapper.createObjectNode(); + } + + try { + convertRow(node, (RowTypeInfo) typeInfo, row); + counter.inc(); + return mapper.writeValueAsBytes(node); + } catch (Throwable t) { + throw new RuntimeException("Could not serialize row '" + row + "'. " + + "Make sure that the schema matches the input.", t); + } + } + + // -------------------------------------------------------------------------------------------- + + private ObjectNode convertRow(ObjectNode reuse, RowTypeInfo info, Row row) { + if (reuse == null) { + reuse = mapper.createObjectNode(); + } + final String[] fieldNames = info.getFieldNames(); + final TypeInformation[] fieldTypes = info.getFieldTypes(); + + // validate the row + if (row.getArity() != fieldNames.length) { + throw new IllegalStateException(String.format( + "Number of elements in the row '%s' is different from number of field names: %d", row, fieldNames.length)); + } + + for (int i = 0; i < fieldNames.length; i++) { + final String name = fieldNames[i]; + + final JsonNode fieldConverted = convert(reuse, reuse.get(name), fieldTypes[i], row.getField(i)); + reuse.set(name, fieldConverted); + } + + return reuse; + } + + private JsonNode convert(ContainerNode container, JsonNode reuse, TypeInformation info, Object object) { + if (info == Types.VOID || object == null) { + return container.nullNode(); + } else if (info == Types.BOOLEAN) { + return container.booleanNode((Boolean) object); + } else if (info == Types.STRING) { + return container.textNode((String) object); + } else if (info == Types.BIG_DEC) { + // convert decimal if necessary + if (object instanceof BigDecimal) { + return container.numberNode((BigDecimal) object); + } + return container.numberNode(BigDecimal.valueOf(((Number) object).doubleValue())); + } else if (info == Types.BIG_INT) { + // convert integer if necessary + if (object instanceof BigInteger) { + return container.numberNode((BigInteger) object); + } + return container.numberNode(BigInteger.valueOf(((Number) object).longValue())); + } else if (info == Types.SQL_DATE) { + return container.textNode(object.toString()); + } else if (info == Types.SQL_TIME) { + final Time time = (Time) object; + // strip milliseconds if possible + if (time.getTime() % 1000 > 0) { + return container.textNode(timeFormatWithMillis.format(time)); + } + return container.textNode(timeFormat.format(time)); + } else if (info == Types.SQL_TIMESTAMP) { + return container.textNode(timestampFormat.format((Timestamp) object)); + } else if (info instanceof RowTypeInfo) { + if (reuse != null && reuse instanceof ObjectNode) { + return convertRow((ObjectNode) reuse, (RowTypeInfo) info, (Row) object); + } else { + return convertRow(null, (RowTypeInfo) info, (Row) object); + } + } else if (info instanceof ObjectArrayTypeInfo) { + if (reuse != null && reuse instanceof ArrayNode) { + return convertObjectArray((ArrayNode) reuse, ((ObjectArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } else { + return convertObjectArray(null, ((ObjectArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } + } else if (info instanceof BasicArrayTypeInfo) { + if (reuse != null && reuse instanceof ArrayNode) { + return convertObjectArray((ArrayNode) reuse, ((BasicArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } else { + return convertObjectArray(null, ((BasicArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } + } else if (info instanceof PrimitiveArrayTypeInfo && ((PrimitiveArrayTypeInfo) info).getComponentType() == Types.BYTE) { + return container.binaryNode((byte[]) object); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return mapper.valueToTree(object); + } catch (IllegalArgumentException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for object: " + object, e); + } + } + } + + private ArrayNode convertObjectArray(ArrayNode reuse, TypeInformation info, Object[] array) { + if (reuse == null) { + reuse = mapper.createArrayNode(); + } else { + reuse.removeAll(); + } + + for (Object object : array) { + reuse.add(convert(reuse, null, info, object)); + } + return reuse; + } + + public Counter getCounter() { + return counter; + } + + public void setCounter(Counter counter) { + this.counter = counter; + } +} diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java index 1b4bbdd95..d9d122678 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java @@ -19,7 +19,6 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase; import org.apache.flink.streaming.connectors.kafka.Kafka010JsonTableSink; import org.apache.flink.streaming.connectors.kafka.KafkaJsonTableSink; @@ -64,7 +63,7 @@ public CustomerKafka11JsonTableSink(String topic, Properties properties, KafkaPa //TODO 暂时使用010 @Override protected FlinkKafkaProducerBase createKafkaProducer(String topic, Properties properties, SerializationSchema serializationSchema, FlinkKafkaPartitioner partitioner) { - return new FlinkKafkaProducer010(topic, serializationSchema, properties, partitioner); + return new CustomerFlinkKafkaProducer011(topic, serializationSchema, properties); } @Override diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 2ddda4901..d4cf64e9b 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -71,7 +71,7 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { properties.setProperty(key, kafka11SinkTableInfo.getKafkaParam(key)); } properties.setProperty("bootstrap.servers", kafka11SinkTableInfo.getBootstrapServers()); - this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); + this.serializationSchema = new CustomerJsonRowSerializationSchema(getOutputType()); return this; } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java index 75c5c4f0f..3da1717d9 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java @@ -106,10 +106,10 @@ public LauncherOptionParser(String[] args) { String localPlugin = Preconditions.checkNotNull(cl.getOptionValue(OPTION_LOCAL_SQL_PLUGIN_PATH)); properties.setLocalSqlPluginPath(localPlugin); String remotePlugin = cl.getOptionValue(OPTION_REMOTE_SQL_PLUGIN_PATH); - if(!ClusterMode.local.name().equals(mode)){ - Preconditions.checkNotNull(remotePlugin); +// if(!ClusterMode.local.name().equals(mode)){ +// Preconditions.checkNotNull(remotePlugin); properties.setRemoteSqlPluginPath(remotePlugin); - } +// } String name = Preconditions.checkNotNull(cl.getOptionValue(OPTION_NAME)); properties.setName(name); String addJar = cl.getOptionValue(OPTION_ADDJAR); diff --git a/pom.xml b/pom.xml index 0e2d9e709..c475de687 100644 --- a/pom.xml +++ b/pom.xml @@ -13,16 +13,16 @@ kafka09 kafka10 kafka11 - mysql - hbase - elasticsearch5 - mongo - redis5 - launcher - rdb - sqlserver - oracle - cassandra + + + + + + + + + + From 82225d5e8a2e23ae41526f3dd88630b8020440c1 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Fri, 26 Apr 2019 11:52:35 +0800 Subject: [PATCH 277/470] format kafka sink --- .../kafka/CustomerFlinkKafkaProducer09.java | 58 +----------------- .../kafka/CustomerFlinkKafkaProducer010.java | 59 +------------------ .../kafka/CustomerFlinkKafkaProducer011.java | 58 +----------------- pom.xml | 20 +++---- 4 files changed, 14 insertions(+), 181 deletions(-) diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer09.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer09.java index 09886b7e9..b026bf2c6 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer09.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer09.java @@ -62,63 +62,7 @@ public void open(Configuration configuration) { schema.setCounter(counter); - if (null != flinkKafkaPartitioner) { - if (flinkKafkaPartitioner instanceof FlinkKafkaDelegatePartitioner) { - ((FlinkKafkaDelegatePartitioner) flinkKafkaPartitioner).setPartitions( - getPartitionsByTopic(this.defaultTopicId, this.producer)); - } - flinkKafkaPartitioner.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks()); - } - - - // register Kafka metrics to Flink accumulators - if (!Boolean.parseBoolean(producerConfig.getProperty(KEY_DISABLE_METRICS, "false"))) { - Map metrics = this.producer.metrics(); - - if (metrics == null) { - // MapR's Kafka implementation returns null here. - } else { - final MetricGroup kafkaMetricGroup = getRuntimeContext().getMetricGroup().addGroup("KafkaProducer"); - for (Map.Entry metric : metrics.entrySet()) { - kafkaMetricGroup.gauge(metric.getKey().name(), new KafkaMetricWrapper(metric.getValue())); - } - } - } - - if (flushOnCheckpoint && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) { - flushOnCheckpoint = false; - } - - if (logFailuresOnly) { - callback = new Callback() { - @Override - public void onCompletion(RecordMetadata metadata, Exception e) { - if (e != null) { - } - acknowledgeMessage(); - } - }; - } else { - callback = new Callback() { - @Override - public void onCompletion(RecordMetadata metadata, Exception exception) { - if (exception != null && asyncException == null) { - asyncException = exception; - } - acknowledgeMessage(); - } - }; - } + super.open(configuration); } - private void acknowledgeMessage() { - if (flushOnCheckpoint) { - synchronized (pendingRecordsLock) { - pendingRecords--; - if (pendingRecords == 0) { - pendingRecordsLock.notifyAll(); - } - } - } - } } diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer010.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer010.java index dcda22f39..50da17099 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer010.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer010.java @@ -62,63 +62,8 @@ public void open(Configuration configuration) { schema.setCounter(counter); - if (null != flinkKafkaPartitioner) { - if (flinkKafkaPartitioner instanceof FlinkKafkaDelegatePartitioner) { - ((FlinkKafkaDelegatePartitioner) flinkKafkaPartitioner).setPartitions( - getPartitionsByTopic(this.defaultTopicId, this.producer)); - } - flinkKafkaPartitioner.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks()); - } - - - // register Kafka metrics to Flink accumulators - if (!Boolean.parseBoolean(producerConfig.getProperty(KEY_DISABLE_METRICS, "false"))) { - Map metrics = this.producer.metrics(); - - if (metrics == null) { - // MapR's Kafka implementation returns null here. - } else { - final MetricGroup kafkaMetricGroup = getRuntimeContext().getMetricGroup().addGroup("KafkaProducer"); - for (Map.Entry metric : metrics.entrySet()) { - kafkaMetricGroup.gauge(metric.getKey().name(), new KafkaMetricWrapper(metric.getValue())); - } - } - } - - if (flushOnCheckpoint && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) { - flushOnCheckpoint = false; - } - - if (logFailuresOnly) { - callback = new Callback() { - @Override - public void onCompletion(RecordMetadata metadata, Exception e) { - if (e != null) { - } - acknowledgeMessage(); - } - }; - } else { - callback = new Callback() { - @Override - public void onCompletion(RecordMetadata metadata, Exception exception) { - if (exception != null && asyncException == null) { - asyncException = exception; - } - acknowledgeMessage(); - } - }; - } + super.open(configuration); } - private void acknowledgeMessage() { - if (flushOnCheckpoint) { - synchronized (pendingRecordsLock) { - pendingRecords--; - if (pendingRecords == 0) { - pendingRecordsLock.notifyAll(); - } - } - } - } + } diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer011.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer011.java index 2b3424e42..fba215a39 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer011.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer011.java @@ -62,63 +62,7 @@ public void open(Configuration configuration) { schema.setCounter(counter); - if (null != flinkKafkaPartitioner) { - if (flinkKafkaPartitioner instanceof FlinkKafkaDelegatePartitioner) { - ((FlinkKafkaDelegatePartitioner) flinkKafkaPartitioner).setPartitions( - getPartitionsByTopic(this.defaultTopicId, this.producer)); - } - flinkKafkaPartitioner.open(ctx.getIndexOfThisSubtask(), ctx.getNumberOfParallelSubtasks()); - } - - - // register Kafka metrics to Flink accumulators - if (!Boolean.parseBoolean(producerConfig.getProperty(KEY_DISABLE_METRICS, "false"))) { - Map metrics = this.producer.metrics(); - - if (metrics == null) { - // MapR's Kafka implementation returns null here. - } else { - final MetricGroup kafkaMetricGroup = getRuntimeContext().getMetricGroup().addGroup("KafkaProducer"); - for (Map.Entry metric : metrics.entrySet()) { - kafkaMetricGroup.gauge(metric.getKey().name(), new KafkaMetricWrapper(metric.getValue())); - } - } - } - - if (flushOnCheckpoint && !((StreamingRuntimeContext) this.getRuntimeContext()).isCheckpointingEnabled()) { - flushOnCheckpoint = false; - } - - if (logFailuresOnly) { - callback = new Callback() { - @Override - public void onCompletion(RecordMetadata metadata, Exception e) { - if (e != null) { - } - acknowledgeMessage(); - } - }; - } else { - callback = new Callback() { - @Override - public void onCompletion(RecordMetadata metadata, Exception exception) { - if (exception != null && asyncException == null) { - asyncException = exception; - } - acknowledgeMessage(); - } - }; - } + super.open(configuration); } - private void acknowledgeMessage() { - if (flushOnCheckpoint) { - synchronized (pendingRecordsLock) { - pendingRecords--; - if (pendingRecords == 0) { - pendingRecordsLock.notifyAll(); - } - } - } - } } diff --git a/pom.xml b/pom.xml index c475de687..0e2d9e709 100644 --- a/pom.xml +++ b/pom.xml @@ -13,16 +13,16 @@ kafka09 kafka10 kafka11 - - - - - - - - - - + mysql + hbase + elasticsearch5 + mongo + redis5 + launcher + rdb + sqlserver + oracle + cassandra From ed09696b922ee9dede64907731d7e09a2d605092 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 7 May 2019 10:22:29 +0800 Subject: [PATCH 278/470] add queue parameter --- .../com/dtstack/flink/sql/launcher/LauncherMain.java | 3 ++- .../flink/sql/launcher/LauncherOptionParser.java | 10 +++++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 0917b9eda..f2d884778 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -60,7 +60,8 @@ public class LauncherMain { private static String getLocalCoreJarPath(String localSqlRootJar) throws Exception { String jarPath = PluginUtil.getCoreJarFileName(localSqlRootJar, CORE_JAR); - return jarPath; + String corePath = localSqlRootJar + SP + jarPath; + return corePath; } public static void main(String[] args) throws Exception { diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java index 3da1717d9..75cd21caf 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java @@ -67,6 +67,8 @@ public class LauncherOptionParser { public static final String OPTION_FLINK_JAR_PATH = "flinkJarPath"; + public static final String OPTION_QUEUE = "queue"; + private Options options = new Options(); private BasicParser parser = new BasicParser(); @@ -87,6 +89,7 @@ public LauncherOptionParser(String[] args) { options.addOption(OPTION_SAVE_POINT_PATH, true, "Savepoint restore path"); options.addOption(OPTION_ALLOW_NON_RESTORED_STATE, true, "Flag indicating whether non restored state is allowed if the savepoint"); options.addOption(OPTION_FLINK_JAR_PATH, true, "flink jar path for submit of perjob mode"); + options.addOption(OPTION_QUEUE, true, "flink jar path for submit of perjob mode"); try { CommandLine cl = parser.parse(options, args); @@ -145,6 +148,10 @@ public LauncherOptionParser(String[] args) { properties.setFlinkJarPath(flinkJarPath); } + String queue = cl.getOptionValue(OPTION_QUEUE); + if(StringUtils.isNotBlank(queue)){ + properties.setQueue(queue); + } } catch (Exception e) { throw new RuntimeException(e); } @@ -161,7 +168,8 @@ public List getProgramExeArgList() throws Exception { String key = one.getKey(); if(OPTION_FLINK_CONF_DIR.equalsIgnoreCase(key) || OPTION_YARN_CONF_DIR.equalsIgnoreCase(key) - || OPTION_FLINK_JAR_PATH.equalsIgnoreCase(key)){ + || OPTION_FLINK_JAR_PATH.equalsIgnoreCase(key) + || OPTION_QUEUE.equalsIgnoreCase(key)){ continue; } From 1912e5d2416c1841085bee68e5fa0bfee635478f Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 7 May 2019 10:29:19 +0800 Subject: [PATCH 279/470] fix description --- .../com/dtstack/flink/sql/launcher/LauncherOptionParser.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java index 75cd21caf..c658cd895 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java @@ -89,7 +89,7 @@ public LauncherOptionParser(String[] args) { options.addOption(OPTION_SAVE_POINT_PATH, true, "Savepoint restore path"); options.addOption(OPTION_ALLOW_NON_RESTORED_STATE, true, "Flag indicating whether non restored state is allowed if the savepoint"); options.addOption(OPTION_FLINK_JAR_PATH, true, "flink jar path for submit of perjob mode"); - options.addOption(OPTION_QUEUE, true, "flink jar path for submit of perjob mode"); + options.addOption(OPTION_QUEUE, true, "flink runing yarn queue"); try { CommandLine cl = parser.parse(options, args); From e6b5a3670086191a7b13da09ed2de92012e35cfc Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 7 May 2019 16:06:20 +0800 Subject: [PATCH 280/470] =?UTF-8?q?=E4=BF=AE=E5=A4=8D=20yarn=20perjob?= =?UTF-8?q?=E6=A8=A1=E5=BC=8Fclass=E5=8A=A0=E8=BD=BD=E4=B8=8D=E5=88=B0?= =?UTF-8?q?=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../sql/launcher/perjob/PerJobSubmitter.java | 44 ++++++++++++++++++- 1 file changed, 43 insertions(+), 1 deletion(-) diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java index 3cdf3f8d4..77672826c 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java @@ -20,15 +20,24 @@ import com.dtstack.flink.sql.launcher.LauncherOptions; import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.commons.io.Charsets; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.shaded.guava18.com.google.common.base.Strings; +import org.apache.flink.shaded.guava18.com.google.common.collect.Sets; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.MalformedURLException; +import java.net.URL; +import java.net.URLDecoder; +import java.util.Map; import java.util.Properties; +import java.util.Set; /** * per job mode submitter @@ -43,7 +52,11 @@ public class PerJobSubmitter { public static String submit(LauncherOptions launcherOptions, JobGraph jobGraph) throws Exception { - Properties confProperties = PluginUtil.jsonStrToObject(launcherOptions.getConfProp(), Properties.class); + fillJobGraphClassPath(jobGraph); + + String confProp = launcherOptions.getConfProp(); + confProp = URLDecoder.decode(confProp, Charsets.UTF_8.toString()); + Properties confProperties = PluginUtil.jsonStrToObject(confProp, Properties.class); ClusterSpecification clusterSpecification = FLinkPerJobResourceUtil.createClusterSpecification(confProperties); PerJobClusterClientBuilder perJobClusterClientBuilder = new PerJobClusterClientBuilder(); @@ -63,4 +76,33 @@ public static String submit(LauncherOptions launcherOptions, JobGraph jobGraph) return applicationId; } + + private static void fillJobGraphClassPath(JobGraph jobGraph) throws MalformedURLException { + Map jobCacheFileConfig = jobGraph.getJobConfiguration().toMap(); + Set classPathKeySet = Sets.newHashSet(); + + for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ + if(Strings.isNullOrEmpty(tmp.getValue())){ + continue; + } + + if(tmp.getValue().startsWith("class_path")){ + //DISTRIBUTED_CACHE_FILE_NAME_1 + //DISTRIBUTED_CACHE_FILE_PATH_1 + String key = tmp.getKey(); + String[] array = key.split("_"); + if(array.length < 5){ + continue; + } + + array[3] = "PATH"; + classPathKeySet.add(StringUtils.join(array, "_")); + } + } + + for(String key : classPathKeySet){ + String pathStr = jobCacheFileConfig.get(key); + jobGraph.getClasspaths().add(new URL("file:" + pathStr)); + } + } } From 7dd42b88ce526894b7fd7e008f68c3ab2cb361bc Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 7 May 2019 18:53:46 +0800 Subject: [PATCH 281/470] add udf jar --- .../sql/launcher/perjob/PerJobSubmitter.java | 82 +++++++++++-------- 1 file changed, 49 insertions(+), 33 deletions(-) diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java index 77672826c..0708b4957 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java @@ -24,6 +24,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.shaded.guava18.com.google.common.base.Strings; import org.apache.flink.shaded.guava18.com.google.common.collect.Sets; @@ -35,9 +36,7 @@ import java.net.MalformedURLException; import java.net.URL; import java.net.URLDecoder; -import java.util.Map; -import java.util.Properties; -import java.util.Set; +import java.util.*; /** * per job mode submitter @@ -52,9 +51,18 @@ public class PerJobSubmitter { public static String submit(LauncherOptions launcherOptions, JobGraph jobGraph) throws Exception { - fillJobGraphClassPath(jobGraph); + fillJobGraphClassPath(jobGraph); - String confProp = launcherOptions.getConfProp(); + String addjarPath = URLDecoder.decode(launcherOptions.getAddjar(), Charsets.UTF_8.toString()); + if (StringUtils.isNotBlank(addjarPath) ){ + List paths = getJarPaths(addjarPath); + paths.forEach( path ->{ + jobGraph.addJar(new Path("file://" + path)); + }); + + } + + String confProp = launcherOptions.getConfProp(); confProp = URLDecoder.decode(confProp, Charsets.UTF_8.toString()); Properties confProperties = PluginUtil.jsonStrToObject(confProp, Properties.class); ClusterSpecification clusterSpecification = FLinkPerJobResourceUtil.createClusterSpecification(confProperties); @@ -77,32 +85,40 @@ public static String submit(LauncherOptions launcherOptions, JobGraph jobGraph) return applicationId; } - private static void fillJobGraphClassPath(JobGraph jobGraph) throws MalformedURLException { - Map jobCacheFileConfig = jobGraph.getJobConfiguration().toMap(); - Set classPathKeySet = Sets.newHashSet(); - - for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ - if(Strings.isNullOrEmpty(tmp.getValue())){ - continue; - } - - if(tmp.getValue().startsWith("class_path")){ - //DISTRIBUTED_CACHE_FILE_NAME_1 - //DISTRIBUTED_CACHE_FILE_PATH_1 - String key = tmp.getKey(); - String[] array = key.split("_"); - if(array.length < 5){ - continue; - } - - array[3] = "PATH"; - classPathKeySet.add(StringUtils.join(array, "_")); - } - } - - for(String key : classPathKeySet){ - String pathStr = jobCacheFileConfig.get(key); - jobGraph.getClasspaths().add(new URL("file:" + pathStr)); - } - } + private static List getJarPaths(String addjarPath) { + if (addjarPath.length() > 2) { + addjarPath = addjarPath.substring(1,addjarPath.length()-1).replace("\"",""); + } + List paths = Arrays.asList(addjarPath.split(",")); + return paths; + } + + private static void fillJobGraphClassPath(JobGraph jobGraph) throws MalformedURLException { + Map jobCacheFileConfig = jobGraph.getJobConfiguration().toMap(); + Set classPathKeySet = Sets.newHashSet(); + + for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ + if(Strings.isNullOrEmpty(tmp.getValue())){ + continue; + } + + if(tmp.getValue().startsWith("class_path")){ + //DISTRIBUTED_CACHE_FILE_NAME_1 + //DISTRIBUTED_CACHE_FILE_PATH_1 + String key = tmp.getKey(); + String[] array = key.split("_"); + if(array.length < 5){ + continue; + } + + array[3] = "PATH"; + classPathKeySet.add(StringUtils.join(array, "_")); + } + } + + for(String key : classPathKeySet){ + String pathStr = jobCacheFileConfig.get(key); + jobGraph.getClasspaths().add(new URL("file:" + pathStr)); + } + } } From 695d9ace30bc79707ab641ee77fe20c532a06cca Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Wed, 29 May 2019 16:14:55 +0800 Subject: [PATCH 282/470] quote view error --- .../com/dtstack/flink/sql/parser/SqlParser.java | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java index 7ab616632..0f9f8ffd7 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java @@ -138,12 +138,15 @@ public static SqlTree parseSql(String sql) throws Exception { if (!sqlTree.getTableInfoMap().keySet().contains(tableName)){ CreateTableParser.SqlParserResult createTableResult = sqlTree.getPreDealTableMap().get(tableName); if(createTableResult == null){ - throw new RuntimeException("can't find table " + tableName); + CreateTmpTableParser.SqlParserResult tmpTableResult = sqlTree.getTmpTableMap().get(tableName); + if (tmpTableResult == null){ + throw new RuntimeException("can't find table " + tableName); + } + } else { + TableInfo tableInfo = tableInfoParser.parseWithTableType(ETableType.SOURCE.getType(), + createTableResult, LOCAL_SQL_PLUGIN_ROOT); + sqlTree.addTableInfo(tableName, tableInfo); } - - TableInfo tableInfo = tableInfoParser.parseWithTableType(ETableType.SOURCE.getType(), - createTableResult, LOCAL_SQL_PLUGIN_ROOT); - sqlTree.addTableInfo(tableName, tableInfo); } } } From bae79ac81f8481b0de18038ca9bba252f71dffcf Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 3 Jun 2019 12:03:01 +0800 Subject: [PATCH 283/470] ignore miss field --- .../flink/sql/source/kafka/CustomerJsonDeserialization.java | 4 ++++ .../flink/sql/source/kafka/CustomerJsonDeserialization.java | 4 ++++ .../flink/sql/source/kafka/CustomerJsonDeserialization.java | 4 ++++ 3 files changed, 12 insertions(+) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 9ee70af04..574fe65cd 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -150,6 +150,10 @@ public void setFailOnMissingField(boolean failOnMissingField) { private JsonNode getIgnoreCase(String key) { String nodeMappingKey = rowAndFieldMapping.getOrDefault(key, key); JsonNode node = nodeAndJsonNodeMapping.get(nodeMappingKey); + if(node == null){ + return null; + } + JsonNodeType nodeType = node.getNodeType(); if (nodeType == JsonNodeType.ARRAY){ diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index d72083762..e24f987f0 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -145,6 +145,10 @@ public Row deserialize(byte[] message) throws IOException { public JsonNode getIgnoreCase(String key) { String nodeMappingKey = rowAndFieldMapping.getOrDefault(key, key); JsonNode node = nodeAndJsonNodeMapping.get(nodeMappingKey); + if(node == null){ + return null; + } + JsonNodeType nodeType = node.getNodeType(); if (nodeType==JsonNodeType.ARRAY){ diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index deb600337..123d9d5cc 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -148,6 +148,10 @@ public Row deserialize(byte[] message) throws IOException { public JsonNode getIgnoreCase(String key) { String nodeMappingKey = rowAndFieldMapping.getOrDefault(key, key); JsonNode node = nodeAndJsonNodeMapping.get(nodeMappingKey); + if(node == null){ + return null; + } + JsonNodeType nodeType = node.getNodeType(); if (nodeType==JsonNodeType.ARRAY){ From 02fe42d76326463b9eee1acf3ee09edbb11c6bcb Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 5 Jun 2019 11:51:36 +0800 Subject: [PATCH 284/470] =?UTF-8?q?json=20array=20=E7=9A=84=E8=AF=BB?= =?UTF-8?q?=E5=8F=96=E4=B8=BAtext?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/source/kafka/CustomerJsonDeserialization.java | 3 +++ .../flink/sql/source/kafka/CustomerJsonDeserialization.java | 3 +++ .../flink/sql/source/kafka/CustomerJsonDeserialization.java | 3 +++ 3 files changed, 9 insertions(+) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 574fe65cd..aecb4e9d4 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -31,6 +31,7 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.types.Row; @@ -174,6 +175,8 @@ private void parseTree(JsonNode jsonNode, String prefix){ if (child.isValueNode()){ nodeAndJsonNodeMapping.put(nodeKey, child); + } else if(child.isArray()){ + nodeAndJsonNodeMapping.put(nodeKey, new TextNode(child.toString())); }else { parseTree(child, nodeKey); } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 321cc01aa..daa487a07 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -31,6 +31,7 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.types.Row; @@ -174,6 +175,8 @@ private void parseTree(JsonNode jsonNode, String prefix){ if (child.isValueNode()){ nodeAndJsonNodeMapping.put(nodeKey, child); + }else if(child.isArray()){ + nodeAndJsonNodeMapping.put(nodeKey, new TextNode(child.toString())); }else { parseTree(child, nodeKey); } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 123d9d5cc..26a27773b 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -31,6 +31,7 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.types.Row; @@ -175,6 +176,8 @@ private void parseTree(JsonNode jsonNode, String prefix){ if (child.isValueNode()){ nodeAndJsonNodeMapping.put(nodeKey, child); + }else if(child.isArray()){ + nodeAndJsonNodeMapping.put(nodeKey, new TextNode(child.toString())); }else { parseTree(child, nodeKey); } From 99aba293ab10261c9d01395446fe3e15aa22db02 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Wed, 12 Jun 2019 12:02:17 +0800 Subject: [PATCH 285/470] support @ --- .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 +- .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 +- .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index e121040a1..c950c9ec7 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -41,7 +41,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((\\w+\\.)@*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index dc3ab7a28..e7f6154b0 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -40,7 +40,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((\\w+\\.)@*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 5b311728b..ad5a40d78 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -40,7 +40,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((\\w+\\.)@*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); From 2f0fa9da737b9961a2c716991a18f2ddf7744c24 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Fri, 14 Jun 2019 11:52:02 +0800 Subject: [PATCH 286/470] add udf --- .../dtstack/flink/sql/udf/TimestampUdf.java | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) create mode 100644 core/src/main/java/com/dtstack/flink/sql/udf/TimestampUdf.java diff --git a/core/src/main/java/com/dtstack/flink/sql/udf/TimestampUdf.java b/core/src/main/java/com/dtstack/flink/sql/udf/TimestampUdf.java new file mode 100644 index 000000000..9f605dde3 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/udf/TimestampUdf.java @@ -0,0 +1,24 @@ +package com.dtstack.flink.sql.udf; + +import org.apache.flink.table.functions.FunctionContext; +import org.apache.flink.table.functions.ScalarFunction; + +import java.sql.Timestamp; + +public class TimestampUdf extends ScalarFunction { + @Override + public void open(FunctionContext context) { + } + public static Timestamp eval(String timestamp) { + if (timestamp.length() == 13){ + return new Timestamp(Long.parseLong(timestamp)); + }else if (timestamp.length() == 10){ + return new Timestamp(Long.parseLong(timestamp)*1000); + } else{ + return Timestamp.valueOf(timestamp); + } + } + @Override + public void close() { + } +} From 395c039dcd75c8497a4cd895cb7b332d910e4e97 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Tue, 18 Jun 2019 16:07:46 +0800 Subject: [PATCH 287/470] support flink180 --- core/pom.xml | 18 +++++++++++++++--- .../environment/MyLocalStreamEnvironment.java | 13 ++++++++----- .../flink/yarn/YarnClusterDescriptor.java | 14 +++++++++----- .../sink/kafka/CustomerCsvSerialization.java | 11 +++-------- .../flink/sql/sink/kafka/KafkaSink.java | 2 +- .../sink/kafka/CustomerCsvSerialization.java | 11 +++-------- .../flink/sql/sink/kafka/KafkaSink.java | 2 +- .../sink/kafka/CustomerCsvSerialization.java | 11 +++-------- .../flink/sql/sink/kafka/KafkaSink.java | 2 +- .../sink/kafka/CustomerCsvSerialization.java | 11 +++-------- .../flink/sql/sink/kafka/KafkaSink.java | 2 +- .../sql/launcher/ClusterClientFactory.java | 9 +++++++-- .../perjob/PerJobClusterClientBuilder.java | 2 +- pom.xml | 2 +- 14 files changed, 57 insertions(+), 53 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 300bc7bd7..3fbcd3125 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -52,10 +52,22 @@ ${flink.version} - + + + + org.apache.flink + flink-table-planner_2.11 + 1.8.0 + + + + org.apache.flink + flink-table-common + 1.8.0 @@ -80,7 +92,7 @@ org.apache.flink flink-shaded-hadoop2 - ${flink.version} + 1.7.2 diff --git a/core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java b/core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java index c1cea1e14..8ac1edd41 100644 --- a/core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java +++ b/core/src/main/java/com/dtstack/flink/sql/environment/MyLocalStreamEnvironment.java @@ -21,11 +21,11 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.graph.StreamGraph; @@ -106,18 +106,21 @@ public JobExecutionResult execute(String jobName) throws Exception { // add (and override) the settings with what the user defined configuration.addAll(this.conf); + MiniClusterConfiguration.Builder configBuilder = new MiniClusterConfiguration.Builder(); + configBuilder.setConfiguration(configuration); + if (LOG.isInfoEnabled()) { LOG.info("Running job on local embedded Flink mini cluster"); } - LocalFlinkMiniCluster exec = new LocalFlinkMiniCluster(configuration, true); + MiniCluster exec = new MiniCluster(configBuilder.build()); try { exec.start(); - return exec.submitJobAndWait(jobGraph, getConfig().isSysoutLoggingEnabled()); + return exec.executeJobBlocking(jobGraph); } finally { transformations.clear(); - exec.stop(); + exec.closeAsync(); } } } diff --git a/core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java b/core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java index d442d16f2..4e8987057 100644 --- a/core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java +++ b/core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.yarn.*; +import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -86,7 +87,7 @@ public YarnClusterDescriptor( @Override protected String getYarnSessionClusterEntrypoint() { - return YarnApplicationMasterRunner.class.getName(); + return YarnSessionClusterEntrypoint.class.getName(); } /** @@ -95,7 +96,7 @@ protected String getYarnSessionClusterEntrypoint() @Override protected String getYarnJobClusterEntrypoint() { - return YarnApplicationMasterRunner.class.getName(); + return YarnSessionClusterEntrypoint.class.getName(); } @Override @@ -113,7 +114,7 @@ public YarnClient getYarnClient() return this.yarnClient; } - public YarnClusterClient deploy() + public RestClusterClient deploy() { ApplicationSubmissionContext context = Records.newRecord(ApplicationSubmissionContext.class); context.setApplicationId(yarnAppId); @@ -124,10 +125,13 @@ public YarnClusterClient deploy() conf.setString(JobManagerOptions.ADDRESS.key(), report.getHost()); conf.setInteger(JobManagerOptions.PORT.key(), report.getRpcPort()); - return new YarnClusterClient(this, + /*return new RestClusterClient(this, appConf.getTaskManagerCount(), appConf.getTaskManagerSlots(), - report, conf, false); + report, conf, false);*/ + return new RestClusterClient<>( + conf, + report.getApplicationId()); } catch (Exception e) { throw new RuntimeException(e); diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 095b950cb..4168edbd1 100644 --- a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -5,6 +5,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -116,13 +117,7 @@ public void copy(DataInputView source, DataOutputView target) throws IOException } @Override - public boolean canEqual(Object obj) { - return obj instanceof CustomerCsvSerialization; - } - - @Override - protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { - return super.isCompatibleSerializationFormatIdentifier(identifier) - || identifier.equals(StringValue.class.getCanonicalName()); + public TypeSerializerSnapshot snapshotConfiguration() { + return null; } } diff --git a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index e5a7b9300..fe3c00f03 100644 --- a/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka08/kafka08-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -25,12 +25,12 @@ import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.Kafka08TableSink; import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.sinks.TableSink; diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 354840ea4..bccf7ac9e 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -4,6 +4,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -113,13 +114,7 @@ public void copy(DataInputView source, DataOutputView target) throws IOException } @Override - public boolean canEqual(Object obj) { - return obj instanceof CustomerCsvSerialization; - } - - @Override - protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { - return super.isCompatibleSerializationFormatIdentifier(identifier) - || identifier.equals(StringValue.class.getCanonicalName()); + public TypeSerializerSnapshot snapshotConfiguration() { + return null; } } diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index db4cf3d1b..4c1940440 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -25,12 +25,12 @@ import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.Kafka09TableSink; import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.sinks.TableSink; diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 09377e376..e39a2d3b6 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -4,6 +4,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -115,13 +116,7 @@ public void copy(DataInputView source, DataOutputView target) throws IOException } @Override - public boolean canEqual(Object obj) { - return obj instanceof CustomerCsvSerialization; - } - - @Override - protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { - return super.isCompatibleSerializationFormatIdentifier(identifier) - || identifier.equals(StringValue.class.getCanonicalName()); + public TypeSerializerSnapshot snapshotConfiguration() { + return null; } } diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 52214594c..04accff58 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -25,12 +25,12 @@ import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.Kafka010TableSink; import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.sinks.TableSink; diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 480133662..032d34bb2 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -131,13 +132,7 @@ public void copy(DataInputView source, DataOutputView target) throws IOException } @Override - public boolean canEqual(Object obj) { - return obj instanceof CustomerCsvSerialization; - } - - @Override - protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { - return super.isCompatibleSerializationFormatIdentifier(identifier) - || identifier.equals(StringValue.class.getCanonicalName()); + public TypeSerializerSnapshot snapshotConfiguration() { + return null; } } diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 3d55088d0..a26014162 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -25,13 +25,13 @@ import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.Kafka011TableSink; import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.sinks.TableSink; diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 4513649fa..032edb7d4 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -24,13 +24,15 @@ import org.apache.commons.io.Charsets; import org.apache.commons.lang.StringUtils; import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.client.program.StandaloneClusterClient; +import org.apache.flink.client.program.MiniClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; @@ -72,7 +74,10 @@ public static ClusterClient createClusterClient(LauncherOptions launcherOptions) public static ClusterClient createStandaloneClient(LauncherOptions launcherOptions) throws Exception { String flinkConfDir = launcherOptions.getFlinkconf(); Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); - StandaloneClusterClient clusterClient = new StandaloneClusterClient(config); + MiniClusterConfiguration.Builder configBuilder = new MiniClusterConfiguration.Builder(); + configBuilder.setConfiguration(config); + MiniCluster miniCluster = new MiniCluster(configBuilder.build()); + MiniClusterClient clusterClient = new MiniClusterClient(config, miniCluster); LeaderConnectionInfo connectionInfo = clusterClient.getClusterConnectionInfo(); InetSocketAddress address = AkkaUtils.getInetSocketAddressFromAkkaURL(connectionInfo.getAddress()); config.setString(JobManagerOptions.ADDRESS, address.getAddress().getHostName()); diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java index 8ddcd541d..1f1b7d679 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java @@ -91,7 +91,7 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties co throw new RuntimeException("The Flink jar path is null"); } - clusterDescriptor.setProvidedUserJarFiles(classpaths); + //clusterDescriptor.setProvidedUserJarFiles(classpaths); if(!Strings.isNullOrEmpty(queue)){ clusterDescriptor.setQueue(queue); diff --git a/pom.xml b/pom.xml index eaf679c2d..5659dbd11 100644 --- a/pom.xml +++ b/pom.xml @@ -31,7 +31,7 @@ UTF-8 - 1.7.0 + 1.8.0 From 3e9db2da3a4d88b1c808b3ce1a9e8d316c886b6a Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Wed, 19 Jun 2019 16:37:46 +0800 Subject: [PATCH 288/470] support else in select --- core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 3623ecfd4..f92552fac 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -435,7 +435,7 @@ private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable Date: Thu, 20 Jun 2019 16:41:54 +0800 Subject: [PATCH 289/470] field parser bug support @ --- .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 +- .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 +- .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index c950c9ec7..ab9944589 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -41,7 +41,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((\\w+\\.)@*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index e7f6154b0..e4f34237b 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -40,7 +40,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((\\w+\\.)@*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index ad5a40d78..dcd0052ae 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -40,7 +40,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((\\w+\\.)@*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); From 55c81853ba0b9e3de94d010c817e2720ff2feffa Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 20 Jun 2019 17:01:37 +0800 Subject: [PATCH 290/470] modify field parser bug --- .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 +- .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 +- .../dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index c950c9ec7..ab9944589 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -41,7 +41,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((\\w+\\.)@*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index e7f6154b0..e4f34237b 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -40,7 +40,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((\\w+\\.)@*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index ad5a40d78..dcd0052ae 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -40,7 +40,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((\\w+\\.)@*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); From a076309bdbf44d70d059f27a717828509e93bdc5 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 24 Jun 2019 21:58:41 +0800 Subject: [PATCH 291/470] add queue config --- core/pom.xml | 4 ++-- .../flink/sql/options/LauncherOptionParser.java | 10 +++++++++- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 3fbcd3125..7937d4884 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -61,13 +61,13 @@ org.apache.flink flink-table-planner_2.11 - 1.8.0 + ${flink.version} org.apache.flink flink-table-common - 1.8.0 + ${flink.version} diff --git a/core/src/main/java/com/dtstack/flink/sql/options/LauncherOptionParser.java b/core/src/main/java/com/dtstack/flink/sql/options/LauncherOptionParser.java index d4b4a19f2..fdf457060 100644 --- a/core/src/main/java/com/dtstack/flink/sql/options/LauncherOptionParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/options/LauncherOptionParser.java @@ -67,6 +67,8 @@ public class LauncherOptionParser { public static final String OPTION_FLINK_JAR_PATH = "flinkJarPath"; + public static final String OPTION_QUEUE = "queue"; + private Options options = new Options(); private BasicParser parser = new BasicParser(); @@ -87,6 +89,7 @@ public LauncherOptionParser(String[] args) throws Exception { options.addOption(OPTION_SAVE_POINT_PATH, true, "Savepoint restore path"); options.addOption(OPTION_ALLOW_NON_RESTORED_STATE, true, "Flag indicating whether non restored state is allowed if the savepoint"); options.addOption(OPTION_FLINK_JAR_PATH, true, "flink jar path for submit of perjob mode"); + options.addOption(OPTION_QUEUE, true, "flink runing yarn queue"); CommandLine cl = parser.parse(options, args); String mode = cl.getOptionValue(OPTION_MODE, ClusterMode.local.name()); //check mode @@ -145,6 +148,10 @@ public LauncherOptionParser(String[] args) throws Exception { if(StringUtils.isNotBlank(flinkJarPath)){ properties.setFlinkJarPath(flinkJarPath); } + String queue = cl.getOptionValue(OPTION_QUEUE); + if(StringUtils.isNotBlank(queue)){ + properties.setQueue(queue); + } } public LauncherOptions getLauncherOptions(){ @@ -159,7 +166,8 @@ public List getProgramExeArgList() throws Exception { String key = one.getKey(); if(OPTION_FLINK_CONF_DIR.equalsIgnoreCase(key) || OPTION_YARN_CONF_DIR.equalsIgnoreCase(key) - || OPTION_FLINK_JAR_PATH.equalsIgnoreCase(key)){ + || OPTION_FLINK_JAR_PATH.equalsIgnoreCase(key) + || OPTION_QUEUE.equalsIgnoreCase(key)){ continue; } From 9dce5d9116a36070e144a9d35941988be288574c Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 27 Jun 2019 16:28:35 +0800 Subject: [PATCH 292/470] modify --- core/src/main/java/com/dtstack/flink/sql/Main.java | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 37ffa4df4..e3c2aa9f8 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -139,15 +139,8 @@ public static void main(String[] args) throws Exception { } ClassLoader threadClassLoader = Thread.currentThread().getContextClassLoader(); - DtClassLoader dtClassLoader = new DtClassLoader(new URL[]{}, threadClassLoader); - Thread.currentThread().setContextClassLoader(dtClassLoader); - - URLClassLoader parentClassloader; - if(!ClusterMode.local.name().equals(deployMode)){ - parentClassloader = (URLClassLoader) threadClassLoader.getParent(); - }else{ - parentClassloader = dtClassLoader; - } + DtClassLoader parentClassloader = new DtClassLoader(new URL[]{}, threadClassLoader); + Thread.currentThread().setContextClassLoader(parentClassloader); confProp = URLDecoder.decode(confProp, Charsets.UTF_8.toString()); Properties confProperties = PluginUtil.jsonStrToObject(confProp, Properties.class); @@ -218,7 +211,7 @@ public static void main(String[] args) throws Exception { if(env instanceof MyLocalStreamEnvironment) { List urlList = new ArrayList<>(); - urlList.addAll(Arrays.asList(dtClassLoader.getURLs())); + urlList.addAll(Arrays.asList(parentClassloader.getURLs())); ((MyLocalStreamEnvironment) env).setClasspaths(urlList); } @@ -254,7 +247,6 @@ private static void registerUDF(SqlTree sqlTree, List jarURList, URLClassLo if (classLoader == null) { classLoader = FlinkUtil.loadExtraJar(jarURList, parentClassloader); } - classLoader.loadClass(funcInfo.getClassName()); FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName(), tableEnv, classLoader); } From 8fb5ddb5cf9a418441da4645638d871019fddd01 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Sun, 30 Jun 2019 15:12:49 +0800 Subject: [PATCH 293/470] parse and --- .../side/cassandra/CassandraAllSideInfo.java | 7 ++---- .../cassandra/CassandraAsyncSideInfo.java | 7 ++---- .../dtstack/flink/sql/side/SideSqlExec.java | 8 ++----- .../dtstack/flink/sql/util/ParseUtils.java | 23 +++++++++++++++++++ .../sql/side/hbase/HbaseAllSideInfo.java | 7 ++---- .../sql/side/hbase/HbaseAsyncSideInfo.java | 7 ++---- .../sql/side/mongo/MongoAllSideInfo.java | 7 ++---- .../sql/side/mongo/MongoAsyncSideInfo.java | 7 ++---- .../sql/side/rdb/all/RdbAllSideInfo.java | 8 +++---- .../sql/side/rdb/async/RdbAsyncSideInfo.java | 7 ++---- .../sql/side/redis/RedisAllSideInfo.java | 8 +++---- .../sql/side/redis/RedisAsyncSideInfo.java | 7 ++---- 12 files changed, 47 insertions(+), 56 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java diff --git a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java index bba39fc3e..d6c816b97 100644 --- a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java +++ b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo; +import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; @@ -86,11 +87,7 @@ public void parseSelectFields(JoinInfo joinInfo) { SqlNode conditionNode = joinInfo.getCondition(); List sqlNodeList = Lists.newArrayList(); - if (conditionNode.getKind() == SqlKind.AND) { - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall) conditionNode).getOperands())); - } else { - sqlNodeList.add(conditionNode); - } + ParseUtils.parseAnd(conditionNode, sqlNodeList); for (SqlNode sqlNode : sqlNodeList) { dealOneEqualCon(sqlNode, sideTableName); diff --git a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java index b1d239440..8ff801064 100644 --- a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java +++ b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo; +import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; @@ -55,11 +56,7 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { SqlNode conditionNode = joinInfo.getCondition(); List sqlNodeList = Lists.newArrayList(); - if (conditionNode.getKind() == SqlKind.AND) { - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall) conditionNode).getOperands())); - } else { - sqlNodeList.add(conditionNode); - } + ParseUtils.parseAnd(conditionNode, sqlNodeList); for (SqlNode sqlNode : sqlNodeList) { dealOneEqualCon(sqlNode, sideTableName); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index f92552fac..14dcf125d 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -25,6 +25,7 @@ import com.dtstack.flink.sql.side.operator.SideAsyncOperator; import com.dtstack.flink.sql.side.operator.SideWithAllCacheOperator; import com.dtstack.flink.sql.util.ClassUtil; +import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlDataTypeSpec; import org.apache.calcite.sql.SqlIdentifier; @@ -461,12 +462,7 @@ private boolean checkJoinCondition(SqlNode conditionNode, String sideTableAlias, public List getConditionFields(SqlNode conditionNode, String specifyTableName){ List sqlNodeList = Lists.newArrayList(); - if(conditionNode.getKind() == SqlKind.AND){ - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); - }else{ - sqlNodeList.add(conditionNode); - } - + ParseUtils.parseAnd(conditionNode, sqlNodeList); List conditionFields = Lists.newArrayList(); for(SqlNode sqlNode : sqlNodeList){ if(sqlNode.getKind() != SqlKind.EQUALS){ diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java new file mode 100644 index 000000000..280594d17 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java @@ -0,0 +1,23 @@ +package com.dtstack.flink.sql.util; + +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; + +import java.util.List; + +/** + * @Auther: jiangjunjie + * @Date: 2019-06-30 14:57 + * @Description: + */ +public class ParseUtils { + public static void parseAnd(SqlNode conditionNode, List sqlNodeList){ + if(conditionNode.getKind() == SqlKind.AND && ((SqlBasicCall)conditionNode).getOperandList().size()==2){ + parseAnd(((SqlBasicCall)conditionNode).getOperands()[0], sqlNodeList); + sqlNodeList.add(((SqlBasicCall)conditionNode).getOperands()[1]); + }else{ + sqlNodeList.add(conditionNode); + } + } +} diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java index 7c02dba85..dde3f41ba 100644 --- a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java @@ -24,6 +24,7 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; @@ -53,11 +54,7 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { SqlNode conditionNode = joinInfo.getCondition(); List sqlNodeList = Lists.newArrayList(); - if(conditionNode.getKind() == SqlKind.AND){ - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); - }else{ - sqlNodeList.add(conditionNode); - } + ParseUtils.parseAnd(conditionNode, sqlNodeList); for(SqlNode sqlNode : sqlNodeList){ dealOneEqualCon(sqlNode, sideTableName); diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncSideInfo.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncSideInfo.java index 4ff25ad9c..84a6358d1 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncSideInfo.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncSideInfo.java @@ -5,6 +5,7 @@ import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo; +import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; @@ -55,11 +56,7 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { SqlNode conditionNode = joinInfo.getCondition(); List sqlNodeList = Lists.newArrayList(); - if(conditionNode.getKind() == SqlKind.AND){ - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); - }else{ - sqlNodeList.add(conditionNode); - } + ParseUtils.parseAnd(conditionNode, sqlNodeList); for(SqlNode sqlNode : sqlNodeList){ dealOneEqualCon(sqlNode, sideTableName); diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java index 5d94531d2..b3ef1f7e7 100644 --- a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.mongo.table.MongoSideTableInfo; +import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; @@ -85,11 +86,7 @@ public void parseSelectFields(JoinInfo joinInfo){ SqlNode conditionNode = joinInfo.getCondition(); List sqlNodeList = Lists.newArrayList(); - if(conditionNode.getKind() == SqlKind.AND){ - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); - }else{ - sqlNodeList.add(conditionNode); - } + ParseUtils.parseAnd(conditionNode, sqlNodeList); for(SqlNode sqlNode : sqlNodeList){ dealOneEqualCon(sqlNode, sideTableName); diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java index b09a12474..1590912a1 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.mongo.table.MongoSideTableInfo; +import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; @@ -55,11 +56,7 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { SqlNode conditionNode = joinInfo.getCondition(); List sqlNodeList = Lists.newArrayList(); - if(conditionNode.getKind() == SqlKind.AND){ - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); - }else{ - sqlNodeList.add(conditionNode); - } + ParseUtils.parseAnd(conditionNode, sqlNodeList); for(SqlNode sqlNode : sqlNodeList){ dealOneEqualCon(sqlNode, sideTableName); diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java index 501686f95..f50126e4a 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; @@ -88,11 +89,8 @@ public void parseSelectFields(JoinInfo joinInfo) { SqlNode conditionNode = joinInfo.getCondition(); List sqlNodeList = Lists.newArrayList(); - if (conditionNode.getKind() == SqlKind.AND) { - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall) conditionNode).getOperands())); - } else { - sqlNodeList.add(conditionNode); - } + + ParseUtils.parseAnd(conditionNode, sqlNodeList); for (SqlNode sqlNode : sqlNodeList) { dealOneEqualCon(sqlNode, sideTableName); diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java index f652bf342..c33916a71 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; @@ -57,11 +58,7 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { SqlNode conditionNode = joinInfo.getCondition(); List sqlNodeList = Lists.newArrayList(); - if (conditionNode.getKind() == SqlKind.AND) { - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall) conditionNode).getOperands())); - } else { - sqlNodeList.add(conditionNode); - } + ParseUtils.parseAnd(conditionNode, sqlNodeList); for (SqlNode sqlNode : sqlNodeList) { dealOneEqualCon(sqlNode, sideTableName); diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java index 29564a46d..b449d8c02 100644 --- a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java @@ -22,6 +22,7 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; @@ -44,11 +45,8 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { SqlNode conditionNode = joinInfo.getCondition(); List sqlNodeList = Lists.newArrayList(); - if(conditionNode.getKind() == SqlKind.AND){ - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); - }else{ - sqlNodeList.add(conditionNode); - } + + ParseUtils.parseAnd(conditionNode, sqlNodeList); for(SqlNode sqlNode : sqlNodeList){ dealOneEqualCon(sqlNode, sideTableName); diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java index a019b74cf..2c85aaf3a 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java @@ -22,6 +22,7 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; @@ -44,11 +45,7 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { SqlNode conditionNode = joinInfo.getCondition(); List sqlNodeList = Lists.newArrayList(); - if(conditionNode.getKind() == SqlKind.AND){ - sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall)conditionNode).getOperands())); - }else{ - sqlNodeList.add(conditionNode); - } + ParseUtils.parseAnd(conditionNode, sqlNodeList); for(SqlNode sqlNode : sqlNodeList){ dealOneEqualCon(sqlNode, sideTableName); From 818798a62aa4b0c8f87b84eef9ad28e79d7a71d2 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Sun, 30 Jun 2019 15:48:09 +0800 Subject: [PATCH 294/470] no as on join table --- .../com/dtstack/flink/sql/side/SideSQLParser.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 388bb5497..6eb242255 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -159,8 +159,16 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< JoinInfo tableInfo = new JoinInfo(); tableInfo.setLeftTableName(leftTbName); tableInfo.setRightTableName(rightTableName); - tableInfo.setLeftTableAlias(leftTbAlias); - tableInfo.setRightTableAlias(rightTableAlias); + if (leftTbAlias.equals("")){ + tableInfo.setLeftTableAlias(leftTbName); + } else { + tableInfo.setLeftTableAlias(leftTbAlias); + } + if (leftTbAlias.equals("")){ + tableInfo.setRightTableAlias(rightTableName); + } else { + tableInfo.setRightTableAlias(rightTableAlias); + } tableInfo.setLeftIsSideTable(leftIsSide); tableInfo.setRightIsSideTable(rightIsSide); tableInfo.setLeftNode(leftNode); From ca7fffbd0d7e1b08a16234d2cd981fde963bf1cc Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 3 Jul 2019 19:33:14 +0800 Subject: [PATCH 295/470] modify appendStream to retractStream --- core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 14dcf125d..127247eea 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -593,7 +593,7 @@ private void joinFun(Object pollObj, Map localTableCache, } RowTypeInfo typeInfo = new RowTypeInfo(targetTable.getSchema().getTypes(), targetTable.getSchema().getColumnNames()); - DataStream adaptStream = tableEnv.toAppendStream(targetTable, org.apache.flink.types.Row.class); + DataStream adaptStream = tableEnv.toRetractStream(targetTable, org.apache.flink.types.Row.class); //join side table before keyby ===> Reducing the size of each dimension table cache of async if(sideTableInfo.isPartitionedJoin()){ From f4a96f3938e0dc4fde2cd4d383ff286d4860ca23 Mon Sep 17 00:00:00 2001 From: maqi Date: Thu, 4 Jul 2019 15:50:59 +0800 Subject: [PATCH 296/470] retractstream to appendstream --- core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 127247eea..3f1dbd21e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -593,7 +593,8 @@ private void joinFun(Object pollObj, Map localTableCache, } RowTypeInfo typeInfo = new RowTypeInfo(targetTable.getSchema().getTypes(), targetTable.getSchema().getColumnNames()); - DataStream adaptStream = tableEnv.toRetractStream(targetTable, org.apache.flink.types.Row.class); + //DataStream adaptStream = tableEnv.toRetractStream(targetTable, org.apache.flink.types.Row.class); + DataStream adaptStream = tableEnv.toAppendStream(targetTable, org.apache.flink.types.Row.class); //join side table before keyby ===> Reducing the size of each dimension table cache of async if(sideTableInfo.isPartitionedJoin()){ From 6ef8be0830b700e2d39fbe3efd2102d0aae64e01 Mon Sep 17 00:00:00 2001 From: maqi Date: Thu, 4 Jul 2019 19:49:45 +0800 Subject: [PATCH 297/470] bugfix --- .../main/java/com/dtstack/flink/sql/util/PluginUtil.java | 4 ---- .../flink/sql/launcher/perjob/PerJobSubmitter.java | 8 ++++---- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java index 1f448b20a..4ecf97d3f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java @@ -160,10 +160,6 @@ public boolean accept(File dir, String name) { } } - if (StringUtils.isEmpty(coreJarFileName)){ - throw new Exception("Can not find core jar file in path:" + path); - } - return coreJarFileName; } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java index 0708b4957..c9c2a63ea 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java @@ -52,16 +52,16 @@ public class PerJobSubmitter { public static String submit(LauncherOptions launcherOptions, JobGraph jobGraph) throws Exception { fillJobGraphClassPath(jobGraph); - - String addjarPath = URLDecoder.decode(launcherOptions.getAddjar(), Charsets.UTF_8.toString()); - if (StringUtils.isNotBlank(addjarPath) ){ + if (StringUtils.isNoneEmpty(launcherOptions.getAddjar())) { + String addjarPath = URLDecoder.decode(launcherOptions.getAddjar(), Charsets.UTF_8.toString()); List paths = getJarPaths(addjarPath); paths.forEach( path ->{ jobGraph.addJar(new Path("file://" + path)); }); - } + + String confProp = launcherOptions.getConfProp(); confProp = URLDecoder.decode(confProp, Charsets.UTF_8.toString()); Properties confProperties = PluginUtil.jsonStrToObject(confProp, Properties.class); From 1955ab92eba1ea567d5c84d64957d33c701b2bff Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Thu, 4 Jul 2019 21:43:19 +0800 Subject: [PATCH 298/470] add default name for jar --- core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java index 4ecf97d3f..82f2ed541 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java @@ -158,8 +158,11 @@ public boolean accept(File dir, String name) { if (jarFiles != null && jarFiles.length > 0){ coreJarFileName = jarFiles[0].getName(); } + }else { + coreJarFileName = prefix.toLowerCase() + ".jar"; } + return coreJarFileName; } From 7d44605398148e1873c85337995872077a307b9f Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 9 Jul 2019 10:23:00 +0800 Subject: [PATCH 299/470] use retract stream --- .../src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 3f1dbd21e..481f7060d 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -593,9 +593,7 @@ private void joinFun(Object pollObj, Map localTableCache, } RowTypeInfo typeInfo = new RowTypeInfo(targetTable.getSchema().getTypes(), targetTable.getSchema().getColumnNames()); - //DataStream adaptStream = tableEnv.toRetractStream(targetTable, org.apache.flink.types.Row.class); - DataStream adaptStream = tableEnv.toAppendStream(targetTable, org.apache.flink.types.Row.class); - + DataStream adaptStream = tableEnv.toRetractStream(targetTable, org.apache.flink.types.Row.class); //join side table before keyby ===> Reducing the size of each dimension table cache of async if(sideTableInfo.isPartitionedJoin()){ List leftJoinColList = getConditionFields(joinInfo.getCondition(), joinInfo.getLeftTableAlias()); From 52328b3641c6a6b825a5f3fded4182c8316acc83 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 10 Jul 2019 13:42:31 +0800 Subject: [PATCH 300/470] retract stream convert type --- core/src/main/java/com/dtstack/flink/sql/Main.java | 6 +++++- .../java/com/dtstack/flink/sql/side/SideSqlExec.java | 10 ++++++---- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index e3c2aa9f8..020d3d0d7 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -46,6 +46,7 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; @@ -271,7 +272,10 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en Table adaptTable = adaptSql == null ? table : tableEnv.sqlQuery(adaptSql); RowTypeInfo typeInfo = new RowTypeInfo(adaptTable.getSchema().getTypes(), adaptTable.getSchema().getColumnNames()); - DataStream adaptStream = tableEnv.toAppendStream(adaptTable, typeInfo); + DataStream adaptStream = tableEnv.toRetractStream(adaptTable, typeInfo) + .map((Tuple2 f0) -> { return f0.f1; }) + .returns(typeInfo); + String fields = String.join(",", typeInfo.getFieldNames()); if(waterMarkerAssigner.checkNeedAssignWaterMarker(sourceTableInfo)){ diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 3f1dbd21e..aa966fd34 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -16,7 +16,7 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.side; @@ -41,6 +41,7 @@ import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.HashBasedTable; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; @@ -48,6 +49,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.types.Row; import java.util.*; @@ -593,9 +595,9 @@ private void joinFun(Object pollObj, Map localTableCache, } RowTypeInfo typeInfo = new RowTypeInfo(targetTable.getSchema().getTypes(), targetTable.getSchema().getColumnNames()); - //DataStream adaptStream = tableEnv.toRetractStream(targetTable, org.apache.flink.types.Row.class); - DataStream adaptStream = tableEnv.toAppendStream(targetTable, org.apache.flink.types.Row.class); - + DataStream adaptStream = tableEnv.toRetractStream(targetTable, org.apache.flink.types.Row.class) + .map((Tuple2 f0) -> { return f0.f1; }) + .returns(typeInfo); //join side table before keyby ===> Reducing the size of each dimension table cache of async if(sideTableInfo.isPartitionedJoin()){ List leftJoinColList = getConditionFields(joinInfo.getCondition(), joinInfo.getLeftTableAlias()); From b0dc18a0dadc5439cfc69a684247e505cb4bed73 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 10 Jul 2019 15:52:07 +0800 Subject: [PATCH 301/470] empty check --- .../dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java index c9c2a63ea..91550404c 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java @@ -52,10 +52,10 @@ public class PerJobSubmitter { public static String submit(LauncherOptions launcherOptions, JobGraph jobGraph) throws Exception { fillJobGraphClassPath(jobGraph); - if (StringUtils.isNoneEmpty(launcherOptions.getAddjar())) { + if (!StringUtils.isBlank(launcherOptions.getAddjar())) { String addjarPath = URLDecoder.decode(launcherOptions.getAddjar(), Charsets.UTF_8.toString()); List paths = getJarPaths(addjarPath); - paths.forEach( path ->{ + paths.forEach( path -> { jobGraph.addJar(new Path("file://" + path)); }); } From bab258ba1e8ac3d967ff97600b805fbda8102655 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 10 Jul 2019 16:55:53 +0800 Subject: [PATCH 302/470] modify sidesqlexc datastream type --- .../src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index aa966fd34..69d352832 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -595,9 +595,11 @@ private void joinFun(Object pollObj, Map localTableCache, } RowTypeInfo typeInfo = new RowTypeInfo(targetTable.getSchema().getTypes(), targetTable.getSchema().getColumnNames()); + DataStream adaptStream = tableEnv.toRetractStream(targetTable, org.apache.flink.types.Row.class) .map((Tuple2 f0) -> { return f0.f1; }) - .returns(typeInfo); + .returns(Row.class); + //join side table before keyby ===> Reducing the size of each dimension table cache of async if(sideTableInfo.isPartitionedJoin()){ List leftJoinColList = getConditionFields(joinInfo.getCondition(), joinInfo.getLeftTableAlias()); From 60beebfbefa8d9b903c129748ac2f5d846d8ef15 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 10 Jul 2019 20:04:18 +0800 Subject: [PATCH 303/470] jar exist check --- .../com/dtstack/flink/sql/util/PluginUtil.java | 15 ++++++++++----- .../dtstack/flink/sql/launcher/LauncherMain.java | 2 +- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java index 82f2ed541..7b55683d9 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java @@ -111,7 +111,7 @@ public static URL getRemoteJarFilePath(String pluginType, String tableType, Stri String dirName = pluginType + tableType.toLowerCase(); String prefix = String.format("%s-%s", pluginType, tableType.toLowerCase()); String jarPath = remoteSqlRootDir + SP + dirName; - String jarName = getCoreJarFileName(jarPath, prefix); + String jarName = getCoreJarFileName(jarPath, prefix, false); return new URL("file:" + remoteSqlRootDir + SP + dirName + SP + jarName); } @@ -119,7 +119,7 @@ public static URL getRemoteSideJarFilePath(String pluginType, String sideOperato String dirName = pluginType + sideOperator + tableType.toLowerCase(); String prefix = String.format("%s-%s-%s", pluginType, sideOperator, tableType.toLowerCase()); String jarPath = remoteSqlRootDir + SP + dirName; - String jarName = getCoreJarFileName(jarPath, prefix); + String jarName = getCoreJarFileName(jarPath, prefix, false); return new URL("file:" + remoteSqlRootDir + SP + dirName + SP + jarName); } @@ -144,7 +144,11 @@ public static void addPluginJar(String pluginDir, DtClassLoader classLoader) thr } } - public static String getCoreJarFileName (String path, String prefix) throws Exception { + public static String getCoreJarFileName (String path, String prefix, boolean existCheck) throws Exception { + if (!existCheck) { + return prefix.toLowerCase() + ".jar"; + } + String coreJarFileName = null; File pluginDir = new File(path); if (pluginDir.exists() && pluginDir.isDirectory()){ @@ -158,10 +162,11 @@ public boolean accept(File dir, String name) { if (jarFiles != null && jarFiles.length > 0){ coreJarFileName = jarFiles[0].getName(); } - }else { - coreJarFileName = prefix.toLowerCase() + ".jar"; } + if (StringUtils.isEmpty(coreJarFileName)){ + throw new Exception("Can not find core jar file in path:" + path); + } return coreJarFileName; } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index f2d884778..cd5a79a4b 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -59,7 +59,7 @@ public class LauncherMain { private static String getLocalCoreJarPath(String localSqlRootJar) throws Exception { - String jarPath = PluginUtil.getCoreJarFileName(localSqlRootJar, CORE_JAR); + String jarPath = PluginUtil.getCoreJarFileName(localSqlRootJar, CORE_JAR, true); String corePath = localSqlRootJar + SP + jarPath; return corePath; } From 6b245bac8a36db9e8472324c3058260be211e384 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Thu, 11 Jul 2019 11:56:25 +0800 Subject: [PATCH 304/470] fill up remote class path --- .../main/java/com/dtstack/flink/sql/Main.java | 6 +++--- .../com/dtstack/flink/sql/util/PluginUtil.java | 18 +++++++----------- .../flink/sql/launcher/LauncherMain.java | 2 +- 3 files changed, 11 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 020d3d0d7..a34ef54e8 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -288,18 +288,18 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en Table regTable = tableEnv.fromDataStream(adaptStream, fields); tableEnv.registerTable(tableInfo.getName(), regTable); registerTableCache.put(tableInfo.getName(), regTable); - classPathSet.add(PluginUtil.getRemoteJarFilePath(tableInfo.getType(), SourceTableInfo.SOURCE_SUFFIX, remoteSqlPluginPath)); + classPathSet.add(PluginUtil.getRemoteJarFilePath(tableInfo.getType(), SourceTableInfo.SOURCE_SUFFIX, remoteSqlPluginPath, localSqlPluginPath)); } else if (tableInfo instanceof TargetTableInfo) { TableSink tableSink = StreamSinkFactory.getTableSink((TargetTableInfo) tableInfo, localSqlPluginPath); TypeInformation[] flinkTypes = FlinkUtil.transformTypes(tableInfo.getFieldClasses()); tableEnv.registerTableSink(tableInfo.getName(), tableInfo.getFields(), flinkTypes, tableSink); - classPathSet.add( PluginUtil.getRemoteJarFilePath(tableInfo.getType(), TargetTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); + classPathSet.add( PluginUtil.getRemoteJarFilePath(tableInfo.getType(), TargetTableInfo.TARGET_SUFFIX, remoteSqlPluginPath, localSqlPluginPath)); } else if(tableInfo instanceof SideTableInfo){ String sideOperator = ECacheType.ALL.name().equals(((SideTableInfo) tableInfo).getCacheType()) ? "all" : "async"; sideTableMap.put(tableInfo.getName(), (SideTableInfo) tableInfo); - classPathSet.add(PluginUtil.getRemoteSideJarFilePath(tableInfo.getType(), sideOperator, SideTableInfo.TARGET_SUFFIX, remoteSqlPluginPath)); + classPathSet.add(PluginUtil.getRemoteSideJarFilePath(tableInfo.getType(), sideOperator, SideTableInfo.TARGET_SUFFIX, remoteSqlPluginPath, localSqlPluginPath)); }else { throw new RuntimeException("not support table type:" + tableInfo.getType()); } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java index 7b55683d9..682df169e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java @@ -107,19 +107,19 @@ public static Properties stringToProperties(String str) throws IOException{ return properties; } - public static URL getRemoteJarFilePath(String pluginType, String tableType, String remoteSqlRootDir) throws Exception { + public static URL getRemoteJarFilePath(String pluginType, String tableType, String remoteSqlRootDir, String localSqlPluginPath) throws Exception { String dirName = pluginType + tableType.toLowerCase(); String prefix = String.format("%s-%s", pluginType, tableType.toLowerCase()); - String jarPath = remoteSqlRootDir + SP + dirName; - String jarName = getCoreJarFileName(jarPath, prefix, false); + String jarPath = localSqlPluginPath + SP + dirName; + String jarName = getCoreJarFileName(jarPath, prefix); return new URL("file:" + remoteSqlRootDir + SP + dirName + SP + jarName); } - public static URL getRemoteSideJarFilePath(String pluginType, String sideOperator, String tableType, String remoteSqlRootDir) throws Exception { + public static URL getRemoteSideJarFilePath(String pluginType, String sideOperator, String tableType, String remoteSqlRootDir, String localSqlPluginPath) throws Exception { String dirName = pluginType + sideOperator + tableType.toLowerCase(); String prefix = String.format("%s-%s-%s", pluginType, sideOperator, tableType.toLowerCase()); - String jarPath = remoteSqlRootDir + SP + dirName; - String jarName = getCoreJarFileName(jarPath, prefix, false); + String jarPath = localSqlPluginPath + SP + dirName; + String jarName = getCoreJarFileName(jarPath, prefix); return new URL("file:" + remoteSqlRootDir + SP + dirName + SP + jarName); } @@ -144,11 +144,7 @@ public static void addPluginJar(String pluginDir, DtClassLoader classLoader) thr } } - public static String getCoreJarFileName (String path, String prefix, boolean existCheck) throws Exception { - if (!existCheck) { - return prefix.toLowerCase() + ".jar"; - } - + public static String getCoreJarFileName (String path, String prefix) throws Exception { String coreJarFileName = null; File pluginDir = new File(path); if (pluginDir.exists() && pluginDir.isDirectory()){ diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index cd5a79a4b..f2d884778 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -59,7 +59,7 @@ public class LauncherMain { private static String getLocalCoreJarPath(String localSqlRootJar) throws Exception { - String jarPath = PluginUtil.getCoreJarFileName(localSqlRootJar, CORE_JAR, true); + String jarPath = PluginUtil.getCoreJarFileName(localSqlRootJar, CORE_JAR); String corePath = localSqlRootJar + SP + jarPath; return corePath; } From c20d1628a3488cb09a85ad9be96fd2c783da360b Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 12 Jul 2019 11:41:13 +0800 Subject: [PATCH 305/470] oracle field cast --- .../flink/sql/side/mysql/MysqlAllReqRow.java | 2 +- .../sql/side/mysql/MysqlAsyncReqRow.java | 2 +- .../sql/side/oracle/OracleAllReqRow.java | 2 +- .../sql/side/oracle/OracleAllSideInfo.java | 41 ++++++++++++ .../sql/side/oracle/OracleAsyncReqRow.java | 8 +-- .../sql/side/oracle/OracleAsyncSideInfo.java | 64 +++++++++++++++++++ .../flink/sql/side/rdb/all/RdbAllReqRow.java | 19 +++--- .../sql/side/rdb/async/RdbAsyncReqRow.java | 4 +- .../side/sqlserver/SqlserverAllReqRow.java | 2 +- .../side/sqlserver/SqlserverAsyncReqRow.java | 2 +- 10 files changed, 124 insertions(+), 22 deletions(-) diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java index aca7593f7..a60f5774e 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java @@ -50,7 +50,7 @@ public class MysqlAllReqRow extends RdbAllReqRow { private static final String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; public MysqlAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + super(new MysqlAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @Override diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java index 456451214..39fb6ac56 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -50,7 +50,7 @@ public class MysqlAsyncReqRow extends RdbAsyncReqRow { private final static String MYSQL_DRIVER = "com.mysql.jdbc.Driver"; public MysqlAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + super(new MysqlAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } diff --git a/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java index b6779529e..278e5e665 100644 --- a/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java +++ b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java @@ -43,7 +43,7 @@ public class OracleAllReqRow extends RdbAllReqRow { private static final String ORACLE_DRIVER = "oracle.jdbc.driver.OracleDriver"; public OracleAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + super(new OracleAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @Override diff --git a/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java index 4063bf9bb..15078a92e 100644 --- a/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java +++ b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java @@ -21,6 +21,8 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; import java.util.List; @@ -30,4 +32,43 @@ public class OracleAllSideInfo extends RdbAllSideInfo { public OracleAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideTableInfo; + + sqlCondition = "select ${selectField} from ${tableName} "; + + + sqlCondition = sqlCondition.replace("${tableName}", dealLowerFiled(rdbSideTableInfo.getTableName())).replace("${selectField}", dealLowerSelectFiled(sideSelectFields)); + System.out.println("---------side_exe_sql-----\n" + sqlCondition); + } + + + private String dealLowerFiled(String field) { + if (StringUtils.isAllUpperCase(field)) { + return field; + } + return "\"" + field + "\""; + } + + private String dealLowerSelectFiled(String fieldsStr) { + if (StringUtils.isAllUpperCase(fieldsStr)) { + return fieldsStr; + } + + StringBuilder sb = new StringBuilder(); + String[] fields = fieldsStr.split(","); + + for(String f : fields) { + if (StringUtils.isAllUpperCase(f)) { + sb.append(f).append(","); + } else { + sb.append("\"").append(f).append("\"").append(","); + } + } + + sb.deleteCharAt(sb.lastIndexOf(",")); + return sb.toString(); + } } diff --git a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java index c68850d21..c7f42a8dd 100644 --- a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java +++ b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java @@ -43,15 +43,15 @@ public class OracleAsyncReqRow extends RdbAsyncReqRow { private static final String ORACLE_DRIVER = "oracle.jdbc.driver.OracleDriver"; public OracleAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + super(new OracleAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @Override public void open(Configuration parameters) throws Exception { super.open(parameters); - JsonObject sqlserverClientConfig = new JsonObject(); + JsonObject oracleClientConfig = new JsonObject(); RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); - sqlserverClientConfig.put("url", rdbSideTableInfo.getUrl()) + oracleClientConfig.put("url", rdbSideTableInfo.getUrl()) .put("driver_class", ORACLE_DRIVER) .put("max_pool_size", DEFAULT_MAX_DB_CONN_POOL_SIZE) .put("user", rdbSideTableInfo.getUserName()) @@ -61,6 +61,6 @@ public void open(Configuration parameters) throws Exception { vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); vo.setWorkerPoolSize(DEFAULT_VERTX_WORKER_POOL_SIZE); Vertx vertx = Vertx.vertx(vo); - setRdbSQLClient(JDBCClient.createNonShared(vertx, sqlserverClientConfig)); + setRdbSQLClient(JDBCClient.createNonShared(vertx, oracleClientConfig)); } } diff --git a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java index 7ff63f3d9..e2222f57f 100644 --- a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java +++ b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java @@ -22,8 +22,14 @@ import com.dtstack.flink.sql.side.JoinInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import com.dtstack.flink.sql.util.ParseUtils; +import org.apache.calcite.sql.SqlNode; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import java.util.Arrays; import java.util.List; @@ -32,4 +38,62 @@ public class OracleAsyncSideInfo extends RdbAsyncSideInfo { public OracleAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideTableInfo; + + String sideTableName = joinInfo.getSideTableName(); + + SqlNode conditionNode = joinInfo.getCondition(); + + List sqlNodeList = Lists.newArrayList(); + ParseUtils.parseAnd(conditionNode, sqlNodeList); + + for (SqlNode sqlNode : sqlNodeList) { + dealOneEqualCon(sqlNode, sideTableName); + } + + sqlCondition = "select ${selectField} from ${tableName} where "; + for (int i = 0; i < equalFieldList.size(); i++) { + String equalField = equalFieldList.get(i); + + sqlCondition += dealLowerWhereFiled(equalField) + "=? "; + if (i != equalFieldList.size() - 1) { + sqlCondition += " and "; + } + } + + + sqlCondition = sqlCondition.replace("${tableName}", dealLowerWhereFiled(rdbSideTableInfo.getTableName())).replace("${selectField}", dealLowerSelectFiled(sideSelectFields)); + System.out.println("---------side_exe_sql-----\n" + sqlCondition); + } + + private String dealLowerWhereFiled(String field) { + if (StringUtils.isAllUpperCase(field)) { + return field; + } + return "\"" + field + "\""; + } + + private String dealLowerSelectFiled(String fieldsStr) { + if (StringUtils.isAllUpperCase(fieldsStr)) { + return fieldsStr; + } + + StringBuilder sb = new StringBuilder(); + String[] fields = fieldsStr.split(","); + + for(String f : fields) { + if (StringUtils.isAllUpperCase(f)) { + sb.append(f).append(","); + } else { + sb.append("\"").append(f).append("\"").append(","); + } + } + + sb.deleteCharAt(sb.lastIndexOf(",")); + return sb.toString(); + } + } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java index abae0cbb2..26ed7648f 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java @@ -19,16 +19,11 @@ package com.dtstack.flink.sql.side.rdb.all; import com.dtstack.flink.sql.side.AllReqRow; -import com.dtstack.flink.sql.side.FieldInfo; -import com.dtstack.flink.sql.side.JoinInfo; -import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; -import com.dtstack.flink.sql.side.rdb.util.MathUtil; import com.dtstack.flink.sql.side.rdb.util.SwitchUtil; import org.apache.calcite.sql.JoinType; import org.apache.commons.collections.CollectionUtils; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; @@ -37,8 +32,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.math.BigDecimal; -import java.sql.*; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Timestamp; import java.util.Calendar; import java.util.List; import java.util.Map; @@ -62,9 +60,8 @@ public abstract class RdbAllReqRow extends AllReqRow { private AtomicReference>>> cacheRef = new AtomicReference<>(); - - public RdbAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(new RdbAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + public RdbAllReqRow(SideInfo sideInfo) { + super(sideInfo); } @Override diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index 6989dd64f..e683a40a8 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -59,8 +59,8 @@ public class RdbAsyncReqRow extends AsyncReqRow { private transient SQLClient rdbSQLClient; - public RdbAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(new RdbAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + public RdbAsyncReqRow(SideInfo sideInfo) { + super(sideInfo); } @Override diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java index d123c8f9c..837236b61 100644 --- a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java @@ -43,7 +43,7 @@ public class SqlserverAllReqRow extends RdbAllReqRow { private static final String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; public SqlserverAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + super(new SqlserverAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @Override diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java index 003d61b27..8abae552e 100644 --- a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java @@ -41,7 +41,7 @@ public class SqlserverAsyncReqRow extends RdbAsyncReqRow { private final static String SQLSERVER_DRIVER = "net.sourceforge.jtds.jdbc.Driver"; public SqlserverAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { - super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + super(new SqlserverAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } @Override From 0e57dddd209e34b1c19e6dff1e39eaa0bc30e287 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 12 Jul 2019 13:07:05 +0800 Subject: [PATCH 306/470] oracle upper field cast --- .../sql/side/oracle/OracleAllSideInfo.java | 17 +++------------- .../sql/side/oracle/OracleAsyncSideInfo.java | 20 ++++--------------- 2 files changed, 7 insertions(+), 30 deletions(-) diff --git a/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java index 15078a92e..801e0a69b 100644 --- a/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java +++ b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java @@ -40,32 +40,21 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { sqlCondition = "select ${selectField} from ${tableName} "; - sqlCondition = sqlCondition.replace("${tableName}", dealLowerFiled(rdbSideTableInfo.getTableName())).replace("${selectField}", dealLowerSelectFiled(sideSelectFields)); + sqlCondition = sqlCondition.replace("${tableName}", dealFiled(rdbSideTableInfo.getTableName())).replace("${selectField}", dealLowerSelectFiled(sideSelectFields)); System.out.println("---------side_exe_sql-----\n" + sqlCondition); } - private String dealLowerFiled(String field) { - if (StringUtils.isAllUpperCase(field)) { - return field; - } + private String dealFiled(String field) { return "\"" + field + "\""; } private String dealLowerSelectFiled(String fieldsStr) { - if (StringUtils.isAllUpperCase(fieldsStr)) { - return fieldsStr; - } - StringBuilder sb = new StringBuilder(); String[] fields = fieldsStr.split(","); for(String f : fields) { - if (StringUtils.isAllUpperCase(f)) { - sb.append(f).append(","); - } else { - sb.append("\"").append(f).append("\"").append(","); - } + sb.append("\"").append(f).append("\"").append(","); } sb.deleteCharAt(sb.lastIndexOf(",")); diff --git a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java index e2222f57f..22f6dadc7 100644 --- a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java +++ b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java @@ -58,38 +58,26 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { for (int i = 0; i < equalFieldList.size(); i++) { String equalField = equalFieldList.get(i); - sqlCondition += dealLowerWhereFiled(equalField) + "=? "; + sqlCondition += dealLowerFiled(equalField) + "=? "; if (i != equalFieldList.size() - 1) { sqlCondition += " and "; } } - - sqlCondition = sqlCondition.replace("${tableName}", dealLowerWhereFiled(rdbSideTableInfo.getTableName())).replace("${selectField}", dealLowerSelectFiled(sideSelectFields)); + sqlCondition = sqlCondition.replace("${tableName}", dealLowerFiled(rdbSideTableInfo.getTableName())).replace("${selectField}", dealLowerSelectFiled(sideSelectFields)); System.out.println("---------side_exe_sql-----\n" + sqlCondition); } - private String dealLowerWhereFiled(String field) { - if (StringUtils.isAllUpperCase(field)) { - return field; - } + private String dealLowerFiled(String field) { return "\"" + field + "\""; } private String dealLowerSelectFiled(String fieldsStr) { - if (StringUtils.isAllUpperCase(fieldsStr)) { - return fieldsStr; - } - StringBuilder sb = new StringBuilder(); String[] fields = fieldsStr.split(","); for(String f : fields) { - if (StringUtils.isAllUpperCase(f)) { - sb.append(f).append(","); - } else { - sb.append("\"").append(f).append("\"").append(","); - } + sb.append("\"").append(f).append("\"").append(","); } sb.deleteCharAt(sb.lastIndexOf(",")); From aa00b16e455b2008035fa7599c481b2086842c34 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 15 Jul 2019 15:50:48 +0800 Subject: [PATCH 307/470] =?UTF-8?q?=E4=BF=AE=E6=94=B9flink1.8.0=20-->=201.?= =?UTF-8?q?8.1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5659dbd11..7560a2bbe 100644 --- a/pom.xml +++ b/pom.xml @@ -31,7 +31,7 @@ UTF-8 - 1.8.0 + 1.8.1 From 89a489686238f6f3333ed726885ac2b5bfde9594 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 15 Jul 2019 17:20:08 +0800 Subject: [PATCH 308/470] print detail log info --- core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java index 0ba59ace6..6b99e2c9a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java @@ -180,7 +180,7 @@ public static void registerScalaUDF(String classPath, String funcName, TableEnvi logger.info("register scala function:{} success.", funcName); }catch (Exception e){ logger.error("", e); - throw new RuntimeException("register UDF exception:" + e.getMessage()); + throw new RuntimeException("register UDF exception:", e); } } @@ -208,7 +208,7 @@ public static void registerTableUDF(String classPath, String funcName, TableEnvi logger.info("register table function:{} success.", funcName); }catch (Exception e){ logger.error("", e); - throw new RuntimeException("register Table UDF exception:" + e.getMessage()); + throw new RuntimeException("register Table UDF exception:", e); } } From 35df05f1b1c3655401e26e6ce414d57247547344 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 16 Jul 2019 14:07:57 +0800 Subject: [PATCH 309/470] =?UTF-8?q?fix=20=E6=AD=A3=E5=88=99=E5=8C=B9?= =?UTF-8?q?=E9=85=8D=E5=87=BA=E7=8E=B0=E5=BE=AA=E7=8E=AF=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flink/sql/table/AbsTableParser.java | 2 +- .../java/com/dtstack/flink/sql/util/DtStringUtil.java | 10 ---------- 2 files changed, 1 insertion(+), 11 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index ddbf269e9..cc92b092a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -79,7 +79,7 @@ public boolean dealKeyPattern(String fieldRow, TableInfo tableInfo){ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ - String[] fieldRows = DtStringUtil.splitIgnoreQuotaBrackets(fieldsInfo, ","); + List fieldRows = DtStringUtil.splitIgnoreQuota(fieldsInfo, ','); for(String fieldRow : fieldRows){ fieldRow = fieldRow.trim(); boolean isMatcherKey = dealKeyPattern(fieldRow, tableInfo); diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index 959d4b13c..b65ab1e57 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -84,16 +84,6 @@ public static List splitIgnoreQuota(String str, char delimiter){ return tokensList; } - /*** - * Split the specified string delimiter --- ignored in brackets and quotation marks delimiter - * @param str - * @param delimter - * @return - */ - public static String[] splitIgnoreQuotaBrackets(String str, String delimter){ - String splitPatternStr = delimter + "(?![^()]*+\\))(?![^{}]*+})(?![^\\[\\]]*+\\])(?=(?:[^\"]|\"[^\"]*\")*$)"; - return str.split(splitPatternStr); - } public static String replaceIgnoreQuota(String str, String oriStr, String replaceStr){ String splitPatternStr = oriStr + "(?=(?:[^\"]*\"[^\"]*\")*[^\"]*$)(?=(?:[^']*'[^']*')*[^']*$)"; From e1dcd7ee13d7dd76968e869447fc9b9ae5993820 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 16 Jul 2019 15:34:33 +0800 Subject: [PATCH 310/470] integer unsigned parse --- .../com/dtstack/flink/sql/util/ClassUtil.java | 18 ++++++++++++++--- .../flink/sql/side/rdb/util/SwitchUtil.java | 20 ++++++++++++++++++- 2 files changed, 34 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java index 4012e5c3b..83f5170fb 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java @@ -22,6 +22,7 @@ import java.math.BigDecimal; import java.sql.Date; +import java.sql.Time; import java.sql.Timestamp; /** @@ -35,22 +36,29 @@ public class ClassUtil { public static Class stringConvertClass(String str) { switch (str.toLowerCase()) { case "boolean": + case "bit": return Boolean.class; case "integer": case "int": + case "mediumint": + case "tinyint": return Integer.class; + case "blob": + return Byte.class; + case "bigint": + case "intunsigned": + case "integerunsigned": return Long.class; - case "tinyint": - return Byte.class; - case "smallint": return Short.class; case "varchar": + case "char": + case "text": return String.class; case "real": @@ -63,9 +71,13 @@ public static Class stringConvertClass(String str) { case "date": return Date.class; + case "datetime": case "timestamp": return Timestamp.class; + case "time": + return Time.class; + case "decimal": return BigDecimal.class; diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java index 1d06a4193..ad12be33a 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java @@ -30,29 +30,47 @@ public class SwitchUtil { public static Object getTarget(Object obj, String targetType) { targetType = targetType.toLowerCase(); switch (targetType) { + + case "tinyint": case "int": case "integer": + case "mediumint": return MathUtil.getIntegerVal(obj); + case "bigint": + case "intunsigned": + case "integerunsigned": return MathUtil.getLongVal(obj); + case "boolean": return MathUtil.getBoolean(obj); - case "tinyint": + + case "blob": return MathUtil.getByte(obj); + case "smallint": return MathUtil.getShort(obj); + case "varchar": + case "char": + case "text": return MathUtil.getString(obj); + case "real": case "float": return MathUtil.getFloatVal(obj); + case "double": return MathUtil.getDoubleVal(obj); + case "decimal": return MathUtil.getBigDecimal(obj); + case "date": return MathUtil.getDate(obj); + case "timestamp": + case "datetime": return MathUtil.getTimestamp(obj); } return obj; From b7c4458b719a8bd583bc307e82056b2e1e5eb07f Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 17 Jul 2019 15:05:44 +0800 Subject: [PATCH 311/470] toAppendStream change to toRetactStream --- core/src/main/java/com/dtstack/flink/sql/Main.java | 6 +++++- .../main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 6 +++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 37ffa4df4..afdafb09f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -46,6 +46,7 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; @@ -279,7 +280,10 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en Table adaptTable = adaptSql == null ? table : tableEnv.sqlQuery(adaptSql); RowTypeInfo typeInfo = new RowTypeInfo(adaptTable.getSchema().getTypes(), adaptTable.getSchema().getColumnNames()); - DataStream adaptStream = tableEnv.toAppendStream(adaptTable, typeInfo); + DataStream adaptStream = tableEnv.toRetractStream(adaptTable, typeInfo) + .map((Tuple2 f0) -> { return f0.f1; }) + .returns(typeInfo); + String fields = String.join(",", typeInfo.getFieldNames()); if(waterMarkerAssigner.checkNeedAssignWaterMarker(sourceTableInfo)){ diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 3623ecfd4..891046da9 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -40,6 +40,7 @@ import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.HashBasedTable; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; @@ -47,6 +48,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.types.Row; import java.util.*; @@ -597,7 +599,9 @@ private void joinFun(Object pollObj, Map localTableCache, } RowTypeInfo typeInfo = new RowTypeInfo(targetTable.getSchema().getTypes(), targetTable.getSchema().getColumnNames()); - DataStream adaptStream = tableEnv.toAppendStream(targetTable, org.apache.flink.types.Row.class); + DataStream adaptStream = tableEnv.toRetractStream(targetTable, org.apache.flink.types.Row.class) + .map((Tuple2 f0) -> { return f0.f1; }) + .returns(Row.class); //join side table before keyby ===> Reducing the size of each dimension table cache of async if(sideTableInfo.isPartitionedJoin()){ From a109d055b6c53e4710c297adf398b15a81d0dff4 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 18 Jul 2019 13:07:03 +0800 Subject: [PATCH 312/470] fix conflict --- .../main/java/com/dtstack/flink/sql/Main.java | 13 +- .../flink/sql/source/kafka/KafkaSource.java | 1 - kafka09/pom.xml | 2 +- kafka10/pom.xml | 2 +- .../sink/kafka/CustomerCsvSerialization.java | 2 +- .../kafka/CustomerKafka11JsonTableSink.java | 4 +- .../flink/sql/sink/kafka/KafkaSink.java | 58 +++++-- .../sql/sink/kafka/table/KafkaSinkParser.java | 15 +- .../sink/kafka/table/KafkaSinkTableInfo.java | 41 +---- .../CustomerCommonDeserialization.java | 156 ------------------ kafka11/pom.xml | 2 +- .../flink/sql/launcher/LauncherOptions.java | 10 -- 12 files changed, 72 insertions(+), 234 deletions(-) delete mode 100644 kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 6ce39c65a..210c9ad82 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -141,15 +141,8 @@ public static void main(String[] args) throws Exception { } ClassLoader threadClassLoader = Thread.currentThread().getContextClassLoader(); - DtClassLoader dtClassLoader = new DtClassLoader(new URL[]{}, threadClassLoader); - Thread.currentThread().setContextClassLoader(dtClassLoader); - - URLClassLoader parentClassloader; - if(!ClusterMode.local.name().equals(deployMode)){ - parentClassloader = (URLClassLoader) threadClassLoader.getParent(); - }else{ - parentClassloader = dtClassLoader; - } + DtClassLoader parentClassloader = new DtClassLoader(new URL[]{}, threadClassLoader); + Thread.currentThread().setContextClassLoader(parentClassloader); confProp = URLDecoder.decode(confProp, Charsets.UTF_8.toString()); Properties confProperties = PluginUtil.jsonStrToObject(confProp, Properties.class); @@ -220,7 +213,7 @@ public static void main(String[] args) throws Exception { if(env instanceof MyLocalStreamEnvironment) { List urlList = new ArrayList<>(); - urlList.addAll(Arrays.asList(dtClassLoader.getURLs())); + urlList.addAll(Arrays.asList(parentClassloader.getURLs())); ((MyLocalStreamEnvironment) env).setClasspaths(urlList); } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 8b2c49d94..9b410b204 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -27,7 +27,6 @@ import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang3.BooleanUtils; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; diff --git a/kafka09/pom.xml b/kafka09/pom.xml index 86613de70..81aef2a96 100644 --- a/kafka09/pom.xml +++ b/kafka09/pom.xml @@ -15,7 +15,7 @@ kafka09-source - kafka09-sink + diff --git a/kafka10/pom.xml b/kafka10/pom.xml index d8ad7370f..ba7d90de9 100644 --- a/kafka10/pom.xml +++ b/kafka10/pom.xml @@ -14,7 +14,7 @@ kafka10-source - kafka10-sink + diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java index 032d34bb2..3f73f667d 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java @@ -135,4 +135,4 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public TypeSerializerSnapshot snapshotConfiguration() { return null; } -} +} \ No newline at end of file diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java index 6b6d1ea5f..4d091d0cc 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.connectors.kafka.KafkaJsonTableSink; +import org.apache.flink.streaming.connectors.kafka.Kafka011TableSink; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; @@ -39,7 +39,7 @@ * * @author maqi */ -public class CustomerKafka11JsonTableSink extends KafkaJsonTableSink { +public class CustomerKafka11JsonTableSink extends Kafka011TableSink { protected SerializationSchema schema; diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 9496a351a..a6184161f 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -22,13 +22,22 @@ import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; import com.dtstack.flink.sql.table.TargetTableInfo; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.serialization.TypeInformationSerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; + +import org.apache.flink.streaming.connectors.kafka.Kafka011TableSink; import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; + +import java.util.Optional; import java.util.Properties; /** @@ -47,38 +56,62 @@ public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener< protected TypeInformation[] fieldTypes; + /** The schema of the table. */ + private TableSchema schema; + + /** The Kafka topic to write to. */ protected String topic; + /** Properties for the Kafka producer. */ protected Properties properties; /** Serialization schema for encoding records to Kafka. */ protected SerializationSchema serializationSchema; + /** Partitioner to select Kafka partition for each item. */ + protected Optional> partitioner; + @Override public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { - KafkaSinkTableInfo kafka11SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; - this.topic = kafka11SinkTableInfo.getTopic(); - this.fieldNames = kafka11SinkTableInfo.getFields(); - TypeInformation[] types = new TypeInformation[kafka11SinkTableInfo.getFields().length]; - for (int i = 0; i < kafka11SinkTableInfo.getFieldClasses().length; i++) { - types[i] = TypeInformation.of(kafka11SinkTableInfo.getFieldClasses()[i]); + KafkaSinkTableInfo kafka011SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; + this.topic = kafka011SinkTableInfo.getKafkaParam("topic"); + + Properties props = new Properties(); + for (String key:kafka011SinkTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka011SinkTableInfo.getKafkaParam(key)); + } + this.properties = props; + this.partitioner = Optional.of(new FlinkFixedPartitioner<>()); + this.fieldNames = kafka011SinkTableInfo.getFields(); + TypeInformation[] types = new TypeInformation[kafka011SinkTableInfo.getFields().length]; + for(int i = 0; i< kafka011SinkTableInfo.getFieldClasses().length; i++){ + types[i] = TypeInformation.of(kafka011SinkTableInfo.getFieldClasses()[i]); } this.fieldTypes = types; - properties = new Properties(); - for (String key : kafka11SinkTableInfo.getKafkaParamKeys()) { - properties.setProperty(key, kafka11SinkTableInfo.getKafkaParam(key)); + TableSchema.Builder schemaBuilder = TableSchema.builder(); + for (int i=0;i dataStream) { - KafkaTableSinkBase kafkaTableSink = new CustomerKafka11JsonTableSink( + KafkaTableSinkBase kafkaTableSink = new Kafka011TableSink( + schema, topic, properties, + partitioner, serializationSchema ); @@ -106,4 +139,5 @@ public TableSink configure(String[] fieldNames, TypeInformation[] fieldT this.fieldTypes = fieldTypes; return this; } + } \ No newline at end of file diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index 4ee9d99f0..f487df080 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -38,16 +38,19 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map kafkaParam = new HashMap(); - - public KafkaSinkTableInfo() { + public KafkaSinkTableInfo(){ super.setType(CURR_TYPE); } + public Map kafkaParam = new HashMap(); + public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); } @@ -62,34 +55,16 @@ public Set getKafkaParamKeys(){ return kafkaParam.keySet(); } - - - public String getBootstrapServers() { - return bootstrapServers; - } - - public void setBootstrapServers(String bootstrapServers) { - this.bootstrapServers = bootstrapServers; - } - - public String getTopic() { - return topic; - } - - public void setTopic(String topic) { - this.topic = topic; - } - @Override public boolean check() { - Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); - Preconditions.checkNotNull(topic, "kafka of topic is required"); + Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + //Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); return false; } @Override public String getType() { -// return super.getType() + SOURCE_SUFFIX; return super.getType(); } -} +} \ No newline at end of file diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java deleted file mode 100644 index 7fc891638..000000000 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCommonDeserialization.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * 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 com.dtstack.flink.sql.source.kafka.deserialization; - -import com.dtstack.flink.sql.source.AbsDeserialization; -import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; -import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; -import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; -import org.apache.flink.types.Row; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.consumer.internals.SubscriptionState; -import org.apache.kafka.common.TopicPartition; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.lang.reflect.Field; -import java.util.Set; - -import static com.dtstack.flink.sql.metric.MetricConstant.DT_PARTITION_GROUP; -import static com.dtstack.flink.sql.metric.MetricConstant.DT_TOPIC_GROUP; -import static com.dtstack.flink.sql.metric.MetricConstant.DT_TOPIC_PARTITION_LAG_GAUGE; -import static java.nio.charset.StandardCharsets.UTF_8; - -/** - * - * Date: 2018/12/18 - * Company: www.dtstack.com - * @author DocLi - * - * @modifyer maqi - * - */ -public class CustomerCommonDeserialization extends AbsDeserialization implements KeyedDeserializationSchema { - private static final Logger LOG = LoggerFactory.getLogger(CustomerCommonDeserialization.class); - - public static final String[] KAFKA_COLUMNS = new String[]{"_TOPIC", "_MESSAGEKEY", "_MESSAGE", "_PARTITION", "_OFFSET"}; - - private AbstractFetcher fetcher; - - private boolean firstMsg = true; - - @Override - public Row deserialize(byte[] messageKey, byte[] message, String topic, int partition, long offset) { - if (firstMsg) { - try { - registerPtMetric(fetcher); - } catch (Exception e) { - LOG.error("register topic partition metric error.", e); - } - firstMsg = false; - } - - numInRecord.inc(); - numInBytes.inc(message.length); - numInBytes.inc(messageKey.length); - - try { - Row row = Row.of( - topic, //topic - messageKey == null ? null : new String(messageKey, UTF_8), //key - new String(message, UTF_8), //message - partition, - offset - ); - return row; - } catch (Throwable t) { - LOG.error(t.getMessage()); - dirtyDataCounter.inc(); - return null; - } - } - - @Override - public Row deserialize(byte[] message) throws IOException { - return null; - } - - public void setFetcher(AbstractFetcher fetcher) { - this.fetcher = fetcher; - } - - - @Override - public boolean isEndOfStream(Row nextElement) { - return false; - } - - public TypeInformation getProducedType() { - TypeInformation[] types = new TypeInformation[]{ - TypeExtractor.createTypeInfo(String.class), - TypeExtractor.createTypeInfo(String.class), //createTypeInformation[String] - TypeExtractor.createTypeInfo(String.class), - Types.INT, - Types.LONG - }; - return new RowTypeInfo(types, KAFKA_COLUMNS); - } - - protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { - - Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); - consumerThreadField.setAccessible(true); - KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); - - Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); - hasAssignedPartitionsField.setAccessible(true); - - //wait until assignedPartitions - - boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); - - if (!hasAssignedPartitions) { - throw new RuntimeException("wait 50 secs, but not assignedPartitions"); - } - - Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); - consumerField.setAccessible(true); - - KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); - Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); - subscriptionStateField.setAccessible(true); - - //topic partitions lag - SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); - Set assignedPartitions = subscriptionState.assignedPartitions(); - for (TopicPartition topicPartition : assignedPartitions) { - MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) - .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); - metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); - } - - } -} diff --git a/kafka11/pom.xml b/kafka11/pom.xml index 90972fa82..a9112cb49 100644 --- a/kafka11/pom.xml +++ b/kafka11/pom.xml @@ -14,7 +14,7 @@ kafka11-source - kafka11-sink + diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java index 0a56f03c6..571229510 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java @@ -55,8 +55,6 @@ public class LauncherOptions { private String queue; - private int defaultParallelism=1; - public String getMode() { return mode; } @@ -160,12 +158,4 @@ public String getQueue() { public void setQueue(String queue) { this.queue = queue; } - - public int getDefaultParallelism() { - return defaultParallelism; - } - - public void setDefaultParallelism(int defaultParallelism) { - this.defaultParallelism = defaultParallelism; - } } From 8abea2b653b4a7ef4fc6a5eea6496e616325ea3c Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Thu, 18 Jul 2019 13:51:04 +0800 Subject: [PATCH 313/470] parse unsigned --- .gitignore | 1 + .../com/dtstack/flink/sql/util/ClassUtil.java | 16 +++++++++++----- .../flink/sql/side/rdb/util/SwitchUtil.java | 15 ++++++++++----- 3 files changed, 22 insertions(+), 10 deletions(-) diff --git a/.gitignore b/.gitignore index cedc8fd14..33d383560 100644 --- a/.gitignore +++ b/.gitignore @@ -10,6 +10,7 @@ target/ plugins/ lib/ .vertx/ +.DS_Store bin/nohup.out bin/sideSql.txt diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java index 83f5170fb..feebbe3f1 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ClassUtil.java @@ -39,10 +39,14 @@ public static Class stringConvertClass(String str) { case "bit": return Boolean.class; + case "smallint": + case "smallintunsigned": + case "tinyint": + case "tinyintunsigned": + case "mediumint": + case "mediumintunsigned": case "integer": case "int": - case "mediumint": - case "tinyint": return Integer.class; case "blob": @@ -51,11 +55,9 @@ public static Class stringConvertClass(String str) { case "bigint": case "intunsigned": case "integerunsigned": + case "bigintunsigned": return Long.class; - case "smallint": - return Short.class; - case "varchar": case "char": case "text": @@ -63,9 +65,12 @@ public static Class stringConvertClass(String str) { case "real": case "float": + case "realunsigned": + case "floatunsigned": return Float.class; case "double": + case "doubleunsigned": return Double.class; case "date": @@ -79,6 +84,7 @@ public static Class stringConvertClass(String str) { return Time.class; case "decimal": + case "decimalunsigned": return BigDecimal.class; } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java index ad12be33a..0c4a0db13 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java @@ -31,10 +31,14 @@ public static Object getTarget(Object obj, String targetType) { targetType = targetType.toLowerCase(); switch (targetType) { + case "smallint": + case "smallintunsigned": case "tinyint": - case "int": - case "integer": + case "tinyintunsigned": case "mediumint": + case "mediumintunsigned": + case "integer": + case "int": return MathUtil.getIntegerVal(obj); case "bigint": @@ -48,9 +52,6 @@ public static Object getTarget(Object obj, String targetType) { case "blob": return MathUtil.getByte(obj); - case "smallint": - return MathUtil.getShort(obj); - case "varchar": case "char": case "text": @@ -58,12 +59,16 @@ public static Object getTarget(Object obj, String targetType) { case "real": case "float": + case "realunsigned": + case "floatunsigned": return MathUtil.getFloatVal(obj); case "double": + case "doubleunsigned": return MathUtil.getDoubleVal(obj); case "decimal": + case "decimalunsigned": return MathUtil.getBigDecimal(obj); case "date": From 1ccd78a1d223431b326a66b972aa21b56a9147e8 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Fri, 19 Jul 2019 10:38:40 +0800 Subject: [PATCH 314/470] set closure cleaner --- core/src/main/java/com/dtstack/flink/sql/Main.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 210c9ad82..1cb64d6c5 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -321,6 +321,7 @@ private static StreamExecutionEnvironment getStreamExeEnv(Properties confPropert StreamExecutionEnvironment.getExecutionEnvironment() : new MyLocalStreamEnvironment(); + env.getConfig().disableClosureCleaner(); env.setParallelism(FlinkUtil.getEnvParallelism(confProperties)); Configuration globalJobParameters = new Configuration(); Method method = Configuration.class.getDeclaredMethod("setValueInternal", String.class, Object.class); From 7d39bb2932bd3bb224b381b4aa6d6e3c1895cc76 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Sat, 20 Jul 2019 10:59:46 +0800 Subject: [PATCH 315/470] print detail error log when async side load data --- .gitignore | 2 +- .../side/cassandra/CassandraAsyncReqRow.java | 2 +- .../com/dtstack/flink/sql/side/AsyncReqRow.java | 17 ++++++++++++++++- hbase/hbase-side/hbase-async-side/pom.xml | 1 + .../rowkeydealer/RowKeyEqualModeDealer.java | 2 +- .../sql/side/rdb/async/RdbAsyncReqRow.java | 4 ++-- .../flink/sql/side/redis/RedisAsyncReqRow.java | 3 ++- 7 files changed, 24 insertions(+), 7 deletions(-) diff --git a/.gitignore b/.gitignore index cedc8fd14..74351781f 100644 --- a/.gitignore +++ b/.gitignore @@ -11,5 +11,5 @@ plugins/ lib/ .vertx/ bin/nohup.out - +.DS_Store bin/sideSql.txt diff --git a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java index 94c8e6fb6..a4b04ed93 100644 --- a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java +++ b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java @@ -256,7 +256,7 @@ public void onFailure(Throwable t) { t.getMessage()); System.out.println("Failed to retrieve the data: " + t.getMessage()); cluster.closeAsync(); - resultFuture.complete(null); + resultFuture.completeExceptionally(t); } }); } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java index 6df1af1d5..a496730e4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AsyncReqRow.java @@ -16,7 +16,7 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.side; @@ -28,9 +28,12 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.streaming.api.functions.async.RichAsyncFunction; +import org.apache.flink.streaming.api.operators.async.queue.StreamRecordQueueEntry; import org.apache.flink.types.Row; +import java.util.Collection; import java.util.Collections; +import java.util.concurrent.TimeoutException; /** * All interfaces inherit naming rules: type + "AsyncReqRow" such as == "MysqlAsyncReqRow @@ -50,6 +53,18 @@ public AsyncReqRow(SideInfo sideInfo){ this.sideInfo = sideInfo; } + @Override + public void timeout(Row input, ResultFuture resultFuture) throws Exception { + StreamRecordQueueEntry future = (StreamRecordQueueEntry)resultFuture; + try { + if (null == future.get()) { + new TimeoutException("Async function call has timed out."); + } + } catch (Exception e) { + throw new Exception(e); + } + } + private void initCache(){ SideTableInfo sideTableInfo = sideInfo.getSideTableInfo(); if(sideTableInfo.getCacheType() == null || ECacheType.NONE.name().equalsIgnoreCase(sideTableInfo.getCacheType())){ diff --git a/hbase/hbase-side/hbase-async-side/pom.xml b/hbase/hbase-side/hbase-async-side/pom.xml index 312d2c2ae..d02498881 100644 --- a/hbase/hbase-side/hbase-async-side/pom.xml +++ b/hbase/hbase-side/hbase-async-side/pom.xml @@ -50,6 +50,7 @@ org.apache.hadoop:hadoop-common + org.apache.hadoop:hadoop-yarn-common org.apache.hadoop:hadoop-auth org.apache.hadoop:hadoop-mapreduce-client-core org.slf4j:* diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java index 4a4b60ff6..fe3149e3d 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java @@ -106,7 +106,7 @@ public void asyncGetData(String tableName, String rowKeyStr, Row input, ResultFu } } }catch (Exception e){ - resultFuture.complete(null); + resultFuture.completeExceptionally(e); LOG.error("record:" + input); LOG.error("get side record exception:", e); } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index e683a40a8..1e0b188ab 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -92,7 +92,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except } resultFuture.complete(rowList); } else { - throw new RuntimeException("not support cache obj type " + val.getType()); + resultFuture.completeExceptionally(new RuntimeException("not support cache obj type " + val.getType())); } return; } @@ -110,7 +110,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except connection.queryWithParams(sqlCondition, inputParams, rs -> { if (rs.failed()) { LOG.error("Cannot retrieve the data from the database", rs.cause()); - resultFuture.complete(null); + resultFuture.completeExceptionally(rs.cause()); return; } diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index 251107287..b5faff102 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -148,7 +148,8 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except } resultFuture.complete(rowList); }else{ - throw new RuntimeException("not support cache obj type " + val.getType()); + RuntimeException exception = new RuntimeException("not support cache obj type " + val.getType()); + resultFuture.completeExceptionally(exception); } return; } From 1bcb89dd36691382224486f917123734400c3bbd Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 22 Jul 2019 14:11:10 +0800 Subject: [PATCH 316/470] support GREATER_THAN --- .../src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 14dcf125d..00f1ecfea 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -392,6 +392,10 @@ private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable Date: Thu, 25 Jul 2019 16:34:11 +0800 Subject: [PATCH 317/470] =?UTF-8?q?=E4=BF=AE=E6=94=B9split=20=E9=81=87?= =?UTF-8?q?=E5=88=B0()=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flink/sql/util/DtStringUtil.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index b65ab1e57..9d23afad6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -57,6 +57,7 @@ public static List splitIgnoreQuota(String str, char delimiter){ List tokensList = new ArrayList<>(); boolean inQuotes = false; boolean inSingleQuotes = false; + int bracketLeftNum = 0; StringBuilder b = new StringBuilder(); for (char c : str.toCharArray()) { if(c == delimiter){ @@ -64,6 +65,8 @@ public static List splitIgnoreQuota(String str, char delimiter){ b.append(c); } else if(inSingleQuotes){ b.append(c); + } else if(bracketLeftNum > 0){ + b.append(c); }else { tokensList.add(b.toString()); b = new StringBuilder(); @@ -74,6 +77,12 @@ public static List splitIgnoreQuota(String str, char delimiter){ }else if(c == '\''){ inSingleQuotes = !inSingleQuotes; b.append(c); + }else if(c == '('){ + bracketLeftNum++; + b.append(c); + }else if(c == ')'){ + bracketLeftNum--; + b.append(c); }else{ b.append(c); } From fe312602919657ff21c55e337881dca646dff7bd Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 25 Jul 2019 16:46:49 +0800 Subject: [PATCH 318/470] =?UTF-8?q?=E5=9B=9E=E6=BB=9A=E6=9C=AC=E5=9C=B0?= =?UTF-8?q?=E5=88=86=E6=94=AF?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flink/sql/util/DtStringUtil.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index 9d23afad6..b65ab1e57 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -57,7 +57,6 @@ public static List splitIgnoreQuota(String str, char delimiter){ List tokensList = new ArrayList<>(); boolean inQuotes = false; boolean inSingleQuotes = false; - int bracketLeftNum = 0; StringBuilder b = new StringBuilder(); for (char c : str.toCharArray()) { if(c == delimiter){ @@ -65,8 +64,6 @@ public static List splitIgnoreQuota(String str, char delimiter){ b.append(c); } else if(inSingleQuotes){ b.append(c); - } else if(bracketLeftNum > 0){ - b.append(c); }else { tokensList.add(b.toString()); b = new StringBuilder(); @@ -77,12 +74,6 @@ public static List splitIgnoreQuota(String str, char delimiter){ }else if(c == '\''){ inSingleQuotes = !inSingleQuotes; b.append(c); - }else if(c == '('){ - bracketLeftNum++; - b.append(c); - }else if(c == ')'){ - bracketLeftNum--; - b.append(c); }else{ b.append(c); } From 0529a423b65dd5dfa0e7f98d5f2caf0b3ee76471 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 25 Jul 2019 16:48:08 +0800 Subject: [PATCH 319/470] =?UTF-8?q?=E4=BF=AE=E6=94=B9=20string=20=E5=88=87?= =?UTF-8?q?=E5=88=86=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flink/sql/util/DtStringUtil.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index b65ab1e57..9d23afad6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -57,6 +57,7 @@ public static List splitIgnoreQuota(String str, char delimiter){ List tokensList = new ArrayList<>(); boolean inQuotes = false; boolean inSingleQuotes = false; + int bracketLeftNum = 0; StringBuilder b = new StringBuilder(); for (char c : str.toCharArray()) { if(c == delimiter){ @@ -64,6 +65,8 @@ public static List splitIgnoreQuota(String str, char delimiter){ b.append(c); } else if(inSingleQuotes){ b.append(c); + } else if(bracketLeftNum > 0){ + b.append(c); }else { tokensList.add(b.toString()); b = new StringBuilder(); @@ -74,6 +77,12 @@ public static List splitIgnoreQuota(String str, char delimiter){ }else if(c == '\''){ inSingleQuotes = !inSingleQuotes; b.append(c); + }else if(c == '('){ + bracketLeftNum++; + b.append(c); + }else if(c == ')'){ + bracketLeftNum--; + b.append(c); }else{ b.append(c); } From 64ffc6538edb70a293bcd77e1daaeebbeafe37f8 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Sat, 27 Jul 2019 11:30:23 +0800 Subject: [PATCH 320/470] add log --- core/src/main/java/com/dtstack/flink/sql/Main.java | 6 +++--- .../java/com/dtstack/flink/sql/side/SideSqlExec.java | 6 ++++++ .../dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java | 6 ++++++ .../sql/source/kafka/CustomerJsonDeserialization.java | 6 ++++++ .../flink/sql/source/kafka/table/KafkaSourceParser.java | 5 +++++ .../sql/source/kafka/CustomerJsonDeserialization.java | 6 ++++++ .../sql/source/kafka/CustomerJsonDeserialization.java | 6 ++++++ .../dtstack/flink/sql/sink/mongo/MongoOutputFormat.java | 6 ++++++ .../sql/sink/rdb/format/RetractJDBCOutputFormat.java | 4 ++++ .../dtstack/flink/sql/sink/redis/RedisOutputFormat.java | 9 +++++++++ 10 files changed, 57 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index a34ef54e8..be432320b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -173,9 +173,7 @@ public static void main(String[] args) throws Exception { } for (InsertSqlParser.SqlParseResult result : sqlTree.getExecSqlList()) { - if(LOG.isInfoEnabled()){ - LOG.info("exe-sql:\n" + result.getExecSql()); - } + LOG.info("exe-sql:\n" + result.getExecSql()); boolean isSide = false; @@ -205,6 +203,7 @@ public static void main(String[] args) throws Exception { sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache); }else{ tableEnv.sqlUpdate(result.getExecSql()); + LOG.info("exec sql: " + result.getExecSql()); } } } @@ -287,6 +286,7 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en Table regTable = tableEnv.fromDataStream(adaptStream, fields); tableEnv.registerTable(tableInfo.getName(), regTable); + LOG.info("registe table {} success.", tableInfo.getName()); registerTableCache.put(tableInfo.getName(), regTable); classPathSet.add(PluginUtil.getRemoteJarFilePath(tableInfo.getType(), SourceTableInfo.SOURCE_SUFFIX, remoteSqlPluginPath, localSqlPluginPath)); } else if (tableInfo instanceof TargetTableInfo) { diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 37dcd84f8..1eb0addfc 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -50,6 +50,8 @@ import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.*; @@ -64,6 +66,8 @@ public class SideSqlExec { + private static final Logger LOG = LoggerFactory.getLogger(SideSqlExec.class); + private String localSqlPluginPath = null; private String tmpFields = null; @@ -102,6 +106,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl if(pollSqlNode.getKind() == INSERT){ tableEnv.sqlUpdate(pollSqlNode.toString()); + LOG.info("exec sql: " + pollSqlNode.toString()); }else if(pollSqlNode.getKind() == AS){ AliasInfo aliasInfo = parseASNode(pollSqlNode); Table table = tableEnv.sql(aliasInfo.getName()); @@ -531,6 +536,7 @@ public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, AliasInfo aliasInfo = parseASNode(pollSqlNode); Table table = tableEnv.sql(aliasInfo.getName()); tableEnv.registerTable(aliasInfo.getAlias(), table); + LOG.info("Register Table {} by {}", aliasInfo.getAlias(), aliasInfo.getName()); localTableCache.put(aliasInfo.getAlias(), table); } else if (pollSqlNode.getKind() == SELECT){ Table table = tableEnv.sqlQuery(pollObj.toString()); diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java index 2ef2c6d52..e88f34372 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/HbaseOutputFormat.java @@ -64,6 +64,8 @@ public class HbaseOutputFormat extends MetricOutputFormat { public final SimpleDateFormat ROWKEY_DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss"); public final SimpleDateFormat FIELD_DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + private static int rowLenth = 1000; + @Override public void configure(Configuration parameters) { LOG.warn("---configure---"); @@ -133,6 +135,10 @@ public void writeRecord(Tuple2 tuple2) throws IOException { } table.put(put); + + if (outRecords.getCount()%rowLenth == 0){ + LOG.info(record.toString()); + } outRecords.inc(); } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index aecb4e9d4..9cc6d2fc6 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -62,6 +62,8 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private static final long serialVersionUID = 2385115520960444192L; + private static int rowLenth = 1000; + private final ObjectMapper objectMapper = new ObjectMapper(); /** Type information describing the result type. */ @@ -113,6 +115,10 @@ public Row deserialize(byte[] message) throws IOException { numInBytes.inc(message.length); JsonNode root = objectMapper.readTree(message); + + if (numInResolveRecord.getCount()%rowLenth == 0){ + LOG.info(root.toString()); + } parseTree(root, null); Row row = new Row(fieldNames.length); diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index ab9944589..e527761ba 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -25,6 +25,8 @@ import com.dtstack.flink.sql.table.TableInfo; import com.dtstack.flink.sql.util.ClassUtil; import com.dtstack.flink.sql.util.MathUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Map; import java.util.regex.Matcher; @@ -39,6 +41,8 @@ public class KafkaSourceParser extends AbsSourceParser { + private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceParser.class); + private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); @@ -59,6 +63,7 @@ static void dealNestField(Matcher matcher, TableInfo tableInfo) { tableInfo.addField(mappingField); tableInfo.addFieldClass(fieldClass); tableInfo.addFieldType(fieldType); + LOG.info(physicalField + "--->" + mappingField + " Class: " + fieldClass.toString()); } @Override diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index daa487a07..1337d917a 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -62,6 +62,8 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private static final long serialVersionUID = 2385115520960444192L; + private static int rowLenth = 1000; + private final ObjectMapper objectMapper = new ObjectMapper(); /** Type information describing the result type. */ @@ -112,6 +114,10 @@ public Row deserialize(byte[] message) throws IOException { numInBytes.inc(message.length); JsonNode root = objectMapper.readTree(message); + + if (numInResolveRecord.getCount()%rowLenth == 0){ + LOG.info(root.toString()); + } parseTree(root, null); Row row = new Row(fieldNames.length); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 26a27773b..f19586e4b 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -64,6 +64,8 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private static final long serialVersionUID = 2385115520960444192L; + private static int rowLenth = 1000; + private final ObjectMapper objectMapper = new ObjectMapper(); /** Type information describing the result type. */ @@ -115,6 +117,10 @@ public Row deserialize(byte[] message) throws IOException { numInBytes.inc(message.length); JsonNode root = objectMapper.readTree(message); + + if (numInResolveRecord.getCount()%rowLenth == 0){ + LOG.info(root.toString()); + } parseTree(root, null); Row row = new Row(fieldNames.length); diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java index 87ffc7354..f37f5f379 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/MongoOutputFormat.java @@ -63,6 +63,8 @@ public class MongoOutputFormat extends MetricOutputFormat { private static String PK = "_ID"; + private static int rowLenth = 1000; + public final SimpleDateFormat ROWKEY_DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss"); @Override @@ -107,6 +109,10 @@ public void writeRecord(Tuple2 tuple2) throws IOException { } else { dbCollection.insertOne(doc); } + + if (outRecords.getCount()%rowLenth == 0){ + LOG.info(record.toString()); + } outRecords.inc(); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 382b687e1..646fe9e0f 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -61,6 +61,7 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private PreparedStatement upload; private int batchCount = 0; + private static int rowLenth = 1000; //index field private Map> realIndexes = Maps.newHashMap(); @@ -140,6 +141,9 @@ public void writeRecord(Tuple2 tuple2) throws IOException { try { if (retract) { insertWrite(row); + if (outRecords.getCount()%rowLenth == 0){ + LOG.info(row.toString()); + } outRecords.inc(); } else { //do nothing diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java index 962054f24..9077c989e 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java @@ -24,12 +24,15 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import redis.clients.jedis.*; import java.io.Closeable; import java.io.IOException; import java.util.*; public class RedisOutputFormat extends MetricOutputFormat { + private static final Logger LOG = LoggerFactory.getLogger(RedisOutputFormat.class); private String url; @@ -65,6 +68,8 @@ public class RedisOutputFormat extends MetricOutputFormat { private GenericObjectPoolConfig poolConfig; + private static int rowLenth = 1000; + private RedisOutputFormat(){ } @Override @@ -168,6 +173,10 @@ public void writeRecord(Tuple2 record) throws IOException { key.append(tableName).append(":").append(perKey).append(":").append(fieldNames[i]); jedis.set(key.toString(), row.getField(i).toString()); } + + if (outRecords.getCount()%rowLenth == 0){ + LOG.info(record.toString()); + } outRecords.inc(); } From dae225f365a06450dfe2bbab487b4068272c6d22 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Sat, 27 Jul 2019 13:51:17 +0800 Subject: [PATCH 321/470] add log --- .../source/kafka/CustomerJsonDeserialization.java | 13 ++++++++----- .../source/kafka/CustomerJsonDeserialization.java | 12 ++++++++---- .../source/kafka/CustomerJsonDeserialization.java | 12 ++++++++---- 3 files changed, 24 insertions(+), 13 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 9cc6d2fc6..58cd41f9c 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -111,14 +111,14 @@ public Row deserialize(byte[] message) throws IOException { } try { - numInRecord.inc(); - numInBytes.inc(message.length); - JsonNode root = objectMapper.readTree(message); - - if (numInResolveRecord.getCount()%rowLenth == 0){ + if (numInRecord.getCount()%rowLenth == 0){ LOG.info(root.toString()); } + + numInRecord.inc(); + numInBytes.inc(message.length); + parseTree(root, null); Row row = new Row(fieldNames.length); @@ -143,6 +143,9 @@ public Row deserialize(byte[] message) throws IOException { return row; } catch (Throwable t) { //add metric of dirty data + if (dirtyDataCounter.getCount()%rowLenth == 0){ + LOG.info(objectMapper.readTree(message).toString()); + } dirtyDataCounter.inc(); return null; }finally { diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 1337d917a..a1ebc091c 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -110,14 +110,15 @@ public Row deserialize(byte[] message) throws IOException { } try { - numInRecord.inc(); - numInBytes.inc(message.length); - JsonNode root = objectMapper.readTree(message); - if (numInResolveRecord.getCount()%rowLenth == 0){ + if (numInRecord.getCount()%rowLenth == 0){ LOG.info(root.toString()); } + + numInRecord.inc(); + numInBytes.inc(message.length); + parseTree(root, null); Row row = new Row(fieldNames.length); @@ -142,6 +143,9 @@ public Row deserialize(byte[] message) throws IOException { return row; } catch (Throwable t) { //add metric of dirty data + if (dirtyDataCounter.getCount()%rowLenth == 0){ + LOG.info(objectMapper.readTree(message).toString()); + } dirtyDataCounter.inc(); return null; }finally { diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index f19586e4b..9416dbbaf 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -113,14 +113,15 @@ public Row deserialize(byte[] message) throws IOException { } try { - numInRecord.inc(); - numInBytes.inc(message.length); - JsonNode root = objectMapper.readTree(message); - if (numInResolveRecord.getCount()%rowLenth == 0){ + if (numInRecord.getCount()%rowLenth == 0){ LOG.info(root.toString()); } + + numInRecord.inc(); + numInBytes.inc(message.length); + parseTree(root, null); Row row = new Row(fieldNames.length); @@ -145,6 +146,9 @@ public Row deserialize(byte[] message) throws IOException { return row; } catch (Throwable t) { //add metric of dirty data + if (dirtyDataCounter.getCount()%rowLenth == 0){ + LOG.info(objectMapper.readTree(message).toString()); + } dirtyDataCounter.inc(); return null; }finally { From 12d23a0a3ab56a41917aa4c72af9091262bba546 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Sat, 27 Jul 2019 16:46:58 +0800 Subject: [PATCH 322/470] add log --- core/src/main/java/com/dtstack/flink/sql/Main.java | 12 +++++++++--- .../java/com/dtstack/flink/sql/side/SideSqlExec.java | 8 ++++++-- .../sql/source/kafka/table/KafkaSourceParser.java | 4 +++- .../sql/source/kafka/table/KafkaSourceParser.java | 7 +++++++ .../sql/source/kafka/table/KafkaSourceParser.java | 7 +++++++ 5 files changed, 32 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index be432320b..79bec1da4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -173,7 +173,9 @@ public static void main(String[] args) throws Exception { } for (InsertSqlParser.SqlParseResult result : sqlTree.getExecSqlList()) { - LOG.info("exe-sql:\n" + result.getExecSql()); + if(LOG.isInfoEnabled()){ + LOG.info("exe-sql:\n" + result.getExecSql()); + } boolean isSide = false; @@ -203,7 +205,9 @@ public static void main(String[] args) throws Exception { sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache); }else{ tableEnv.sqlUpdate(result.getExecSql()); - LOG.info("exec sql: " + result.getExecSql()); + if(LOG.isInfoEnabled()){ + + }LOG.info("exec sql: " + result.getExecSql()); } } } @@ -286,7 +290,9 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en Table regTable = tableEnv.fromDataStream(adaptStream, fields); tableEnv.registerTable(tableInfo.getName(), regTable); - LOG.info("registe table {} success.", tableInfo.getName()); + if(LOG.isInfoEnabled()){ + LOG.info("registe table {} success.", tableInfo.getName()); + } registerTableCache.put(tableInfo.getName(), regTable); classPathSet.add(PluginUtil.getRemoteJarFilePath(tableInfo.getType(), SourceTableInfo.SOURCE_SUFFIX, remoteSqlPluginPath, localSqlPluginPath)); } else if (tableInfo instanceof TargetTableInfo) { diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 1eb0addfc..18fb4d2d8 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -106,7 +106,9 @@ public void exec(String sql, Map sideTableMap, StreamTabl if(pollSqlNode.getKind() == INSERT){ tableEnv.sqlUpdate(pollSqlNode.toString()); - LOG.info("exec sql: " + pollSqlNode.toString()); + if(LOG.isInfoEnabled()){ + LOG.info("exec sql: " + pollSqlNode.toString()); + } }else if(pollSqlNode.getKind() == AS){ AliasInfo aliasInfo = parseASNode(pollSqlNode); Table table = tableEnv.sql(aliasInfo.getName()); @@ -536,7 +538,9 @@ public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, AliasInfo aliasInfo = parseASNode(pollSqlNode); Table table = tableEnv.sql(aliasInfo.getName()); tableEnv.registerTable(aliasInfo.getAlias(), table); - LOG.info("Register Table {} by {}", aliasInfo.getAlias(), aliasInfo.getName()); + if(LOG.isInfoEnabled()){ + LOG.info("Register Table {} by {}", aliasInfo.getAlias(), aliasInfo.getName()); + } localTableCache.put(aliasInfo.getAlias(), table); } else if (pollSqlNode.getKind() == SELECT){ Table table = tableEnv.sqlQuery(pollObj.toString()); diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index e527761ba..c2fe6cb21 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -63,7 +63,9 @@ static void dealNestField(Matcher matcher, TableInfo tableInfo) { tableInfo.addField(mappingField); tableInfo.addFieldClass(fieldClass); tableInfo.addFieldType(fieldType); - LOG.info(physicalField + "--->" + mappingField + " Class: " + fieldClass.toString()); + if(LOG.isInfoEnabled()){ + LOG.info(physicalField + "--->" + mappingField + " Class: " + fieldClass.toString()); + } } @Override diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index e4f34237b..020d54781 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -24,6 +24,8 @@ import com.dtstack.flink.sql.table.TableInfo; import com.dtstack.flink.sql.util.ClassUtil; import com.dtstack.flink.sql.util.MathUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Map; import java.util.regex.Matcher; @@ -38,6 +40,8 @@ public class KafkaSourceParser extends AbsSourceParser { + private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceParser.class); + private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); @@ -63,6 +67,9 @@ static void dealNestField(Matcher matcher, TableInfo tableInfo) { tableInfo.addField(mappingField); tableInfo.addFieldClass(fieldClass); tableInfo.addFieldType(fieldType); + if(LOG.isInfoEnabled()){ + LOG.info(physicalField + "--->" + mappingField + " Class: " + fieldClass.toString()); + } } @Override diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index dcd0052ae..85f6cd189 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -24,6 +24,8 @@ import com.dtstack.flink.sql.table.TableInfo; import com.dtstack.flink.sql.util.ClassUtil; import com.dtstack.flink.sql.util.MathUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Map; import java.util.regex.Matcher; @@ -38,6 +40,8 @@ public class KafkaSourceParser extends AbsSourceParser { + private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceParser.class); + private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); @@ -63,6 +67,9 @@ static void dealNestField(Matcher matcher, TableInfo tableInfo) { tableInfo.addField(mappingField); tableInfo.addFieldClass(fieldClass); tableInfo.addFieldType(fieldType); + if(LOG.isInfoEnabled()){ + LOG.info(physicalField + "--->" + mappingField + " Class: " + fieldClass.toString()); + } } @Override From 5fe64dd1c1836fe356ad32180ec668379f8b27f0 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Sat, 27 Jul 2019 16:58:12 +0800 Subject: [PATCH 323/470] add log --- core/src/main/java/com/dtstack/flink/sql/Main.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 79bec1da4..9ddb45682 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -206,8 +206,8 @@ public static void main(String[] args) throws Exception { }else{ tableEnv.sqlUpdate(result.getExecSql()); if(LOG.isInfoEnabled()){ - - }LOG.info("exec sql: " + result.getExecSql()); + LOG.info("exec sql: " + result.getExecSql()); + } } } } From e41fe385d2bb028598f0aa01b01210edde82d15f Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Sun, 28 Jul 2019 12:46:03 +0800 Subject: [PATCH 324/470] add log --- .../flink/sql/source/kafka/CustomerJsonDeserialization.java | 2 +- .../flink/sql/source/kafka/CustomerJsonDeserialization.java | 2 +- .../flink/sql/source/kafka/CustomerJsonDeserialization.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 58cd41f9c..28f0e470b 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -144,7 +144,7 @@ public Row deserialize(byte[] message) throws IOException { } catch (Throwable t) { //add metric of dirty data if (dirtyDataCounter.getCount()%rowLenth == 0){ - LOG.info(objectMapper.readTree(message).toString()); + LOG.info("dirtyData: " + new String(message)); } dirtyDataCounter.inc(); return null; diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index a1ebc091c..66d41e0ba 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -144,7 +144,7 @@ public Row deserialize(byte[] message) throws IOException { } catch (Throwable t) { //add metric of dirty data if (dirtyDataCounter.getCount()%rowLenth == 0){ - LOG.info(objectMapper.readTree(message).toString()); + LOG.info("dirtyData: " + new String(message)); } dirtyDataCounter.inc(); return null; diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 9416dbbaf..3b73f0d51 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -147,7 +147,7 @@ public Row deserialize(byte[] message) throws IOException { } catch (Throwable t) { //add metric of dirty data if (dirtyDataCounter.getCount()%rowLenth == 0){ - LOG.info(objectMapper.readTree(message).toString()); + LOG.info("dirtyData: " + new String(message)); } dirtyDataCounter.inc(); return null; From 86f93cc0cccfdcf9856ee18027c17fa303786bb2 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 29 Jul 2019 16:09:02 +0800 Subject: [PATCH 325/470] add scheduled --- pom.xml | 10 +-- .../dtstack/flink/sql/sink/rdb/RdbSink.java | 28 ++++--- .../rdb/format/RetractJDBCOutputFormat.java | 76 +++++++++++++++---- .../sql/sink/rdb/table/RdbSinkParser.java | 1 + .../sql/sink/rdb/table/RdbTableInfo.java | 12 +++ 5 files changed, 97 insertions(+), 30 deletions(-) diff --git a/pom.xml b/pom.xml index 0e2d9e709..eeb4d5fa4 100644 --- a/pom.xml +++ b/pom.xml @@ -14,15 +14,15 @@ kafka10 kafka11 mysql - hbase - elasticsearch5 - mongo - redis5 + + + + launcher rdb sqlserver oracle - cassandra + diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java index 5ee2c44c4..cb93ea634 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java @@ -61,7 +61,9 @@ public abstract class RdbSink implements RetractStreamTableSink, Serializab protected String dbType; - protected int batchInterval = 1; + protected int batchNum = 1; + + protected long batchWaitInterval; protected int[] sqlTypes; @@ -89,7 +91,8 @@ public RichSinkFunction createJdbcSinkFunc() { outputFormat.setUsername(userName); outputFormat.setPassword(password); outputFormat.setInsertQuery(sql); - outputFormat.setBatchInterval(batchInterval); + outputFormat.setBatchNum(batchNum); + outputFormat.setBatchWaitInterval(batchWaitInterval); outputFormat.setTypesArray(sqlTypes); outputFormat.setTableName(tableName); outputFormat.setDbType(dbType); @@ -112,7 +115,12 @@ public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { Integer tmpSqlBatchSize = rdbTableInfo.getBatchSize(); if (tmpSqlBatchSize != null) { - setBatchInterval(tmpSqlBatchSize); + setBatchNum(tmpSqlBatchSize); + } + + long batchWaitInterval = rdbTableInfo.getBatchWaitInterval(); + if (batchWaitInterval > 0) { + setBatchWaitInterval(batchWaitInterval); } Integer tmpSinkParallelism = rdbTableInfo.getParallelism(); @@ -198,13 +206,13 @@ public TableSink> configure(String[] fieldNames, TypeInform return this; } - /** - * Set the default frequency submit updated every submission - * - * @param batchInterval - */ - public void setBatchInterval(int batchInterval) { - this.batchInterval = batchInterval; + + public void setBatchNum(int batchNum) { + this.batchNum = batchNum; + } + + public void setBatchWaitInterval(long batchWaitInterval) { + this.batchWaitInterval = batchWaitInterval; } @Override diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 382b687e1..0d2973d12 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -20,7 +20,6 @@ import com.dtstack.flink.sql.sink.rdb.RdbSink; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; @@ -28,9 +27,18 @@ import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.sql.*; -import java.util.*; import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import com.dtstack.flink.sql.sink.MetricOutputFormat; @@ -38,8 +46,6 @@ * OutputFormat to write tuples into a database. * The OutputFormat has to be configured using the supplied OutputFormatBuilder. * - * @see Tuple - * @see DriverManager */ public class RetractJDBCOutputFormat extends MetricOutputFormat { private static final long serialVersionUID = 1L; @@ -53,22 +59,25 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private String tableName; private String dbType; private RdbSink dbSink; - private int batchInterval = 5000; + private long batchWaitInterval; + private int batchNum; private String insertQuery; public int[] typesArray; private Connection dbConn; private PreparedStatement upload; - private int batchCount = 0; + private AtomicInteger batchCount = new AtomicInteger(0); //index field private Map> realIndexes = Maps.newHashMap(); //full field private List fullField = Lists.newArrayList(); - public RetractJDBCOutputFormat() { + private transient ScheduledThreadPoolExecutor timerService; + public RetractJDBCOutputFormat() { + this.timerService = new ScheduledThreadPoolExecutor(1); } @Override @@ -151,13 +160,23 @@ public void writeRecord(Tuple2 tuple2) throws IOException { private void insertWrite(Row row) throws SQLException { - updatePreparedStmt(row, upload); upload.addBatch(); - batchCount++; - if (batchCount >= batchInterval) { + ScheduledFuture scheduledFuture = null; + + if (batchWaitInterval > 0 && batchCount.get() == 0) { + scheduledFuture = registerTimer(batchWaitInterval, this); + } + + batchCount.incrementAndGet(); + + if (batchCount.get() >= batchNum) { upload.executeBatch(); - batchCount = 0; + batchCount.set(0); + + if (scheduledFuture != null) { + scheduledFuture.cancel(true); + } } } @@ -250,6 +269,28 @@ private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLExce } } + public ScheduledFuture registerTimer(long delay, RetractJDBCOutputFormat outputFormat) { + return timerService.schedule(new DelayExecuteBatch(outputFormat), delay, TimeUnit.MILLISECONDS); + } + + private final static class DelayExecuteBatch implements Runnable { + RetractJDBCOutputFormat outputFormat; + + private DelayExecuteBatch(RetractJDBCOutputFormat outputFormat) { + this.outputFormat = outputFormat; + } + + @Override + public void run() { + try { + outputFormat.upload.executeBatch(); + outputFormat.batchCount.set(0); + } catch (SQLException e) { + LOG.error("delay batch insert error...", e); + } + } + } + /** * Executes prepared statement and closes all resources of this instance. * @@ -266,7 +307,7 @@ public void close() throws IOException { LOG.info("Inputformat couldn't be closed - " + se.getMessage()); } finally { upload = null; - batchCount = 0; + batchCount.set(0); } try { @@ -332,8 +373,8 @@ public void setDbSink(RdbSink dbSink) { this.dbSink = dbSink; } - public void setBatchInterval(int batchInterval) { - this.batchInterval = batchInterval; + public void setBatchNum(int batchNum) { + this.batchNum = batchNum; } public void setInsertQuery(String insertQuery) { @@ -364,6 +405,11 @@ public Map> getRealIndexes() { return realIndexes; } + + public void setBatchWaitInterval(long batchWaitInterval) { + this.batchWaitInterval = batchWaitInterval; + } + public List getFullField() { return fullField; } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java index 0c248997b..f135dc2f3 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java @@ -43,6 +43,7 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Mon, 29 Jul 2019 16:58:05 +0800 Subject: [PATCH 326/470] save checkpoint when sql.checkpoint.cleanup.mode is null --- .../com/dtstack/flink/sql/util/FlinkUtil.java | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java index 6b99e2c9a..89476380e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java @@ -101,16 +101,14 @@ public static void openCheckpoint(StreamExecutionEnvironment env, Properties pro } String cleanupModeStr = properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_CLEANUPMODE_KEY); - if(cleanupModeStr != null){//设置在cancel job情况下checkpoint是否被保存 - if("true".equalsIgnoreCase(cleanupModeStr)){ - env.getCheckpointConfig().enableExternalizedCheckpoints( - CheckpointConfig.ExternalizedCheckpointCleanup.DELETE_ON_CANCELLATION); - }else if("false".equalsIgnoreCase(cleanupModeStr)){ - env.getCheckpointConfig().enableExternalizedCheckpoints( - CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); - }else{ - throw new RuntimeException("not support value of cleanup mode :" + cleanupModeStr); - } + if ("true".equalsIgnoreCase(cleanupModeStr)){ + env.getCheckpointConfig().enableExternalizedCheckpoints( + CheckpointConfig.ExternalizedCheckpointCleanup.DELETE_ON_CANCELLATION); + } else if("false".equalsIgnoreCase(cleanupModeStr) || cleanupModeStr == null){ + env.getCheckpointConfig().enableExternalizedCheckpoints( + CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); + } else{ + throw new RuntimeException("not support value of cleanup mode :" + cleanupModeStr); } String backendPath = properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_DATAURI_KEY); From 40128d4d3dabcfa03281c35aae7291ff60e2b35a Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 29 Jul 2019 17:10:43 +0800 Subject: [PATCH 327/470] delay init scheduled --- .../flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 0d2973d12..614d33804 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -77,7 +77,6 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private transient ScheduledThreadPoolExecutor timerService; public RetractJDBCOutputFormat() { - this.timerService = new ScheduledThreadPoolExecutor(1); } @Override @@ -96,6 +95,8 @@ public void open(int taskNumber, int numTasks) throws IOException { try { establishConnection(); initMetric(); + + this.timerService = new ScheduledThreadPoolExecutor(1); if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { if (isReplaceInsertQuery()) { insertQuery = dbSink.buildUpdateSql(tableName, Arrays.asList(dbSink.getFieldNames()), realIndexes, fullField); @@ -174,7 +175,7 @@ private void insertWrite(Row row) throws SQLException { upload.executeBatch(); batchCount.set(0); - if (scheduledFuture != null) { + if (scheduledFuture != null && !scheduledFuture.isCancelled()) { scheduledFuture.cancel(true); } } From 18345fe13c1c72940a864421e480d415183dec57 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 29 Jul 2019 17:18:22 +0800 Subject: [PATCH 328/470] pom revert --- pom.xml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index eeb4d5fa4..0e2d9e709 100644 --- a/pom.xml +++ b/pom.xml @@ -14,15 +14,15 @@ kafka10 kafka11 mysql - - - - + hbase + elasticsearch5 + mongo + redis5 launcher rdb sqlserver oracle - + cassandra From ad59b02f902b704d523df7cb4fc9980e2e19af6c Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 29 Jul 2019 22:08:18 +0800 Subject: [PATCH 329/470] modify --- .../dtstack/flink/sql/sink/rdb/RdbSink.java | 4 +- .../rdb/format/RetractJDBCOutputFormat.java | 57 +++++++------------ .../sql/sink/rdb/table/RdbTableInfo.java | 7 ++- 3 files changed, 27 insertions(+), 41 deletions(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java index cb93ea634..43b93f431 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java @@ -118,8 +118,8 @@ public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { setBatchNum(tmpSqlBatchSize); } - long batchWaitInterval = rdbTableInfo.getBatchWaitInterval(); - if (batchWaitInterval > 0) { + Long batchWaitInterval = rdbTableInfo.getBatchWaitInterval(); + if (batchWaitInterval != null) { setBatchWaitInterval(batchWaitInterval); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 614d33804..51a661c23 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -35,7 +35,6 @@ import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -59,7 +58,7 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private String tableName; private String dbType; private RdbSink dbSink; - private long batchWaitInterval; + private long batchWaitInterval = 10000l; private int batchNum; private String insertQuery; public int[] typesArray; @@ -69,13 +68,13 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private AtomicInteger batchCount = new AtomicInteger(0); + private transient ScheduledThreadPoolExecutor timerService; + //index field private Map> realIndexes = Maps.newHashMap(); //full field private List fullField = Lists.newArrayList(); - private transient ScheduledThreadPoolExecutor timerService; - public RetractJDBCOutputFormat() { } @@ -96,7 +95,14 @@ public void open(int taskNumber, int numTasks) throws IOException { establishConnection(); initMetric(); - this.timerService = new ScheduledThreadPoolExecutor(1); + if (batchWaitInterval > 0) { + timerService = new ScheduledThreadPoolExecutor(1); + timerService.scheduleAtFixedRate(() -> { + submitExecuteBatch(); + }, 0, batchWaitInterval, TimeUnit.MILLISECONDS); + + } + if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { if (isReplaceInsertQuery()) { insertQuery = dbSink.buildUpdateSql(tableName, Arrays.asList(dbSink.getFieldNames()), realIndexes, fullField); @@ -163,21 +169,9 @@ public void writeRecord(Tuple2 tuple2) throws IOException { private void insertWrite(Row row) throws SQLException { updatePreparedStmt(row, upload); upload.addBatch(); - ScheduledFuture scheduledFuture = null; - - if (batchWaitInterval > 0 && batchCount.get() == 0) { - scheduledFuture = registerTimer(batchWaitInterval, this); - } - batchCount.incrementAndGet(); - if (batchCount.get() >= batchNum) { - upload.executeBatch(); - batchCount.set(0); - - if (scheduledFuture != null && !scheduledFuture.isCancelled()) { - scheduledFuture.cancel(true); - } + submitExecuteBatch(); } } @@ -270,25 +264,13 @@ private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLExce } } - public ScheduledFuture registerTimer(long delay, RetractJDBCOutputFormat outputFormat) { - return timerService.schedule(new DelayExecuteBatch(outputFormat), delay, TimeUnit.MILLISECONDS); - } - - private final static class DelayExecuteBatch implements Runnable { - RetractJDBCOutputFormat outputFormat; - private DelayExecuteBatch(RetractJDBCOutputFormat outputFormat) { - this.outputFormat = outputFormat; - } - - @Override - public void run() { - try { - outputFormat.upload.executeBatch(); - outputFormat.batchCount.set(0); - } catch (SQLException e) { - LOG.error("delay batch insert error...", e); - } + private synchronized void submitExecuteBatch() { + try { + this.upload.executeBatch(); + this.batchCount.set(0); + } catch (SQLException e) { + LOG.error("", e); } } @@ -304,6 +286,9 @@ public void close() throws IOException { upload.executeBatch(); upload.close(); } + if (null != timerService) { + timerService.shutdown(); + } } catch (SQLException se) { LOG.info("Inputformat couldn't be closed - " + se.getMessage()); } finally { diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java index 4b1518f88..275c72c51 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java @@ -55,7 +55,7 @@ public class RdbTableInfo extends TargetTableInfo { private Integer batchSize; - private long batchWaitInterval; + private Long batchWaitInterval; private String bufferSize; @@ -117,11 +117,12 @@ public void setFlushIntervalMs(String flushIntervalMs) { this.flushIntervalMs = flushIntervalMs; } - public long getBatchWaitInterval() { + + public Long getBatchWaitInterval() { return batchWaitInterval; } - public void setBatchWaitInterval(long batchWaitInterval) { + public void setBatchWaitInterval(Long batchWaitInterval) { this.batchWaitInterval = batchWaitInterval; } From 6ba61af6fb0eed6f3cdf8d7041063566c0a540d9 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 30 Jul 2019 09:43:30 +0800 Subject: [PATCH 330/470] add default batch wait interval --- .../src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java index 43b93f431..16d97e992 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java @@ -63,7 +63,7 @@ public abstract class RdbSink implements RetractStreamTableSink, Serializab protected int batchNum = 1; - protected long batchWaitInterval; + protected long batchWaitInterval = 10000; protected int[] sqlTypes; From f57a60ce1aceb58777f0cb22cd721f22fe140fe0 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 30 Jul 2019 11:10:34 +0800 Subject: [PATCH 331/470] add comment --- .../sql/sink/rdb/format/RetractJDBCOutputFormat.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 51a661c23..e41cd5d28 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -58,8 +58,10 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private String tableName; private String dbType; private RdbSink dbSink; + // trigger preparedStatement execute batch interval private long batchWaitInterval = 10000l; - private int batchNum; + // PreparedStatement execute batch num + private int batchNum = 1; private String insertQuery; public int[] typesArray; @@ -290,7 +292,7 @@ public void close() throws IOException { timerService.shutdown(); } } catch (SQLException se) { - LOG.info("Inputformat couldn't be closed - " + se.getMessage()); + LOG.info("Inputformat couldn't be closed - ", se); } finally { upload = null; batchCount.set(0); @@ -301,7 +303,7 @@ public void close() throws IOException { dbConn.close(); } } catch (SQLException se) { - LOG.info("Inputformat couldn't be closed - " + se.getMessage()); + LOG.info("Inputformat couldn't be closed - ", se); } finally { dbConn = null; } From 65f220c4df64d010ab337926a234c4c33e410ade Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 30 Jul 2019 11:51:15 +0800 Subject: [PATCH 332/470] add log --- .../flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 1a155fad6..14cfad10a 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -93,10 +93,13 @@ public void configure(Configuration parameters) { @Override public void open(int taskNumber, int numTasks) throws IOException { try { + LOG.info("PreparedStatement execute batch num is {}", batchNum); establishConnection(); initMetric(); if (batchWaitInterval > 0) { + LOG.info("open batch wait interval scheduled, interval is {} ms", batchWaitInterval); + timerService = new ScheduledThreadPoolExecutor(1); timerService.scheduleAtFixedRate(() -> { submitExecuteBatch(); @@ -289,6 +292,7 @@ public void close() throws IOException { } if (null != timerService) { timerService.shutdown(); + LOG.info("batch wait interval scheduled service closed "); } } catch (SQLException se) { LOG.info("Inputformat couldn't be closed - ", se); From f275739e3737e0c5df2f95046280e3831a0eebc9 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 2 Aug 2019 15:47:02 +0800 Subject: [PATCH 333/470] bigint unsigned --- pom.xml | 18 +++++++++--------- .../flink/sql/side/rdb/util/MathUtil.java | 2 ++ .../flink/sql/side/rdb/util/SwitchUtil.java | 1 + 3 files changed, 12 insertions(+), 9 deletions(-) diff --git a/pom.xml b/pom.xml index 0e2d9e709..8dd76e437 100644 --- a/pom.xml +++ b/pom.xml @@ -10,19 +10,19 @@ http://maven.apache.org core - kafka09 - kafka10 + + kafka11 mysql - hbase - elasticsearch5 - mongo - redis5 + + + + launcher rdb - sqlserver - oracle - cassandra + + + diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java index 284d458b3..cc4f229a7 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java @@ -48,6 +48,8 @@ public static Long getLongVal(Object obj) { return Long.valueOf(obj.toString()); } else if (obj instanceof BigDecimal) { return ((BigDecimal) obj).longValue(); + } else if (obj instanceof BigInteger) { + return ((BigInteger) obj).longValue(); } throw new RuntimeException("not support type of " + obj.getClass() + " convert to Long."); diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java index 0c4a0db13..02164d1d9 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java @@ -42,6 +42,7 @@ public static Object getTarget(Object obj, String targetType) { return MathUtil.getIntegerVal(obj); case "bigint": + case "bigintunsigned": case "intunsigned": case "integerunsigned": return MathUtil.getLongVal(obj); From 08d77d30775cb4ad5f9b1a81a7e53e6f65d6aa67 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 2 Aug 2019 15:49:01 +0800 Subject: [PATCH 334/470] revert pom --- pom.xml | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/pom.xml b/pom.xml index 8dd76e437..0e2d9e709 100644 --- a/pom.xml +++ b/pom.xml @@ -10,19 +10,19 @@ http://maven.apache.org core - - + kafka09 + kafka10 kafka11 mysql - - - - + hbase + elasticsearch5 + mongo + redis5 launcher rdb - - - + sqlserver + oracle + cassandra From 66410b2bae1b9be201e546473992e9660beec3db Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BE=90=E8=B6=85?= Date: Mon, 5 Aug 2019 16:51:47 +0800 Subject: [PATCH 335/470] Revert "Merge branch '1.5_3.6.0_bigunsigned' into '1.5_v3.6.0' " This reverts merge request !44 --- .../main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java | 2 -- .../java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java | 1 - 2 files changed, 3 deletions(-) diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java index cc4f229a7..284d458b3 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java @@ -48,8 +48,6 @@ public static Long getLongVal(Object obj) { return Long.valueOf(obj.toString()); } else if (obj instanceof BigDecimal) { return ((BigDecimal) obj).longValue(); - } else if (obj instanceof BigInteger) { - return ((BigInteger) obj).longValue(); } throw new RuntimeException("not support type of " + obj.getClass() + " convert to Long."); diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java index 02164d1d9..0c4a0db13 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java @@ -42,7 +42,6 @@ public static Object getTarget(Object obj, String targetType) { return MathUtil.getIntegerVal(obj); case "bigint": - case "bigintunsigned": case "intunsigned": case "integerunsigned": return MathUtil.getLongVal(obj); From 5e28f3a76f026a6d0d222cefbff82067cce1b1ab Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 5 Aug 2019 17:08:16 +0800 Subject: [PATCH 336/470] bigint unsigned --- .../java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java | 3 ++- .../java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java index 284d458b3..27e59e566 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/MathUtil.java @@ -48,8 +48,9 @@ public static Long getLongVal(Object obj) { return Long.valueOf(obj.toString()); } else if (obj instanceof BigDecimal) { return ((BigDecimal) obj).longValue(); + } else if (obj instanceof BigInteger) { + return ((BigInteger) obj).longValue(); } - throw new RuntimeException("not support type of " + obj.getClass() + " convert to Long."); } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java index 0c4a0db13..02164d1d9 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/util/SwitchUtil.java @@ -42,6 +42,7 @@ public static Object getTarget(Object obj, String targetType) { return MathUtil.getIntegerVal(obj); case "bigint": + case "bigintunsigned": case "intunsigned": case "integerunsigned": return MathUtil.getLongVal(obj); From d83e5cde9428ae0dbe0b8d8786e088284d3410b3 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 12 Aug 2019 19:53:55 +0800 Subject: [PATCH 337/470] kafka11sink --- .../sink/kafka/CustomerCsvSerialization.java | 138 ------------------ .../kafka/CustomerKafka11JsonTableSink.java | 31 ++-- .../flink/sql/sink/kafka/KafkaSink.java | 11 +- .../sql/sink/kafka/table/KafkaSinkParser.java | 14 +- .../sink/kafka/table/KafkaSinkTableInfo.java | 30 +++- kafka11/pom.xml | 2 +- 6 files changed, 55 insertions(+), 171 deletions(-) delete mode 100644 kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java deleted file mode 100644 index 3f73f667d..000000000 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerCsvSerialization.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * 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 com.dtstack.flink.sql.sink.kafka; - -import org.apache.commons.lang3.StringEscapeUtils; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.types.Row; -import org.apache.flink.types.StringValue; - -import java.io.IOException; - -import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; -/** - * - * Date: 2018/12/18 - * Company: www.dtstack.com - * @author DocLi - * - * @modifyer maqi - * - */ -public final class CustomerCsvSerialization extends TypeSerializerSingleton { - - private static final long serialVersionUID = 1L; - - private String fieldDelimiter = "\u0001"; - private TypeInformation[] fieldTypes; - private TypeSerializer[] fieldSerializers; - private static final Row EMPTY = null; - - public CustomerCsvSerialization(String fielddelimiter,TypeInformation[] fieldTypes) { - this.fieldDelimiter = fielddelimiter; - this.fieldTypes = fieldTypes; - this.fieldSerializers = (TypeSerializer[])createSerializer(new ExecutionConfig()); - } - - public TypeSerializer[] createSerializer(ExecutionConfig config) { - int len = fieldTypes.length; - TypeSerializer[] fieldSerializers = new TypeSerializer[len]; - for (int i = 0; i < len; i++) { - fieldSerializers[i] = fieldTypes[i].createSerializer(config); - } - return fieldSerializers; - } - - @Override - public boolean isImmutableType() { - return true; - } - - @Override - public Row createInstance() { - return EMPTY; - } - - @Override - public Row copy(Row from) { - return null; - } - - @Override - public Row copy(Row from, Row reuse) { - return null; - } - - @Override - public int getLength() { - return -1; - } - - @Override - public void serialize(Row record, DataOutputView target) throws IOException { - int len = fieldSerializers.length; - - if (record.getArity() != len) { - throw new RuntimeException("Row arity of from does not match serializers."); - } - - // write a null mask - writeNullMask(len, record, target); - - // serialize non-null fields - StringBuffer stringBuffer = new StringBuffer(); - for (int i = 0; i < len; i++) { - Object o = record.getField(i); - if (o != null) { - //fieldSerializers[i].serialize(o, target); - stringBuffer.append(o); - } - if(i != len-1){ - stringBuffer.append(StringEscapeUtils.unescapeJava(fieldDelimiter)); - //fieldSerializers[i].serialize(fieldDelimiter, target); - } - } - StringValue.writeString(stringBuffer.toString(), target); - } - - @Override - public Row deserialize(DataInputView source) throws IOException { - return null; - } - - @Override - public Row deserialize(Row reuse, DataInputView source) throws IOException { - return null; - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - StringValue.copyString(source, target); - } - - @Override - public TypeSerializerSnapshot snapshotConfiguration() { - return null; - } -} \ No newline at end of file diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java index 4d091d0cc..1a03da35e 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java @@ -21,11 +21,9 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.connectors.kafka.Kafka011TableSink; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; -import org.apache.flink.table.util.TableConnectorUtil; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.utils.TableConnectorUtils; import org.apache.flink.types.Row; import java.util.Optional; @@ -41,26 +39,19 @@ */ public class CustomerKafka11JsonTableSink extends Kafka011TableSink { - protected SerializationSchema schema; - public CustomerKafka11JsonTableSink(String topic, Properties properties, SerializationSchema schema) { - super(topic, properties, new FlinkFixedPartitioner<>()); - this.schema = schema; - } - public CustomerKafka11JsonTableSink(String topic, Properties properties, FlinkKafkaPartitioner partitioner, SerializationSchema schema) { - super(topic, properties, partitioner); - this.schema = schema; - } + public CustomerKafka11JsonTableSink(TableSchema schema, + String topic, + Properties properties, + Optional> partitioner, + SerializationSchema serializationSchema) { - - @Deprecated - public CustomerKafka11JsonTableSink(String topic, Properties properties, KafkaPartitioner partitioner, SerializationSchema schema) { - super(topic, properties, new FlinkKafkaDelegatePartitioner<>(partitioner)); - this.schema = schema; + super(schema, topic, properties, partitioner, serializationSchema); + this.schema = serializationSchema; } - //TODO 暂时使用010 + @Override protected SinkFunction createKafkaProducer(String s, Properties properties, SerializationSchema serializationSchema, Optional> optional) { return new CustomerFlinkKafkaProducer011(topic, serializationSchema, properties); @@ -71,6 +62,6 @@ public void emitDataStream(DataStream dataStream) { SinkFunction kafkaProducer = createKafkaProducer(topic, properties, schema, partitioner); // always enable flush on checkpoint to achieve at-least-once if query runs with checkpointing enabled. //kafkaProducer.setFlushOnCheckpoint(true); - dataStream.addSink(kafkaProducer).name(TableConnectorUtil.generateRuntimeName(this.getClass(), fieldNames)); + dataStream.addSink(kafkaProducer).name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); } } \ No newline at end of file diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index a6184161f..e2c7e3b70 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -74,9 +74,11 @@ public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener< @Override public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { KafkaSinkTableInfo kafka011SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; - this.topic = kafka011SinkTableInfo.getKafkaParam("topic"); + this.topic = kafka011SinkTableInfo.getTopic(); Properties props = new Properties(); + props.setProperty("bootstrap.servers", kafka011SinkTableInfo.getBootstrapServers()); + for (String key:kafka011SinkTableInfo.getKafkaParamKeys()) { props.setProperty(key, kafka011SinkTableInfo.getKafkaParam(key)); } @@ -97,17 +99,14 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { //this.serializationSchema = Optional.of(JsonRowSerializationSchema.class); if ("json".equalsIgnoreCase(kafka011SinkTableInfo.getSinkDataType())) { - this.serializationSchema = new JsonRowSerializationSchema(getOutputType()); - } else if ("csv".equalsIgnoreCase(kafka011SinkTableInfo.getSinkDataType())){ - this.serializationSchema = new TypeInformationSerializationSchema(TypeInformation.of(Row.class), - new CustomerCsvSerialization(kafka011SinkTableInfo.getFieldDelimiter(),fieldTypes)); + this.serializationSchema = new CustomerJsonRowSerializationSchema(getOutputType()); } return this; } @Override public void emitDataStream(DataStream dataStream) { - KafkaTableSinkBase kafkaTableSink = new Kafka011TableSink( + KafkaTableSinkBase kafkaTableSink = new CustomerKafka11JsonTableSink( schema, topic, properties, diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index f487df080..b9f84971a 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -39,18 +39,24 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map kafkaParam = new HashMap(); + private String topic; + + public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); } @@ -55,10 +63,28 @@ public Set getKafkaParamKeys(){ return kafkaParam.keySet(); } + + public String getBootstrapServers() { + return bootstrapServers; + } + + public void setBootstrapServers(String bootstrapServers) { + this.bootstrapServers = bootstrapServers; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + @Override public boolean check() { - Preconditions.checkNotNull(kafkaParam.get("bootstrap.servers"), "kafka of bootstrapServers is required"); - Preconditions.checkNotNull(kafkaParam.get("topic"), "kafka of topic is required"); + Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(topic, "kafka of topic is required"); //Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); return false; } diff --git a/kafka11/pom.xml b/kafka11/pom.xml index a9112cb49..90972fa82 100644 --- a/kafka11/pom.xml +++ b/kafka11/pom.xml @@ -14,7 +14,7 @@ kafka11-source - + kafka11-sink From 96ad99b38c6518585a6fbed020405395eb3416df Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 12 Aug 2019 20:30:56 +0800 Subject: [PATCH 338/470] kafka10 sink --- .../kafka/CustomerKafka10JsonTableSink.java | 41 ++++++++-------- .../flink/sql/sink/kafka/KafkaSink.java | 47 ++++++++++++++----- .../sink/kafka/table/KafkaSinkTableInfo.java | 17 ++++--- kafka10/pom.xml | 2 +- .../kafka/CustomerKafka11JsonTableSink.java | 2 +- 5 files changed, 68 insertions(+), 41 deletions(-) diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java index d2e6d4e12..01762079a 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java @@ -19,15 +19,14 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase; -import org.apache.flink.streaming.connectors.kafka.Kafka010JsonTableSink; -import org.apache.flink.streaming.connectors.kafka.KafkaJsonTableSink; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; +import org.apache.flink.streaming.connectors.kafka.Kafka010TableSink; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.util.TableConnectorUtil; +import org.apache.flink.table.utils.TableConnectorUtils; import org.apache.flink.types.Row; import java.util.Optional; @@ -40,38 +39,38 @@ * * @author maqi */ -public class CustomerKafka10JsonTableSink extends KafkaJsonTableSink { +public class CustomerKafka10JsonTableSink extends Kafka010TableSink { protected SerializationSchema schema; - public CustomerKafka10JsonTableSink(String topic, Properties properties, SerializationSchema schema) { - super(topic, properties, new FlinkFixedPartitioner<>()); - this.schema = schema; - } - - public CustomerKafka10JsonTableSink(String topic, Properties properties, FlinkKafkaPartitioner partitioner, SerializationSchema schema) { - super(topic, properties, partitioner); - this.schema = schema; + public CustomerKafka10JsonTableSink(TableSchema schema, + String topic, + Properties properties, + Optional> partitioner, + SerializationSchema serializationSchema) { + super(schema, topic, properties, partitioner, serializationSchema); + this.schema = serializationSchema; } - @Deprecated - public CustomerKafka10JsonTableSink(String topic, Properties properties, KafkaPartitioner partitioner, SerializationSchema schema) { - super(topic, properties, new FlinkKafkaDelegatePartitioner<>(partitioner)); - this.schema = schema; - } @Override - protected SinkFunction createKafkaProducer(String s, Properties properties, SerializationSchema serializationSchema, Optional> optional) { + protected FlinkKafkaProducerBase createKafkaProducer( + String topic, + Properties properties, + SerializationSchema serializationSchema, + Optional> partitioner) { return new CustomerFlinkKafkaProducer010(topic, serializationSchema, properties); } + @Override public void emitDataStream(DataStream dataStream) { SinkFunction kafkaProducer = createKafkaProducer(topic, properties, schema, partitioner); // always enable flush on checkpoint to achieve at-least-once if query runs with checkpointing enabled. //kafkaProducer.setFlushOnCheckpoint(true); - dataStream.addSink(kafkaProducer).name(TableConnectorUtil.generateRuntimeName(this.getClass(), fieldNames)); + dataStream.addSink(kafkaProducer).name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); + } } diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 4ab34c3ef..95fab91c3 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -26,10 +26,14 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; +import java.util.Optional; import java.util.Properties; /** * @@ -48,39 +52,60 @@ public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener[] fieldTypes; + /** The schema of the table. */ + private TableSchema schema; + + /** The Kafka topic to write to. */ protected String topic; + /** Properties for the Kafka producer. */ protected Properties properties; /** Serialization schema for encoding records to Kafka. */ protected SerializationSchema serializationSchema; + /** Partitioner to select Kafka partition for each item. */ + protected Optional> partitioner; @Override public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { - KafkaSinkTableInfo kafka10SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; - this.topic = kafka10SinkTableInfo.getTopic(); - this.fieldNames = kafka10SinkTableInfo.getFields(); - TypeInformation[] types = new TypeInformation[kafka10SinkTableInfo.getFields().length]; - for (int i = 0; i < kafka10SinkTableInfo.getFieldClasses().length; i++) { - types[i] = TypeInformation.of(kafka10SinkTableInfo.getFieldClasses()[i]); + KafkaSinkTableInfo kafka010SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; + this.topic = kafka010SinkTableInfo.getTopic(); + + Properties props = new Properties(); + props.setProperty("bootstrap.servers", kafka010SinkTableInfo.getBootstrapServers()); + + for (String key:kafka010SinkTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka010SinkTableInfo.getKafkaParam(key)); + } + this.properties = props; + this.partitioner = Optional.of(new FlinkFixedPartitioner<>()); + this.fieldNames = kafka010SinkTableInfo.getFields(); + TypeInformation[] types = new TypeInformation[kafka010SinkTableInfo.getFields().length]; + for(int i = 0; i< kafka010SinkTableInfo.getFieldClasses().length; i++){ + types[i] = TypeInformation.of(kafka010SinkTableInfo.getFieldClasses()[i]); } this.fieldTypes = types; - properties = new Properties(); - for (String key : kafka10SinkTableInfo.getKafkaParamKeys()) { - properties.setProperty(key, kafka10SinkTableInfo.getKafkaParam(key)); + TableSchema.Builder schemaBuilder = TableSchema.builder(); + for (int i=0;i dataStream) { KafkaTableSinkBase kafkaTableSink = new CustomerKafka10JsonTableSink( + schema, topic, properties, + partitioner, serializationSchema ); diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index 23ce28e5f..b4fd0a76c 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -19,7 +19,6 @@ package com.dtstack.flink.sql.sink.kafka.table; import com.dtstack.flink.sql.table.TargetTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; import java.util.HashMap; import java.util.Map; @@ -38,6 +37,10 @@ public class KafkaSinkTableInfo extends TargetTableInfo { //version private static final String CURR_TYPE = "kafka10"; + public KafkaSinkTableInfo() { + super.setType(CURR_TYPE); + } + public static final String BOOTSTRAPSERVERS_KEY = "bootstrapServers"; public static final String TOPIC_KEY = "topic"; @@ -48,9 +51,7 @@ public class KafkaSinkTableInfo extends TargetTableInfo { private String topic; - public KafkaSinkTableInfo() { - super.setType(CURR_TYPE); - } + public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); } @@ -80,16 +81,18 @@ public void setTopic(String topic) { this.topic = topic; } + @Override public boolean check() { - Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); - Preconditions.checkNotNull(topic, "kafka of topic is required"); + com.google.common.base.Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); + com.google.common.base.Preconditions.checkNotNull(topic, "kafka of topic is required"); + //Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); return false; } @Override public String getType() { -// return super.getType() + SOURCE_SUFFIX; return super.getType(); } + } diff --git a/kafka10/pom.xml b/kafka10/pom.xml index ba7d90de9..d8ad7370f 100644 --- a/kafka10/pom.xml +++ b/kafka10/pom.xml @@ -14,7 +14,7 @@ kafka10-source - + kafka10-sink diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java index 1a03da35e..89d1543a5 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka11JsonTableSink.java @@ -53,7 +53,7 @@ public CustomerKafka11JsonTableSink(TableSchema schema, } @Override - protected SinkFunction createKafkaProducer(String s, Properties properties, SerializationSchema serializationSchema, Optional> optional) { + protected SinkFunction createKafkaProducer(String topic, Properties properties, SerializationSchema serializationSchema, Optional> optional) { return new CustomerFlinkKafkaProducer011(topic, serializationSchema, properties); } From c8ce7e61c58d6da86536258320e420698aa5c9ee Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 12 Aug 2019 20:45:12 +0800 Subject: [PATCH 339/470] kafka09 sink --- .../kafka/CustomerKafka09JsonTableSink.java | 32 +++++++-------- .../flink/sql/sink/kafka/KafkaSink.java | 41 ++++++++++++++++--- .../sink/kafka/table/KafkaSinkTableInfo.java | 7 ++-- kafka09/pom.xml | 2 +- 4 files changed, 54 insertions(+), 28 deletions(-) diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka09JsonTableSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka09JsonTableSink.java index f7b7bcd1f..47ff5dab4 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka09JsonTableSink.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka09JsonTableSink.java @@ -20,12 +20,12 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.connectors.kafka.*; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase; +import org.apache.flink.streaming.connectors.kafka.Kafka09TableSink; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; -import org.apache.flink.table.util.TableConnectorUtil; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.utils.TableConnectorUtils; import org.apache.flink.types.Row; import java.util.Optional; @@ -38,30 +38,26 @@ * * @author maqi */ -public class CustomerKafka09JsonTableSink extends KafkaJsonTableSink { +public class CustomerKafka09JsonTableSink extends Kafka09TableSink { protected SerializationSchema schema; - public CustomerKafka09JsonTableSink(String topic, Properties properties, SerializationSchema schema) { - super(topic, properties, new FlinkFixedPartitioner<>()); - this.schema = schema; - } - public CustomerKafka09JsonTableSink(String topic, Properties properties, FlinkKafkaPartitioner partitioner, SerializationSchema schema) { - super(topic, properties, partitioner); - this.schema = schema; - } @Deprecated - public CustomerKafka09JsonTableSink(String topic, Properties properties, KafkaPartitioner partitioner, SerializationSchema schema) { - super(topic, properties, new FlinkKafkaDelegatePartitioner<>(partitioner)); - this.schema = schema; + public CustomerKafka09JsonTableSink(TableSchema schema, + String topic, + Properties properties, + Optional> partitioner, + SerializationSchema serializationSchema) { + super(schema, topic, properties, partitioner, serializationSchema); + this.schema = serializationSchema; } @Override - protected SinkFunction createKafkaProducer(String s, Properties properties, SerializationSchema serializationSchema, Optional> optional) { + protected FlinkKafkaProducerBase createKafkaProducer(String s, Properties properties, SerializationSchema serializationSchema, Optional> optional) { return new CustomerFlinkKafkaProducer09<>(topic, serializationSchema, properties); } @@ -70,6 +66,6 @@ public void emitDataStream(DataStream dataStream) { SinkFunction kafkaProducer = createKafkaProducer(topic, properties, schema, partitioner); // always enable flush on checkpoint to achieve at-least-once if query runs with checkpointing enabled. //kafkaProducer.setFlushOnCheckpoint(true); - dataStream.addSink(kafkaProducer).name(TableConnectorUtil.generateRuntimeName(this.getClass(), fieldNames)); + dataStream.addSink(kafkaProducer).name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); } } diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 70515e4ed..cc8db2250 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -26,9 +26,14 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.sinks.AppendStreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; + +import java.util.Optional; import java.util.Properties; /** @@ -44,39 +49,63 @@ public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener[] fieldTypes; + /** The schema of the table. */ + private TableSchema schema; + + /** The Kafka topic to write to. */ protected String topic; + /** Properties for the Kafka producer. */ protected Properties properties; /** Serialization schema for encoding records to Kafka. */ protected SerializationSchema serializationSchema; + /** Partitioner to select Kafka partition for each item. */ + protected Optional> partitioner; + @Override public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { + KafkaSinkTableInfo kafka09SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; this.topic = kafka09SinkTableInfo.getTopic(); + + Properties props = new Properties(); + props.setProperty("bootstrap.servers", kafka09SinkTableInfo.getBootstrapServers()); + + for (String key:kafka09SinkTableInfo.getKafkaParamKeys()) { + props.setProperty(key, kafka09SinkTableInfo.getKafkaParam(key)); + } + this.properties = props; + this.partitioner = Optional.of(new FlinkFixedPartitioner<>()); this.fieldNames = kafka09SinkTableInfo.getFields(); TypeInformation[] types = new TypeInformation[kafka09SinkTableInfo.getFields().length]; - for (int i = 0; i < kafka09SinkTableInfo.getFieldClasses().length; i++) { + for(int i = 0; i< kafka09SinkTableInfo.getFieldClasses().length; i++){ types[i] = TypeInformation.of(kafka09SinkTableInfo.getFieldClasses()[i]); } this.fieldTypes = types; - properties = new Properties(); - for (String key : kafka09SinkTableInfo.getKafkaParamKeys()) { - properties.setProperty(key, kafka09SinkTableInfo.getKafkaParam(key)); + TableSchema.Builder schemaBuilder = TableSchema.builder(); + for (int i=0;i dataStream) { KafkaTableSinkBase kafkaTableSink = new CustomerKafka09JsonTableSink( + schema, topic, properties, + partitioner, serializationSchema ); diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index 19d492ccd..d4b3e5b62 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -36,6 +36,10 @@ public class KafkaSinkTableInfo extends TargetTableInfo { //version private static final String CURR_TYPE = "kafka09"; + public KafkaSinkTableInfo() { + super.setType(CURR_TYPE); + } + public static final String BOOTSTRAPSERVERS_KEY = "bootstrapServers"; public static final String TOPIC_KEY = "topic"; @@ -46,9 +50,6 @@ public class KafkaSinkTableInfo extends TargetTableInfo { public Map kafkaParam = new HashMap(); - public KafkaSinkTableInfo() { - super.setType(CURR_TYPE); - } public void addKafkaParam(String key,String value){ kafkaParam.put(key,value); diff --git a/kafka09/pom.xml b/kafka09/pom.xml index 81aef2a96..86613de70 100644 --- a/kafka09/pom.xml +++ b/kafka09/pom.xml @@ -15,7 +15,7 @@ kafka09-source - + kafka09-sink From bcedd98987b7d97a08fdf3c1b3758ec90e11f7c9 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 13 Aug 2019 17:33:29 +0800 Subject: [PATCH 340/470] sidetable primary key convert --- .../com/dtstack/flink/sql/side/SideInfo.java | 2 +- .../dtstack/flink/sql/side/SideSqlExec.java | 24 ++++++++++++------- .../sql/side/oracle/OracleAsyncSideInfo.java | 2 +- .../sql/side/rdb/async/RdbAsyncSideInfo.java | 2 +- 4 files changed, 18 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java index 924ef53f2..063bfd2fd 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java @@ -89,7 +89,7 @@ public void parseSelectFields(JoinInfo joinInfo){ for( int i=0; i primaryKeys){ - - List conditionFields = getConditionFields(conditionNode, sideTableAlias); - if(CollectionUtils.isEqualCollection(conditionFields, primaryKeys)){ + private boolean checkJoinCondition(SqlNode conditionNode, String sideTableAlias, SideTableInfo sideTableInfo){ + List conditionFields = getConditionFields(conditionNode, sideTableAlias, sideTableInfo); + if(CollectionUtils.isEqualCollection(conditionFields, convertPrimaryAlias(sideTableInfo))){ return true; } - return false; } - public List getConditionFields(SqlNode conditionNode, String specifyTableName){ + private List convertPrimaryAlias(SideTableInfo sideTableInfo){ + List res = Lists.newArrayList(); + sideTableInfo.getPrimaryKeys().forEach(field -> { + res.add(sideTableInfo.getPhysicalFields().getOrDefault(field, field)); + }); + return res; + } + + public List getConditionFields(SqlNode conditionNode, String specifyTableName, SideTableInfo sideTableInfo){ List sqlNodeList = Lists.newArrayList(); ParseUtils.parseAnd(conditionNode, sqlNodeList); List conditionFields = Lists.newArrayList(); @@ -496,7 +502,7 @@ public List getConditionFields(SqlNode conditionNode, String specifyTabl }else{ throw new RuntimeException(String.format("side table:%s join condition is wrong", specifyTableName)); } - + tableCol = sideTableInfo.getPhysicalFields().getOrDefault(tableCol, tableCol); conditionFields.add(tableCol); } @@ -590,7 +596,7 @@ private void joinFun(Object pollObj, Map localTableCache, throw new RuntimeException("can't not find side table:" + joinInfo.getRightTableName()); } - if(!checkJoinCondition(joinInfo.getCondition(), joinInfo.getRightTableAlias(), sideTableInfo.getPrimaryKeys())){ + if(!checkJoinCondition(joinInfo.getCondition(), joinInfo.getRightTableAlias(), sideTableInfo)){ throw new RuntimeException("ON condition must contain all equal fields!!!"); } @@ -616,7 +622,7 @@ private void joinFun(Object pollObj, Map localTableCache, //join side table before keyby ===> Reducing the size of each dimension table cache of async if(sideTableInfo.isPartitionedJoin()){ - List leftJoinColList = getConditionFields(joinInfo.getCondition(), joinInfo.getLeftTableAlias()); + List leftJoinColList = getConditionFields(joinInfo.getCondition(), joinInfo.getLeftTableAlias(), sideTableInfo); String[] leftJoinColArr = new String[leftJoinColList.size()]; leftJoinColArr = leftJoinColList.toArray(leftJoinColArr); adaptStream = adaptStream.keyBy(leftJoinColArr); diff --git a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java index 22f6dadc7..61aa6dc14 100644 --- a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java +++ b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java @@ -56,7 +56,7 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { sqlCondition = "select ${selectField} from ${tableName} where "; for (int i = 0; i < equalFieldList.size(); i++) { - String equalField = equalFieldList.get(i); + String equalField = sideTableInfo.getPhysicalFields().getOrDefault(equalFieldList.get(i), equalFieldList.get(i)); sqlCondition += dealLowerFiled(equalField) + "=? "; if (i != equalFieldList.size() - 1) { diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java index c33916a71..c35ad55d5 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java @@ -66,7 +66,7 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { sqlCondition = "select ${selectField} from ${tableName} where "; for (int i = 0; i < equalFieldList.size(); i++) { - String equalField = equalFieldList.get(i); + String equalField = sideTableInfo.getPhysicalFields().getOrDefault(equalFieldList.get(i), equalFieldList.get(i)); sqlCondition += equalField + "=? "; if (i != equalFieldList.size() - 1) { From 268c1d3f0e6ab13721659f3e91836cf373cb29fb Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 13 Aug 2019 17:38:26 +0800 Subject: [PATCH 341/470] rdb batch num 100 --- .../src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java index 16d97e992..1c8cda15e 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java @@ -61,7 +61,7 @@ public abstract class RdbSink implements RetractStreamTableSink, Serializab protected String dbType; - protected int batchNum = 1; + protected int batchNum = 100; protected long batchWaitInterval = 10000; From 7b884c7cf2e498a7b121c61b66987ab2ab1ebbce Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Thu, 15 Aug 2019 10:12:15 +0800 Subject: [PATCH 342/470] rdb field mapping by name --- core/pom.xml | 6 +++ .../main/java/com/dtstack/flink/sql/Main.java | 38 ++++++++++++++++++- .../dtstack/flink/sql/side/SideSqlExec.java | 4 +- 3 files changed, 46 insertions(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index cbcc8eaae..08f10defb 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -17,6 +17,7 @@ UTF-8 core + 1.8.1 @@ -56,6 +57,11 @@ flink-streaming-scala_2.11 ${flink.version} + + org.apache.flink + flink-table-planner_2.11 + ${flink.table.planner} + diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 9ddb45682..5b4b051e6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -35,6 +35,7 @@ import com.dtstack.flink.sql.util.FlinkUtil; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.calcite.config.Lex; +import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlNode; import org.apache.commons.cli.CommandLine; @@ -60,7 +61,12 @@ import org.apache.flink.streaming.api.environment.StreamContextEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.calcite.FlinkPlannerImpl; +import org.apache.flink.table.plan.logical.LogicalRelNode; +import org.apache.flink.table.plan.schema.TableSinkTable; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; import org.slf4j.Logger; @@ -204,7 +210,7 @@ public static void main(String[] args) throws Exception { //sql-dimensional table contains the dimension table of execution sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache); }else{ - tableEnv.sqlUpdate(result.getExecSql()); + sqlUpdate(tableEnv, result.getExecSql()); if(LOG.isInfoEnabled()){ LOG.info("exec sql: " + result.getExecSql()); } @@ -222,6 +228,36 @@ public static void main(String[] args) throws Exception { env.execute(name); } + public static void sqlUpdate(StreamTableEnvironment tableEnv, String stmt) { + + FlinkPlannerImpl planner = new FlinkPlannerImpl(tableEnv.getFrameworkConfig(), tableEnv.getPlanner(), tableEnv.getTypeFactory()); + SqlNode insert = planner.parse(stmt); + + if (!(insert instanceof SqlInsert)) { + throw new TableException( + "Unsupported SQL query! sqlUpdate() only accepts SQL statements of type INSERT."); + } + SqlNode query = ((SqlInsert) insert).getSource(); + + SqlNode validatedQuery = planner.validate(query); + + Table queryResult = new Table(tableEnv, new LogicalRelNode(planner.rel(validatedQuery).rel)); + String targetTableName = ((SqlIdentifier) ((SqlInsert) insert).getTargetTable()).names.get(0); + + try { + Method method = TableEnvironment.class.getDeclaredMethod("getTable", String.class); + method.setAccessible(true); + + TableSinkTable targetTable = (TableSinkTable) method.invoke(tableEnv, targetTableName); + String[] fieldNames = targetTable.tableSink().getFieldNames(); + Table newTable = queryResult.select(String.join(",", fieldNames)); + // insert query result into sink table + tableEnv.insertInto(newTable, targetTableName, tableEnv.queryConfig()); + } catch (Exception e) { + e.printStackTrace(); + } + } + /** * This part is just to add classpath for the jar when reading remote execution, and will not submit jar from a local * @param env diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 18fb4d2d8..31ee93663 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.side; +import com.dtstack.flink.sql.Main; import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.parser.CreateTmpTableParser; import com.dtstack.flink.sql.side.operator.SideAsyncOperator; @@ -105,7 +106,8 @@ public void exec(String sql, Map sideTableMap, StreamTabl } if(pollSqlNode.getKind() == INSERT){ - tableEnv.sqlUpdate(pollSqlNode.toString()); +// tableEnv.sqlUpdate(pollSqlNode.toString()); + Main.sqlUpdate(tableEnv, pollSqlNode.toString()); if(LOG.isInfoEnabled()){ LOG.info("exec sql: " + pollSqlNode.toString()); } From d51125a7dc12c4eb011280182fa6c9e10c76b233 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Thu, 15 Aug 2019 12:50:53 +0800 Subject: [PATCH 343/470] mapping by name when insert into sink table --- .../main/java/com/dtstack/flink/sql/Main.java | 45 +++------------- .../dtstack/flink/sql/exec/FlinkSQLExec.java | 52 +++++++++++++++++++ .../dtstack/flink/sql/side/SideSqlExec.java | 4 +- 3 files changed, 61 insertions(+), 40 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/exec/FlinkSQLExec.java diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 5b4b051e6..d55f0ceea 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -22,7 +22,12 @@ import com.dtstack.flink.sql.classloader.DtClassLoader; import com.dtstack.flink.sql.enums.ECacheType; -import com.dtstack.flink.sql.parser.*; +import com.dtstack.flink.sql.exec.FlinkSQLExec; +import com.dtstack.flink.sql.parser.CreateFuncParser; +import com.dtstack.flink.sql.parser.CreateTmpTableParser; +import com.dtstack.flink.sql.parser.InsertSqlParser; +import com.dtstack.flink.sql.parser.SqlParser; +import com.dtstack.flink.sql.parser.SqlTree; import com.dtstack.flink.sql.side.SideSqlExec; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.table.SourceTableInfo; @@ -35,7 +40,6 @@ import com.dtstack.flink.sql.util.FlinkUtil; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.calcite.config.Lex; -import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlNode; import org.apache.commons.cli.CommandLine; @@ -61,12 +65,7 @@ import org.apache.flink.streaming.api.environment.StreamContextEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.java.StreamTableEnvironment; -import org.apache.flink.table.calcite.FlinkPlannerImpl; -import org.apache.flink.table.plan.logical.LogicalRelNode; -import org.apache.flink.table.plan.schema.TableSinkTable; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; import org.slf4j.Logger; @@ -210,7 +209,7 @@ public static void main(String[] args) throws Exception { //sql-dimensional table contains the dimension table of execution sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache); }else{ - sqlUpdate(tableEnv, result.getExecSql()); + FlinkSQLExec.sqlUpdate(tableEnv, result.getExecSql()); if(LOG.isInfoEnabled()){ LOG.info("exec sql: " + result.getExecSql()); } @@ -228,36 +227,6 @@ public static void main(String[] args) throws Exception { env.execute(name); } - public static void sqlUpdate(StreamTableEnvironment tableEnv, String stmt) { - - FlinkPlannerImpl planner = new FlinkPlannerImpl(tableEnv.getFrameworkConfig(), tableEnv.getPlanner(), tableEnv.getTypeFactory()); - SqlNode insert = planner.parse(stmt); - - if (!(insert instanceof SqlInsert)) { - throw new TableException( - "Unsupported SQL query! sqlUpdate() only accepts SQL statements of type INSERT."); - } - SqlNode query = ((SqlInsert) insert).getSource(); - - SqlNode validatedQuery = planner.validate(query); - - Table queryResult = new Table(tableEnv, new LogicalRelNode(planner.rel(validatedQuery).rel)); - String targetTableName = ((SqlIdentifier) ((SqlInsert) insert).getTargetTable()).names.get(0); - - try { - Method method = TableEnvironment.class.getDeclaredMethod("getTable", String.class); - method.setAccessible(true); - - TableSinkTable targetTable = (TableSinkTable) method.invoke(tableEnv, targetTableName); - String[] fieldNames = targetTable.tableSink().getFieldNames(); - Table newTable = queryResult.select(String.join(",", fieldNames)); - // insert query result into sink table - tableEnv.insertInto(newTable, targetTableName, tableEnv.queryConfig()); - } catch (Exception e) { - e.printStackTrace(); - } - } - /** * This part is just to add classpath for the jar when reading remote execution, and will not submit jar from a local * @param env diff --git a/core/src/main/java/com/dtstack/flink/sql/exec/FlinkSQLExec.java b/core/src/main/java/com/dtstack/flink/sql/exec/FlinkSQLExec.java new file mode 100644 index 000000000..364106c69 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/exec/FlinkSQLExec.java @@ -0,0 +1,52 @@ +package com.dtstack.flink.sql.exec; + +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlInsert; +import org.apache.calcite.sql.SqlNode; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.calcite.FlinkPlannerImpl; +import org.apache.flink.table.plan.logical.LogicalRelNode; +import org.apache.flink.table.plan.schema.TableSinkTable; + +import java.lang.reflect.Method; + +/** + * @description: mapping by name when insert into sink table + * @author: maqi + * @create: 2019/08/15 11:09 + */ +public class FlinkSQLExec { + + public static void sqlUpdate(StreamTableEnvironment tableEnv, String stmt) throws Exception { + + FlinkPlannerImpl planner = new FlinkPlannerImpl(tableEnv.getFrameworkConfig(), tableEnv.getPlanner(), tableEnv.getTypeFactory()); + SqlNode insert = planner.parse(stmt); + + if (!(insert instanceof SqlInsert)) { + throw new TableException( + "Unsupported SQL query! sqlUpdate() only accepts SQL statements of type INSERT."); + } + SqlNode query = ((SqlInsert) insert).getSource(); + + SqlNode validatedQuery = planner.validate(query); + + Table queryResult = new Table(tableEnv, new LogicalRelNode(planner.rel(validatedQuery).rel)); + String targetTableName = ((SqlIdentifier) ((SqlInsert) insert).getTargetTable()).names.get(0); + + try { + Method method = TableEnvironment.class.getDeclaredMethod("getTable", String.class); + method.setAccessible(true); + + TableSinkTable targetTable = (TableSinkTable) method.invoke(tableEnv, targetTableName); + String[] fieldNames = targetTable.tableSink().getFieldNames(); + Table newTable = queryResult.select(String.join(",", fieldNames)); + // insert query result into sink table + tableEnv.insertInto(newTable, targetTableName, tableEnv.queryConfig()); + } catch (Exception e) { + throw e; + } + } +} \ No newline at end of file diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 31ee93663..f6de92079 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -22,6 +22,7 @@ import com.dtstack.flink.sql.Main; import com.dtstack.flink.sql.enums.ECacheType; +import com.dtstack.flink.sql.exec.FlinkSQLExec; import com.dtstack.flink.sql.parser.CreateTmpTableParser; import com.dtstack.flink.sql.side.operator.SideAsyncOperator; import com.dtstack.flink.sql.side.operator.SideWithAllCacheOperator; @@ -106,8 +107,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl } if(pollSqlNode.getKind() == INSERT){ -// tableEnv.sqlUpdate(pollSqlNode.toString()); - Main.sqlUpdate(tableEnv, pollSqlNode.toString()); + FlinkSQLExec.sqlUpdate(tableEnv, pollSqlNode.toString()); if(LOG.isInfoEnabled()){ LOG.info("exec sql: " + pollSqlNode.toString()); } From 82875da472f7e1e559dbf778ba5309c589586b74 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Thu, 15 Aug 2019 14:26:14 +0800 Subject: [PATCH 344/470] remove maven dependency of flink-table-planner --- core/pom.xml | 6 ------ 1 file changed, 6 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 08f10defb..cbcc8eaae 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -17,7 +17,6 @@ UTF-8 core - 1.8.1 @@ -57,11 +56,6 @@ flink-streaming-scala_2.11 ${flink.version} - - org.apache.flink - flink-table-planner_2.11 - ${flink.table.planner} - From 0169fa9d3842219ac9019efa6a8cbb05e04c64ab Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 16 Aug 2019 12:48:00 +0800 Subject: [PATCH 345/470] convert field name --- .../dtstack/flink/sql/exec/FlinkSQLExec.java | 25 ++-- .../dtstack/flink/sql/side/SideSqlExec.java | 135 +++++++++++++----- 2 files changed, 118 insertions(+), 42 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/exec/FlinkSQLExec.java b/core/src/main/java/com/dtstack/flink/sql/exec/FlinkSQLExec.java index 364106c69..d0191aec3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/exec/FlinkSQLExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/exec/FlinkSQLExec.java @@ -6,6 +6,7 @@ import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.table.calcite.FlinkPlannerImpl; import org.apache.flink.table.plan.logical.LogicalRelNode; @@ -36,17 +37,23 @@ public static void sqlUpdate(StreamTableEnvironment tableEnv, String stmt) throw Table queryResult = new Table(tableEnv, new LogicalRelNode(planner.rel(validatedQuery).rel)); String targetTableName = ((SqlIdentifier) ((SqlInsert) insert).getTargetTable()).names.get(0); + Method method = TableEnvironment.class.getDeclaredMethod("getTable", String.class); + method.setAccessible(true); + + TableSinkTable targetTable = (TableSinkTable) method.invoke(tableEnv, targetTableName); + String[] fieldNames = targetTable.tableSink().getFieldNames(); + + Table newTable = null; + try { - Method method = TableEnvironment.class.getDeclaredMethod("getTable", String.class); - method.setAccessible(true); - - TableSinkTable targetTable = (TableSinkTable) method.invoke(tableEnv, targetTableName); - String[] fieldNames = targetTable.tableSink().getFieldNames(); - Table newTable = queryResult.select(String.join(",", fieldNames)); - // insert query result into sink table - tableEnv.insertInto(newTable, targetTableName, tableEnv.queryConfig()); + newTable = queryResult.select(String.join(",", fieldNames)); } catch (Exception e) { - throw e; + throw new ValidationException( + "Field name of query result and registered TableSink "+targetTableName +" do not match.\n" + + "Query result schema: " + String.join(",", queryResult.getSchema().getColumnNames()) + "\n" + + "TableSink schema: " + String.join(",", fieldNames)); } + + tableEnv.insertInto(newTable, targetTableName, tableEnv.queryConfig()); } } \ No newline at end of file diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 1a002ca9d..b260a2889 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -17,10 +17,8 @@ */ - package com.dtstack.flink.sql.side; -import com.dtstack.flink.sql.Main; import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.exec.FlinkSQLExec; import com.dtstack.flink.sql.parser.CreateTmpTableParser; @@ -28,6 +26,7 @@ import com.dtstack.flink.sql.side.operator.SideWithAllCacheOperator; import com.dtstack.flink.sql.util.ClassUtil; import com.dtstack.flink.sql.util.ParseUtils; +import org.apache.calcite.sql.SqlAsOperator; import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlDataTypeSpec; import org.apache.calcite.sql.SqlIdentifier; @@ -37,6 +36,7 @@ import org.apache.calcite.sql.SqlLiteral; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.fun.SqlCase; import org.apache.calcite.sql.parser.SqlParseException; @@ -101,8 +101,11 @@ public void exec(String sql, Map sideTableMap, StreamTabl if(preIsSideJoin){ preIsSideJoin = false; + List fieldNames = null; for(FieldReplaceInfo replaceInfo : replaceInfoList){ - replaceFieldName(pollSqlNode, replaceInfo.getMappingTable(), replaceInfo.getTargetTableName(), replaceInfo.getTargetTableAlias()); + fieldNames = Lists.newArrayList(); + replaceFieldName(pollSqlNode, replaceInfo.getMappingTable(), replaceInfo.getTargetTableName(), replaceInfo.getTargetTableAlias(), fieldNames); + dealMidConvertField(pollSqlNode, fieldNames); } } @@ -126,6 +129,66 @@ public void exec(String sql, Map sideTableMap, StreamTabl } + + private void dealMidConvertField(SqlNode pollSqlNode, List field) { + SqlKind sqlKind = pollSqlNode.getKind(); + switch (sqlKind) { + case INSERT: + SqlNode source = ((SqlInsert) pollSqlNode).getSource(); + dealMidConvertField(source, field); + break; + + case AS: + dealMidConvertField(((SqlBasicCall) pollSqlNode).getOperands()[0], field); + break; + + case SELECT: + + SqlNodeList selectList = ((SqlSelect) pollSqlNode).getSelectList(); + + selectList.getList().forEach(node -> { + if (node.getKind() == IDENTIFIER) { + SqlIdentifier sqlIdentifier = (SqlIdentifier) node; + if (sqlIdentifier.names.size() == 1) { + return; + } + String name = sqlIdentifier.names.get(1); + if (!name.endsWith("0")) { + field.add(name); + } + + } + }); + // convert + for (int i = 0; i < selectList.getList().size(); i++) { + SqlNode node = selectList.get(i); + if (node.getKind() == IDENTIFIER) { + SqlIdentifier sqlIdentifier = (SqlIdentifier) node; + if (sqlIdentifier.names.size() == 1) { + return; + } + + String name = sqlIdentifier.names.get(1); + if (name.endsWith("0") && !field.contains(name)) { + SqlOperator operator = new SqlAsOperator(); + SqlParserPos sqlParserPos = new SqlParserPos(0, 0); + + SqlIdentifier sqlIdentifierAlias = new SqlIdentifier(name.substring(0, name.length() - 1), null, sqlParserPos); + SqlNode[] sqlNodes = new SqlNode[2]; + sqlNodes[0] = sqlIdentifier; + sqlNodes[1] = sqlIdentifierAlias; + SqlBasicCall sqlBasicCall = new SqlBasicCall(operator, sqlNodes, sqlParserPos); + + selectList.set(i, sqlBasicCall); + } + + } + } + break; + } + } + + public AliasInfo parseASNode(SqlNode sqlNode) throws SqlParseException { SqlKind sqlKind = sqlNode.getKind(); if(sqlKind != AS){ @@ -164,16 +227,16 @@ public RowTypeInfo buildOutRowTypeInfo(List sideJoinFieldInfo, HashBa } //需要考虑更多的情况 - private void replaceFieldName(SqlNode sqlNode, HashBasedTable mappingTable, String targetTableName, String tableAlias) { + private void replaceFieldName(SqlNode sqlNode, HashBasedTable mappingTable, String targetTableName, String tableAlias, List fieldNames) { SqlKind sqlKind = sqlNode.getKind(); switch (sqlKind) { case INSERT: SqlNode sqlSource = ((SqlInsert) sqlNode).getSource(); - replaceFieldName(sqlSource, mappingTable, targetTableName, tableAlias); + replaceFieldName(sqlSource, mappingTable, targetTableName, tableAlias, fieldNames); break; case AS: - SqlNode asNode = ((SqlBasicCall)sqlNode).getOperands()[0]; - replaceFieldName(asNode, mappingTable, targetTableName, tableAlias); + SqlNode asNode = ((SqlBasicCall) sqlNode).getOperands()[0]; + replaceFieldName(asNode, mappingTable, targetTableName, tableAlias, fieldNames); break; case SELECT: SqlSelect sqlSelect = (SqlSelect) filterNodeWithTargetName(sqlNode, targetTableName); @@ -202,7 +265,7 @@ private void replaceFieldName(SqlNode sqlNode, HashBasedTable mappingTable, String tableAlias){ + private SqlNode replaceNodeInfo(SqlNode groupNode, HashBasedTable mappingTable, String tableAlias, List fieldNames){ if(groupNode.getKind() == IDENTIFIER){ SqlIdentifier sqlIdentifier = (SqlIdentifier) groupNode; String mappingFieldName = mappingTable.get(sqlIdentifier.getComponent(0).getSimple(), sqlIdentifier.getComponent(1).getSimple()); @@ -257,7 +320,7 @@ private SqlNode replaceNodeInfo(SqlNode groupNode, HashBasedTable replaceSelectStarFieldName(SqlNode selectNode, HashBasedTa } } - private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable mappingTable, String tableAlias){ - if(selectNode.getKind() == AS){ - SqlNode leftNode = ((SqlBasicCall)selectNode).getOperands()[0]; - SqlNode replaceNode = replaceSelectFieldName(leftNode, mappingTable, tableAlias); - if(replaceNode != null){ - ((SqlBasicCall)selectNode).getOperands()[0] = replaceNode; + private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable mappingTable, String tableAlias, List fieldNames) { + if (selectNode.getKind() == AS) { + SqlNode leftNode = ((SqlBasicCall) selectNode).getOperands()[0]; + SqlNode rightNode = ((SqlBasicCall) selectNode).getOperands()[1]; + fieldNames.add(rightNode.toString()); + SqlNode replaceNode = replaceSelectFieldName(leftNode, mappingTable, tableAlias, fieldNames); + if (replaceNode != null) { + ((SqlBasicCall) selectNode).getOperands()[0] = replaceNode; } return selectNode; @@ -419,7 +484,7 @@ private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable conditionFields = getConditionFields(conditionNode, sideTableAlias, sideTableInfo); if(CollectionUtils.isEqualCollection(conditionFields, convertPrimaryAlias(sideTableInfo))){ return true; @@ -473,7 +539,7 @@ private boolean checkJoinCondition(SqlNode conditionNode, String sideTableAlias, return false; } - private List convertPrimaryAlias(SideTableInfo sideTableInfo){ + private List convertPrimaryAlias(SideTableInfo sideTableInfo) { List res = Lists.newArrayList(); sideTableInfo.getPrimaryKeys().forEach(field -> { res.add(sideTableInfo.getPhysicalFields().getOrDefault(field, field)); @@ -535,8 +601,11 @@ public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, if(preIsSideJoin){ preIsSideJoin = false; - for(FieldReplaceInfo replaceInfo : replaceInfoList){ - replaceFieldName(pollSqlNode, replaceInfo.getMappingTable(), replaceInfo.getTargetTableName(), replaceInfo.getTargetTableAlias()); + List fieldNames = null; + for (FieldReplaceInfo replaceInfo : replaceInfoList) { + fieldNames = Lists.newArrayList(); + replaceFieldName(pollSqlNode, replaceInfo.getMappingTable(), replaceInfo.getTargetTableName(), replaceInfo.getTargetTableAlias(), fieldNames); + dealMidConvertField(pollSqlNode, fieldNames); } } @@ -572,6 +641,7 @@ public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, } } } + private void joinFun(Object pollObj, Map localTableCache, Map sideTableMap, StreamTableEnvironment tableEnv, List replaceInfoList) throws Exception{ @@ -655,12 +725,11 @@ private void joinFun(Object pollObj, Map localTableCache, } } - private boolean checkFieldsInfo(CreateTmpTableParser.SqlParserResult result, Table table){ + private boolean checkFieldsInfo(CreateTmpTableParser.SqlParserResult result, Table table) { List fieldNames = new LinkedList<>(); String fieldsInfo = result.getFieldsInfoStr(); String[] fields = fieldsInfo.split(","); - for (int i=0; i < fields.length; i++) - { + for (int i = 0; i < fields.length; i++) { String[] filed = fields[i].split("\\s"); if (filed.length < 2 || fields.length != table.getSchema().getColumnNames().length){ return false; From ea26fdb49c3b86f857524c34ca9cbd28883f3fcf Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 16 Aug 2019 20:24:46 +0800 Subject: [PATCH 346/470] add as operate --- .../dtstack/flink/sql/side/SideSqlExec.java | 55 +++++++++---------- .../rdb/format/RetractJDBCOutputFormat.java | 20 +++---- 2 files changed, 37 insertions(+), 38 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index b260a2889..e420d4545 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -104,8 +104,8 @@ public void exec(String sql, Map sideTableMap, StreamTabl List fieldNames = null; for(FieldReplaceInfo replaceInfo : replaceInfoList){ fieldNames = Lists.newArrayList(); - replaceFieldName(pollSqlNode, replaceInfo.getMappingTable(), replaceInfo.getTargetTableName(), replaceInfo.getTargetTableAlias(), fieldNames); - dealMidConvertField(pollSqlNode, fieldNames); + replaceFieldName(pollSqlNode, replaceInfo.getMappingTable(), replaceInfo.getTargetTableName(), replaceInfo.getTargetTableAlias()); + addAliasForFiledNode(pollSqlNode, fieldNames, replaceInfo.getMappingTable()); } } @@ -130,16 +130,16 @@ public void exec(String sql, Map sideTableMap, StreamTabl } - private void dealMidConvertField(SqlNode pollSqlNode, List field) { + private void addAliasForFiledNode(SqlNode pollSqlNode, List fieldList, HashBasedTable mappingTable) { SqlKind sqlKind = pollSqlNode.getKind(); switch (sqlKind) { case INSERT: SqlNode source = ((SqlInsert) pollSqlNode).getSource(); - dealMidConvertField(source, field); + addAliasForFiledNode(source, fieldList, mappingTable); break; case AS: - dealMidConvertField(((SqlBasicCall) pollSqlNode).getOperands()[0], field); + addAliasForFiledNode(((SqlBasicCall) pollSqlNode).getOperands()[0], fieldList, mappingTable); break; case SELECT: @@ -152,14 +152,15 @@ private void dealMidConvertField(SqlNode pollSqlNode, List field) { if (sqlIdentifier.names.size() == 1) { return; } - String name = sqlIdentifier.names.get(1); - if (!name.endsWith("0")) { - field.add(name); + + String filedName = sqlIdentifier.names.get(1); + if (!filedName.endsWith("0") ) { + fieldList.add(filedName); } } }); - // convert + for (int i = 0; i < selectList.getList().size(); i++) { SqlNode node = selectList.get(i); if (node.getKind() == IDENTIFIER) { @@ -169,7 +170,7 @@ private void dealMidConvertField(SqlNode pollSqlNode, List field) { } String name = sqlIdentifier.names.get(1); - if (name.endsWith("0") && !field.contains(name)) { + if (name.endsWith("0") && !fieldList.contains(name.substring(0, name.length() - 1)) && !mappingTable.columnMap().containsKey(name)) { SqlOperator operator = new SqlAsOperator(); SqlParserPos sqlParserPos = new SqlParserPos(0, 0); @@ -227,16 +228,16 @@ public RowTypeInfo buildOutRowTypeInfo(List sideJoinFieldInfo, HashBa } //需要考虑更多的情况 - private void replaceFieldName(SqlNode sqlNode, HashBasedTable mappingTable, String targetTableName, String tableAlias, List fieldNames) { + private void replaceFieldName(SqlNode sqlNode, HashBasedTable mappingTable, String targetTableName, String tableAlias) { SqlKind sqlKind = sqlNode.getKind(); switch (sqlKind) { case INSERT: SqlNode sqlSource = ((SqlInsert) sqlNode).getSource(); - replaceFieldName(sqlSource, mappingTable, targetTableName, tableAlias, fieldNames); + replaceFieldName(sqlSource, mappingTable, targetTableName, tableAlias); break; case AS: SqlNode asNode = ((SqlBasicCall) sqlNode).getOperands()[0]; - replaceFieldName(asNode, mappingTable, targetTableName, tableAlias, fieldNames); + replaceFieldName(asNode, mappingTable, targetTableName, tableAlias); break; case SELECT: SqlSelect sqlSelect = (SqlSelect) filterNodeWithTargetName(sqlNode, targetTableName); @@ -265,7 +266,7 @@ private void replaceFieldName(SqlNode sqlNode, HashBasedTable mappingTable, String tableAlias, List fieldNames){ + private SqlNode replaceNodeInfo(SqlNode groupNode, HashBasedTable mappingTable, String tableAlias){ if(groupNode.getKind() == IDENTIFIER){ SqlIdentifier sqlIdentifier = (SqlIdentifier) groupNode; String mappingFieldName = mappingTable.get(sqlIdentifier.getComponent(0).getSimple(), sqlIdentifier.getComponent(1).getSimple()); @@ -320,7 +321,7 @@ private SqlNode replaceNodeInfo(SqlNode groupNode, HashBasedTable replaceSelectStarFieldName(SqlNode selectNode, HashBasedTa } } - private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable mappingTable, String tableAlias, List fieldNames) { + private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable mappingTable, String tableAlias) { if (selectNode.getKind() == AS) { SqlNode leftNode = ((SqlBasicCall) selectNode).getOperands()[0]; - SqlNode rightNode = ((SqlBasicCall) selectNode).getOperands()[1]; - fieldNames.add(rightNode.toString()); - SqlNode replaceNode = replaceSelectFieldName(leftNode, mappingTable, tableAlias, fieldNames); + SqlNode replaceNode = replaceSelectFieldName(leftNode, mappingTable, tableAlias); if (replaceNode != null) { ((SqlBasicCall) selectNode).getOperands()[0] = replaceNode; } @@ -484,7 +483,7 @@ private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable fieldNames = null; for (FieldReplaceInfo replaceInfo : replaceInfoList) { fieldNames = Lists.newArrayList(); - replaceFieldName(pollSqlNode, replaceInfo.getMappingTable(), replaceInfo.getTargetTableName(), replaceInfo.getTargetTableAlias(), fieldNames); - dealMidConvertField(pollSqlNode, fieldNames); + replaceFieldName(pollSqlNode, replaceInfo.getMappingTable(), replaceInfo.getTargetTableName(), replaceInfo.getTargetTableAlias()); + addAliasForFiledNode(pollSqlNode, fieldNames, replaceInfo.getMappingTable()); } } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 14cfad10a..ae46a40ad 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -61,7 +61,7 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { // trigger preparedStatement execute batch interval private long batchWaitInterval = 10000l; // PreparedStatement execute batch num - private int batchNum = 1; + private int batchNum = 100; private String insertQuery; public int[] typesArray; @@ -97,6 +97,15 @@ public void open(int taskNumber, int numTasks) throws IOException { establishConnection(); initMetric(); + if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { + if (isReplaceInsertQuery()) { + insertQuery = dbSink.buildUpdateSql(tableName, Arrays.asList(dbSink.getFieldNames()), realIndexes, fullField); + } + upload = dbConn.prepareStatement(insertQuery); + } else { + throw new SQLException("Table " + tableName + " doesn't exist"); + } + if (batchWaitInterval > 0) { LOG.info("open batch wait interval scheduled, interval is {} ms", batchWaitInterval); @@ -107,15 +116,6 @@ public void open(int taskNumber, int numTasks) throws IOException { } - if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { - if (isReplaceInsertQuery()) { - insertQuery = dbSink.buildUpdateSql(tableName, Arrays.asList(dbSink.getFieldNames()), realIndexes, fullField); - } - upload = dbConn.prepareStatement(insertQuery); - } else { - throw new SQLException("Table " + tableName + " doesn't exist"); - } - } catch (SQLException sqe) { throw new IllegalArgumentException("open() failed.", sqe); } catch (ClassNotFoundException cnfe) { From 98bbfa23fd0df3828bacb3eb0dafc9599de3bc50 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 16 Aug 2019 22:03:42 +0800 Subject: [PATCH 347/470] mysql log --- .../flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 14cfad10a..9cdaeae6e 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -40,6 +40,7 @@ import java.util.concurrent.atomic.AtomicInteger; import com.dtstack.flink.sql.sink.MetricOutputFormat; +import sun.rmi.runtime.Log; /** * OutputFormat to write tuples into a database. @@ -117,8 +118,10 @@ public void open(int taskNumber, int numTasks) throws IOException { } } catch (SQLException sqe) { + LOG.error("", sqe); throw new IllegalArgumentException("open() failed.", sqe); } catch (ClassNotFoundException cnfe) { + LOG.error("", cnfe); throw new IllegalArgumentException("JDBC driver class not found.", cnfe); } } @@ -271,6 +274,7 @@ private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLExce private synchronized void submitExecuteBatch() { try { + LOG.info("submitExecuteBatch start......"); this.upload.executeBatch(); this.batchCount.set(0); } catch (SQLException e) { From 3f4d6534d31e5c33e40485e26944810cff863160 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 16 Aug 2019 22:10:18 +0800 Subject: [PATCH 348/470] mysql log --- .../rdb/format/RetractJDBCOutputFormat.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 9cdaeae6e..6e737aff2 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -98,6 +98,15 @@ public void open(int taskNumber, int numTasks) throws IOException { establishConnection(); initMetric(); + if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { + if (isReplaceInsertQuery()) { + insertQuery = dbSink.buildUpdateSql(tableName, Arrays.asList(dbSink.getFieldNames()), realIndexes, fullField); + } + upload = dbConn.prepareStatement(insertQuery); + } else { + throw new SQLException("Table " + tableName + " doesn't exist"); + } + if (batchWaitInterval > 0) { LOG.info("open batch wait interval scheduled, interval is {} ms", batchWaitInterval); @@ -108,15 +117,6 @@ public void open(int taskNumber, int numTasks) throws IOException { } - if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { - if (isReplaceInsertQuery()) { - insertQuery = dbSink.buildUpdateSql(tableName, Arrays.asList(dbSink.getFieldNames()), realIndexes, fullField); - } - upload = dbConn.prepareStatement(insertQuery); - } else { - throw new SQLException("Table " + tableName + " doesn't exist"); - } - } catch (SQLException sqe) { LOG.error("", sqe); throw new IllegalArgumentException("open() failed.", sqe); From b93a2656b97644c80758d2bfa667e4c3a1946b33 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Sat, 17 Aug 2019 12:29:40 +0800 Subject: [PATCH 349/470] deal real field name --- .../dtstack/flink/sql/side/SideSqlExec.java | 21 ++++++++++--------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index e420d4545..ac727ec05 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -105,7 +105,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl for(FieldReplaceInfo replaceInfo : replaceInfoList){ fieldNames = Lists.newArrayList(); replaceFieldName(pollSqlNode, replaceInfo.getMappingTable(), replaceInfo.getTargetTableName(), replaceInfo.getTargetTableAlias()); - addAliasForFiledNode(pollSqlNode, fieldNames, replaceInfo.getMappingTable()); + addAliasForFieldNode(pollSqlNode, fieldNames, replaceInfo.getMappingTable()); } } @@ -130,16 +130,16 @@ public void exec(String sql, Map sideTableMap, StreamTabl } - private void addAliasForFiledNode(SqlNode pollSqlNode, List fieldList, HashBasedTable mappingTable) { + private void addAliasForFieldNode(SqlNode pollSqlNode, List fieldList, HashBasedTable mappingTable) { SqlKind sqlKind = pollSqlNode.getKind(); switch (sqlKind) { case INSERT: SqlNode source = ((SqlInsert) pollSqlNode).getSource(); - addAliasForFiledNode(source, fieldList, mappingTable); + addAliasForFieldNode(source, fieldList, mappingTable); break; case AS: - addAliasForFiledNode(((SqlBasicCall) pollSqlNode).getOperands()[0], fieldList, mappingTable); + addAliasForFieldNode(((SqlBasicCall) pollSqlNode).getOperands()[0], fieldList, mappingTable); break; case SELECT: @@ -152,10 +152,10 @@ private void addAliasForFiledNode(SqlNode pollSqlNode, List fieldList, H if (sqlIdentifier.names.size() == 1) { return; } - - String filedName = sqlIdentifier.names.get(1); - if (!filedName.endsWith("0") ) { - fieldList.add(filedName); + // save real field + String fieldName = sqlIdentifier.names.get(1); + if (!fieldName.endsWith("0") || fieldName.endsWith("0") && mappingTable.columnMap().containsKey(fieldName)) { + fieldList.add(fieldName); } } @@ -170,7 +170,8 @@ private void addAliasForFiledNode(SqlNode pollSqlNode, List fieldList, H } String name = sqlIdentifier.names.get(1); - if (name.endsWith("0") && !fieldList.contains(name.substring(0, name.length() - 1)) && !mappingTable.columnMap().containsKey(name)) { + // avoid real field pv0 convert pv + if (name.endsWith("0") && !fieldList.contains(name) && !fieldList.contains(name.substring(0, name.length() - 1))) { SqlOperator operator = new SqlAsOperator(); SqlParserPos sqlParserPos = new SqlParserPos(0, 0); @@ -604,7 +605,7 @@ public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, for (FieldReplaceInfo replaceInfo : replaceInfoList) { fieldNames = Lists.newArrayList(); replaceFieldName(pollSqlNode, replaceInfo.getMappingTable(), replaceInfo.getTargetTableName(), replaceInfo.getTargetTableAlias()); - addAliasForFiledNode(pollSqlNode, fieldNames, replaceInfo.getMappingTable()); + addAliasForFieldNode(pollSqlNode, fieldNames, replaceInfo.getMappingTable()); } } From 53ce0625de04f6b79fc8d342bdffc1479a38b702 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 19 Aug 2019 19:27:16 +0800 Subject: [PATCH 350/470] kafka sink use retractstream --- .../flink/sql/sink/kafka/KafkaSink.java | 28 +++++++++++++----- .../flink/sql/sink/kafka/KafkaSink.java | 28 +++++++++++++----- .../flink/sql/sink/kafka/KafkaSink.java | 29 +++++++++++++------ 3 files changed, 60 insertions(+), 25 deletions(-) diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 864bf997f..29b97e6d2 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -23,10 +23,12 @@ import com.dtstack.flink.sql.table.TargetTableInfo; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; -import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.RetractStreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; import java.util.Properties; @@ -38,7 +40,7 @@ * @author DocLi * @modifyer maqi */ -public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { +public class KafkaSink implements RetractStreamTableSink, IStreamSinkGener { protected String[] fieldNames; @@ -68,24 +70,34 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { } properties.setProperty("bootstrap.servers", kafka09SinkTableInfo.getBootstrapServers()); - this.serializationSchema = new CustomerJsonRowSerializationSchema(getOutputType()); + this.serializationSchema = new CustomerJsonRowSerializationSchema(getOutputType().getTypeAt(1)); return this; } @Override - public void emitDataStream(DataStream dataStream) { + public TypeInformation getRecordType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public void emitDataStream(DataStream> dataStream) { KafkaTableSink kafkaTableSink = new CustomerKafka09JsonTableSink( topic, properties, serializationSchema ); - kafkaTableSink.emitDataStream(dataStream); + + DataStream ds = dataStream.map((Tuple2 record) -> { + return record.f1; + }).returns(getOutputType().getTypeAt(1)); + + kafkaTableSink.emitDataStream(ds); } @Override - public TypeInformation getOutputType() { - return new RowTypeInfo(fieldTypes, fieldNames); + public TupleTypeInfo> getOutputType() { + return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), new RowTypeInfo(fieldTypes, fieldNames)); } @Override @@ -99,7 +111,7 @@ public TypeInformation[] getFieldTypes() { } @Override - public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { this.fieldNames = fieldNames; this.fieldTypes = fieldTypes; return this; diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 427bf14cc..4b5c04db0 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -23,10 +23,12 @@ import com.dtstack.flink.sql.table.TargetTableInfo; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; -import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.RetractStreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; @@ -42,7 +44,7 @@ * @modifyer maqi * */ -public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { +public class KafkaSink implements RetractStreamTableSink, IStreamSinkGener { protected String[] fieldNames; @@ -73,24 +75,34 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { } properties.setProperty("bootstrap.servers", kafka10SinkTableInfo.getBootstrapServers()); - this.serializationSchema = new CustomerJsonRowSerializationSchema(getOutputType()); + this.serializationSchema = new CustomerJsonRowSerializationSchema(getOutputType().getTypeAt(1)); return this; } @Override - public void emitDataStream(DataStream dataStream) { + public TypeInformation getRecordType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public void emitDataStream(DataStream> dataStream) { KafkaTableSink kafkaTableSink = new CustomerKafka10JsonTableSink( topic, properties, serializationSchema ); - kafkaTableSink.emitDataStream(dataStream); + + DataStream ds = dataStream.map((Tuple2 record) -> { + return record.f1; + }).returns(getOutputType().getTypeAt(1)); + + kafkaTableSink.emitDataStream(ds); } @Override - public TypeInformation getOutputType() { - return new RowTypeInfo(fieldTypes, fieldNames); + public TupleTypeInfo> getOutputType() { + return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), new RowTypeInfo(fieldTypes, fieldNames)); } @Override @@ -104,7 +116,7 @@ public TypeInformation[] getFieldTypes() { } @Override - public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { this.fieldNames = fieldNames; this.fieldTypes = fieldTypes; return this; diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index d4cf64e9b..f5cc2be2d 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -23,11 +23,12 @@ import com.dtstack.flink.sql.table.TargetTableInfo; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.formats.json.JsonRowSerializationSchema; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; -import org.apache.flink.table.sinks.AppendStreamTableSink; +import org.apache.flink.table.sinks.RetractStreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; import java.util.Properties; @@ -42,7 +43,7 @@ * @modifyer maqi * */ -public class KafkaSink implements AppendStreamTableSink, IStreamSinkGener { +public class KafkaSink implements RetractStreamTableSink, IStreamSinkGener { protected String[] fieldNames; @@ -71,24 +72,34 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { properties.setProperty(key, kafka11SinkTableInfo.getKafkaParam(key)); } properties.setProperty("bootstrap.servers", kafka11SinkTableInfo.getBootstrapServers()); - this.serializationSchema = new CustomerJsonRowSerializationSchema(getOutputType()); + this.serializationSchema = new CustomerJsonRowSerializationSchema(getOutputType().getTypeAt(1)); return this; } @Override - public void emitDataStream(DataStream dataStream) { + public TypeInformation getRecordType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public void emitDataStream(DataStream> dataStream) { KafkaTableSink kafkaTableSink = new CustomerKafka11JsonTableSink( topic, properties, serializationSchema ); - kafkaTableSink.emitDataStream(dataStream); + + DataStream ds = dataStream.map((Tuple2 record) -> { + return record.f1; + }).returns(getOutputType().getTypeAt(1)); + + kafkaTableSink.emitDataStream(ds); } @Override - public TypeInformation getOutputType() { - return new RowTypeInfo(fieldTypes, fieldNames); + public TupleTypeInfo> getOutputType() { + return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), new RowTypeInfo(fieldTypes, fieldNames)); } @Override @@ -102,7 +113,7 @@ public TypeInformation[] getFieldTypes() { } @Override - public TableSink configure(String[] fieldNames, TypeInformation[] fieldTypes) { + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { this.fieldNames = fieldNames; this.fieldTypes = fieldTypes; return this; From 6c0c0a7c422ddc40aebb29cf2c2b659a45c07c6e Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 19 Aug 2019 19:55:17 +0800 Subject: [PATCH 351/470] modify guava clas --- core/src/main/java/com/dtstack/flink/sql/Main.java | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index d55f0ceea..834c57d69 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -22,6 +22,7 @@ import com.dtstack.flink.sql.classloader.DtClassLoader; import com.dtstack.flink.sql.enums.ECacheType; +import com.dtstack.flink.sql.environment.MyLocalStreamEnvironment; import com.dtstack.flink.sql.exec.FlinkSQLExec; import com.dtstack.flink.sql.parser.CreateFuncParser; import com.dtstack.flink.sql.parser.CreateTmpTableParser; @@ -53,19 +54,21 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; -import org.apache.flink.calcite.shaded.com.google.common.base.Strings; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; -import org.apache.flink.calcite.shaded.com.google.common.collect.Sets; import org.apache.flink.client.program.ContextEnvironment; import org.apache.flink.configuration.Configuration; +import org.apache.flink.shaded.guava18.com.google.common.base.Preconditions; +import org.apache.flink.shaded.guava18.com.google.common.base.Strings; +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import org.apache.flink.shaded.guava18.com.google.common.collect.Sets; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamContextEnvironment; + import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; + import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; import org.slf4j.Logger; From bc75b15377955d62edcb58e4857de14f11bd522c Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 19 Aug 2019 20:53:11 +0800 Subject: [PATCH 352/470] insert sink table by name --- core/src/main/java/com/dtstack/flink/sql/Main.java | 2 +- .../com/dtstack/flink/sql/exec/FlinkSQLExec.java | 13 ++++++++++--- .../com/dtstack/flink/sql/side/SideSqlExec.java | 6 +++--- 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 834c57d69..d321de41b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -333,7 +333,7 @@ private static StreamExecutionEnvironment getStreamExeEnv(Properties confPropert StreamExecutionEnvironment env = !ClusterMode.local.name().equals(deployMode) ? StreamExecutionEnvironment.getExecutionEnvironment() : new MyLocalStreamEnvironment(); - + env.getConfig().disableClosureCleaner(); env.setParallelism(FlinkUtil.getEnvParallelism(confProperties)); Configuration globalJobParameters = new Configuration(); Method method = Configuration.class.getDeclaredMethod("setValueInternal", String.class, Object.class); diff --git a/core/src/main/java/com/dtstack/flink/sql/exec/FlinkSQLExec.java b/core/src/main/java/com/dtstack/flink/sql/exec/FlinkSQLExec.java index d0191aec3..e1162bc78 100644 --- a/core/src/main/java/com/dtstack/flink/sql/exec/FlinkSQLExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/exec/FlinkSQLExec.java @@ -11,6 +11,8 @@ import org.apache.flink.table.calcite.FlinkPlannerImpl; import org.apache.flink.table.plan.logical.LogicalRelNode; import org.apache.flink.table.plan.schema.TableSinkTable; +import org.apache.flink.table.plan.schema.TableSourceSinkTable; +import scala.Option; import java.lang.reflect.Method; @@ -39,12 +41,17 @@ public static void sqlUpdate(StreamTableEnvironment tableEnv, String stmt) throw Method method = TableEnvironment.class.getDeclaredMethod("getTable", String.class); method.setAccessible(true); + Option sinkTab = (Option)method.invoke(tableEnv, targetTableName); - TableSinkTable targetTable = (TableSinkTable) method.invoke(tableEnv, targetTableName); - String[] fieldNames = targetTable.tableSink().getFieldNames(); + if (sinkTab.isEmpty()) { + throw new ValidationException("Sink table " + targetTableName + "not found in flink"); + } - Table newTable = null; + TableSourceSinkTable targetTable = (TableSourceSinkTable) sinkTab.get(); + TableSinkTable tableSinkTable = (TableSinkTable)targetTable.tableSinkTable().get(); + String[] fieldNames = tableSinkTable.tableSink().getFieldNames(); + Table newTable = null; try { newTable = queryResult.select(String.join(",", fieldNames)); } catch (Exception e) { diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index ac727ec05..938d9cb34 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -116,7 +116,7 @@ public void exec(String sql, Map sideTableMap, StreamTabl } }else if(pollSqlNode.getKind() == AS){ AliasInfo aliasInfo = parseASNode(pollSqlNode); - Table table = tableEnv.sql(aliasInfo.getName()); + Table table = tableEnv.sqlQuery(aliasInfo.getName()); tableEnv.registerTable(aliasInfo.getAlias(), table); localTableCache.put(aliasInfo.getAlias(), table); } @@ -613,7 +613,7 @@ public void registerTmpTable(CreateTmpTableParser.SqlParserResult result, tableEnv.sqlUpdate(pollSqlNode.toString()); }else if(pollSqlNode.getKind() == AS){ AliasInfo aliasInfo = parseASNode(pollSqlNode); - Table table = tableEnv.sql(aliasInfo.getName()); + Table table = tableEnv.sqlQuery(aliasInfo.getName()); tableEnv.registerTable(aliasInfo.getAlias(), table); if(LOG.isInfoEnabled()){ LOG.info("Register Table {} by {}", aliasInfo.getAlias(), aliasInfo.getName()); @@ -740,7 +740,7 @@ private boolean checkFieldsInfo(CreateTmpTableParser.SqlParserResult result, Tab fieldNames.add(fieldName); String fieldType = filed[filed.length - 1 ].trim(); Class fieldClass = ClassUtil.stringConvertClass(fieldType); - Class tableField = table.getSchema().getType(i).get().getTypeClass(); + Class tableField = table.getSchema().getFieldType(i).get().getTypeClass(); if (fieldClass == tableField){ continue; } else { From 37ee353b9309d4ec2181c9694443519c608b4809 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 20 Aug 2019 11:35:17 +0800 Subject: [PATCH 353/470] kafka sink use retractStream --- .../flink/sql/sink/kafka/KafkaSink.java | 34 ++++++++++++++---- .../flink/sql/sink/kafka/KafkaSink.java | 32 +++++++++++++---- .../flink/sql/sink/kafka/KafkaSink.java | 35 +++++++++++++++---- 3 files changed, 83 insertions(+), 18 deletions(-) diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 29b97e6d2..efc1e0b88 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -27,10 +27,15 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.sinks.RetractStreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; + +import java.util.Optional; import java.util.Properties; /** @@ -53,10 +58,25 @@ public class KafkaSink implements RetractStreamTableSink, IStreamSinkGener< /** Serialization schema for encoding records to Kafka. */ protected SerializationSchema serializationSchema; + /** The schema of the table. */ + private TableSchema schema; + + /** Partitioner to select Kafka partition for each item. */ + protected Optional> partitioner; + + @Override public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { KafkaSinkTableInfo kafka09SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; this.topic = kafka09SinkTableInfo.getTopic(); + + properties = new Properties(); + properties.setProperty("bootstrap.servers", kafka09SinkTableInfo.getBootstrapServers()); + for (String key : kafka09SinkTableInfo.getKafkaParamKeys()) { + properties.setProperty(key, kafka09SinkTableInfo.getKafkaParam(key)); + } + + this.partitioner = Optional.of(new FlinkFixedPartitioner<>()); this.fieldNames = kafka09SinkTableInfo.getFields(); TypeInformation[] types = new TypeInformation[kafka09SinkTableInfo.getFields().length]; for (int i = 0; i < kafka09SinkTableInfo.getFieldClasses().length; i++) { @@ -64,11 +84,11 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { } this.fieldTypes = types; - properties = new Properties(); - for (String key : kafka09SinkTableInfo.getKafkaParamKeys()) { - properties.setProperty(key, kafka09SinkTableInfo.getKafkaParam(key)); + TableSchema.Builder schemaBuilder = TableSchema.builder(); + for (int i=0;i getRecordType() { @Override public void emitDataStream(DataStream> dataStream) { - KafkaTableSink kafkaTableSink = new CustomerKafka09JsonTableSink( + KafkaTableSinkBase kafkaTableSink = new CustomerKafka09JsonTableSink( + schema, topic, properties, + partitioner, serializationSchema ); diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 4b5c04db0..367e57341 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -27,7 +27,10 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.sinks.RetractStreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; @@ -58,10 +61,25 @@ public class KafkaSink implements RetractStreamTableSink, IStreamSinkGener< /** Serialization schema for encoding records to Kafka. */ protected SerializationSchema serializationSchema; + /** The schema of the table. */ + private TableSchema schema; + + /** Partitioner to select Kafka partition for each item. */ + protected Optional> partitioner; + @Override public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { KafkaSinkTableInfo kafka10SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; this.topic = kafka10SinkTableInfo.getTopic(); + + properties = new Properties(); + properties.setProperty("bootstrap.servers", kafka10SinkTableInfo.getBootstrapServers()); + + for (String key : kafka10SinkTableInfo.getKafkaParamKeys()) { + properties.setProperty(key, kafka10SinkTableInfo.getKafkaParam(key)); + } + + this.partitioner = Optional.of(new FlinkFixedPartitioner<>()); this.fieldNames = kafka10SinkTableInfo.getFields(); TypeInformation[] types = new TypeInformation[kafka10SinkTableInfo.getFields().length]; for (int i = 0; i < kafka10SinkTableInfo.getFieldClasses().length; i++) { @@ -69,11 +87,11 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { } this.fieldTypes = types; - properties = new Properties(); - for (String key : kafka10SinkTableInfo.getKafkaParamKeys()) { - properties.setProperty(key, kafka10SinkTableInfo.getKafkaParam(key)); + TableSchema.Builder schemaBuilder = TableSchema.builder(); + for (int i=0;i getRecordType() { @Override public void emitDataStream(DataStream> dataStream) { - KafkaTableSink kafkaTableSink = new CustomerKafka10JsonTableSink( + KafkaTableSinkBase kafkaTableSink = new CustomerKafka10JsonTableSink( + schema, topic, properties, + partitioner, serializationSchema ); diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index f5cc2be2d..41cb877f5 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -27,10 +27,15 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.sinks.RetractStreamTableSink; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; + +import java.util.Optional; import java.util.Properties; /** @@ -56,10 +61,25 @@ public class KafkaSink implements RetractStreamTableSink, IStreamSinkGener /** Serialization schema for encoding records to Kafka. */ protected SerializationSchema serializationSchema; + /** The schema of the table. */ + private TableSchema schema; + + /** Partitioner to select Kafka partition for each item. */ + protected Optional> partitioner; + + @Override public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { KafkaSinkTableInfo kafka11SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; this.topic = kafka11SinkTableInfo.getTopic(); + + properties = new Properties(); + properties.setProperty("bootstrap.servers", kafka11SinkTableInfo.getBootstrapServers()); + + for (String key : kafka11SinkTableInfo.getKafkaParamKeys()) { + properties.setProperty(key, kafka11SinkTableInfo.getKafkaParam(key)); + } + this.partitioner = Optional.of(new FlinkFixedPartitioner<>()); this.fieldNames = kafka11SinkTableInfo.getFields(); TypeInformation[] types = new TypeInformation[kafka11SinkTableInfo.getFields().length]; for (int i = 0; i < kafka11SinkTableInfo.getFieldClasses().length; i++) { @@ -67,11 +87,12 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { } this.fieldTypes = types; - properties = new Properties(); - for (String key : kafka11SinkTableInfo.getKafkaParamKeys()) { - properties.setProperty(key, kafka11SinkTableInfo.getKafkaParam(key)); + TableSchema.Builder schemaBuilder = TableSchema.builder(); + for (int i=0;i getRecordType() { @Override public void emitDataStream(DataStream> dataStream) { - KafkaTableSink kafkaTableSink = new CustomerKafka11JsonTableSink( + KafkaTableSinkBase kafkaTableSink = new CustomerKafka11JsonTableSink( + schema, topic, properties, + partitioner, serializationSchema ); From 69de98914e9f063796a8541f91671aa53306ec25 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 23 Aug 2019 13:59:06 +0800 Subject: [PATCH 354/470] parse not in --- core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 37dcd84f8..978072e93 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -376,6 +376,7 @@ private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable Date: Fri, 23 Aug 2019 15:13:58 +0800 Subject: [PATCH 355/470] sql keyword parse --- .../dtstack/flink/sql/side/SideSqlExec.java | 25 +++++++------------ 1 file changed, 9 insertions(+), 16 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 978072e93..2c355e797 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -365,26 +365,18 @@ private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable Date: Tue, 27 Aug 2019 19:04:19 +0800 Subject: [PATCH 356/470] rdb dirty data --- .../flink/sql/metric/MetricConstant.java | 2 + .../flink/sql/sink/MetricOutputFormat.java | 3 + .../rdb/format/RetractJDBCOutputFormat.java | 87 ++++++++++++++----- 3 files changed, 70 insertions(+), 22 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java b/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java index 89c411479..17bb75e82 100644 --- a/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java +++ b/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java @@ -45,6 +45,8 @@ public class MetricConstant { public static final String DT_NUM_RECORDS_OUT = "dtNumRecordsOut"; + public static final String DT_NUM_DIRTY_RECORDS_OUT = "dtNumDirtyRecordsOut"; + public static final String DT_NUM_RECORDS_OUT_RATE = "dtNumRecordsOutRate"; public static final String DT_EVENT_DELAY_GAUGE = "dtEventDelay"; diff --git a/core/src/main/java/com/dtstack/flink/sql/sink/MetricOutputFormat.java b/core/src/main/java/com/dtstack/flink/sql/sink/MetricOutputFormat.java index 4e11280e7..f56f531e0 100644 --- a/core/src/main/java/com/dtstack/flink/sql/sink/MetricOutputFormat.java +++ b/core/src/main/java/com/dtstack/flink/sql/sink/MetricOutputFormat.java @@ -32,10 +32,13 @@ public abstract class MetricOutputFormat extends RichOutputFormat{ protected transient Counter outRecords; + protected transient Counter outDirtyRecords; + protected transient Meter outRecordsRate; public void initMetric() { outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + outDirtyRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_DIRTY_RECORDS_OUT); outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index bfe683e46..701497bf4 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -32,6 +32,7 @@ import java.sql.DriverManager; import java.sql.PreparedStatement; import java.sql.SQLException; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -66,9 +67,11 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private String insertQuery; public int[] typesArray; + /** 存储用于批量写入的数据 */ + protected List rows = new ArrayList(); + private Connection dbConn; private PreparedStatement upload; - private AtomicInteger batchCount = new AtomicInteger(0); private transient ScheduledThreadPoolExecutor timerService; @@ -107,7 +110,7 @@ public void open(int taskNumber, int numTasks) throws IOException { throw new SQLException("Table " + tableName + " doesn't exist"); } - if (batchWaitInterval > 0) { + if (batchWaitInterval > 0 && batchNum > 1) { LOG.info("open batch wait interval scheduled, interval is {} ms", batchWaitInterval); timerService = new ScheduledThreadPoolExecutor(1); @@ -134,6 +137,7 @@ private void establishConnection() throws SQLException, ClassNotFoundException { } else { dbConn = DriverManager.getConnection(dbURL, username, password); } + dbConn.setAutoCommit(false); } /** @@ -150,35 +154,53 @@ private void establishConnection() throws SQLException, ClassNotFoundException { * @see PreparedStatement */ @Override - public void writeRecord(Tuple2 tuple2) throws IOException { + public void writeRecord(Tuple2 tuple2) { Tuple2 tupleTrans = tuple2; Boolean retract = tupleTrans.getField(0); Row row = tupleTrans.getField(1); - if (typesArray != null && typesArray.length > 0 && typesArray.length != row.getArity()) { LOG.warn("Column SQL types array doesn't match arity of passed Row! Check the passed array..."); } - try { - if (retract) { - insertWrite(row); - outRecords.inc(); - } else { - //do nothing - } - } catch (SQLException | IllegalArgumentException e) { - throw new IllegalArgumentException("writeRecord() failed", e); + + if (retract) { + insertWrite(row); + outRecords.inc(); + } else { + //do nothing } } - private void insertWrite(Row row) throws SQLException { - updatePreparedStmt(row, upload); - upload.addBatch(); - batchCount.incrementAndGet(); - if (batchCount.get() >= batchNum) { - submitExecuteBatch(); + private void insertWrite(Row row) { + checkConnectionOpen(dbConn); + + if (batchNum == 1) { + writeSingleRecord(row); + } else { + try { + rows.add(row); + updatePreparedStmt(row, upload); + upload.addBatch(); + } catch (SQLException e) { + LOG.error("", e); + } + + if (rows.size() >= batchNum) { + submitExecuteBatch(); + } + } + } + + private void writeSingleRecord(Row row) { + try { + updatePreparedStmt(row, upload); + upload.execute(); + } catch (SQLException e) { + outDirtyRecords.inc(); + LOG.error("record insert failed ..", row.toString()); + LOG.error("", e); } } @@ -276,9 +298,30 @@ private synchronized void submitExecuteBatch() { try { LOG.info("submitExecuteBatch start......"); this.upload.executeBatch(); - this.batchCount.set(0); + dbConn.commit(); } catch (SQLException e) { - LOG.error("", e); + try { + dbConn.rollback(); + } catch (SQLException e1) { + LOG.error("rollback data error !", e); + } + rows.forEach(this::writeSingleRecord); + } finally { + rows.clear(); + } + } + + private void checkConnectionOpen(Connection dbConn) { + try { + if (dbConn.isClosed()) { + LOG.info("db connection reconnect.."); + establishConnection(); + upload = dbConn.prepareStatement(insertQuery); + } + } catch (SQLException e) { + LOG.error("check connection open failed..", e); + } catch (ClassNotFoundException e) { + LOG.error("load jdbc class error when reconnect db..", e); } } @@ -302,7 +345,7 @@ public void close() throws IOException { LOG.info("Inputformat couldn't be closed - ", se); } finally { upload = null; - batchCount.set(0); + rows.clear(); } try { From 1a8193d2bf99e7e3f32cee7b87732a1c8cccbf78 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 28 Aug 2019 16:28:46 +0800 Subject: [PATCH 357/470] jdbc reconnection --- .../rdb/format/RetractJDBCOutputFormat.java | 38 ++++++++++--------- 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 701497bf4..6b2529583 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -98,7 +98,7 @@ public void configure(Configuration parameters) { public void open(int taskNumber, int numTasks) throws IOException { try { LOG.info("PreparedStatement execute batch num is {}", batchNum); - establishConnection(); + dbConn = establishConnection(); initMetric(); if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { @@ -130,14 +130,16 @@ public void open(int taskNumber, int numTasks) throws IOException { } - private void establishConnection() throws SQLException, ClassNotFoundException { + private Connection establishConnection() throws SQLException, ClassNotFoundException { + Connection connection ; Class.forName(drivername); if (username == null) { - dbConn = DriverManager.getConnection(dbURL); + connection = DriverManager.getConnection(dbURL); } else { - dbConn = DriverManager.getConnection(dbURL, username, password); + connection = DriverManager.getConnection(dbURL, username, password); } - dbConn.setAutoCommit(false); + connection.setAutoCommit(false); + return connection; } /** @@ -175,22 +177,21 @@ public void writeRecord(Tuple2 tuple2) { private void insertWrite(Row row) { checkConnectionOpen(dbConn); - - if (batchNum == 1) { - writeSingleRecord(row); - } else { - try { - rows.add(row); + try { + if (batchNum == 1) { + writeSingleRecord(row); + } else { updatePreparedStmt(row, upload); + rows.add(row); upload.addBatch(); - } catch (SQLException e) { - LOG.error("", e); - } - - if (rows.size() >= batchNum) { - submitExecuteBatch(); + if (rows.size() >= batchNum) { + submitExecuteBatch(); + } } + } catch (SQLException e) { + LOG.error("", e); } + } private void writeSingleRecord(Row row) { @@ -305,6 +306,7 @@ private synchronized void submitExecuteBatch() { } catch (SQLException e1) { LOG.error("rollback data error !", e); } + rows.forEach(this::writeSingleRecord); } finally { rows.clear(); @@ -315,7 +317,7 @@ private void checkConnectionOpen(Connection dbConn) { try { if (dbConn.isClosed()) { LOG.info("db connection reconnect.."); - establishConnection(); + dbConn= establishConnection(); upload = dbConn.prepareStatement(insertQuery); } } catch (SQLException e) { From 6f9350ddba88803cde8ce04a3fd588956b1d9fb0 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 30 Aug 2019 10:15:05 +0800 Subject: [PATCH 358/470] db reconnection --- pom.xml | 18 ++++++------- .../rdb/format/RetractJDBCOutputFormat.java | 26 +++++++++++++++---- 2 files changed, 30 insertions(+), 14 deletions(-) diff --git a/pom.xml b/pom.xml index 0e2d9e709..8dd76e437 100644 --- a/pom.xml +++ b/pom.xml @@ -10,19 +10,19 @@ http://maven.apache.org core - kafka09 - kafka10 + + kafka11 mysql - hbase - elasticsearch5 - mongo - redis5 + + + + launcher rdb - sqlserver - oracle - cassandra + + + diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 382b687e1..d12d1a5e2 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -85,7 +85,7 @@ public void configure(Configuration parameters) { @Override public void open(int taskNumber, int numTasks) throws IOException { try { - establishConnection(); + dbConn = establishConnection(); initMetric(); if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { if (isReplaceInsertQuery()) { @@ -104,13 +104,15 @@ public void open(int taskNumber, int numTasks) throws IOException { } - private void establishConnection() throws SQLException, ClassNotFoundException { + private Connection establishConnection() throws SQLException, ClassNotFoundException { + Connection connection ; Class.forName(drivername); if (username == null) { - dbConn = DriverManager.getConnection(dbURL); + connection = DriverManager.getConnection(dbURL); } else { - dbConn = DriverManager.getConnection(dbURL, username, password); + connection = DriverManager.getConnection(dbURL, username, password); } + return connection; } /** @@ -151,7 +153,7 @@ public void writeRecord(Tuple2 tuple2) throws IOException { private void insertWrite(Row row) throws SQLException { - + checkConnectionOpen(dbConn); updatePreparedStmt(row, upload); upload.addBatch(); batchCount++; @@ -161,6 +163,20 @@ private void insertWrite(Row row) throws SQLException { } } + private void checkConnectionOpen(Connection dbConn) { + try { + if (dbConn.isClosed()) { + LOG.info("db connection reconnect.."); + dbConn= establishConnection(); + upload = dbConn.prepareStatement(insertQuery); + } + } catch (SQLException e) { + LOG.error("check connection open failed..", e); + } catch (ClassNotFoundException e) { + LOG.error("load jdbc class error when reconnect db..", e); + } + } + private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLException { if (typesArray == null) { // no types provided From 945171eb444e3ce068638364689dbdfa48886523 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 30 Aug 2019 10:17:04 +0800 Subject: [PATCH 359/470] revert pom --- pom.xml | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/pom.xml b/pom.xml index 8dd76e437..0e2d9e709 100644 --- a/pom.xml +++ b/pom.xml @@ -10,19 +10,19 @@ http://maven.apache.org core - - + kafka09 + kafka10 kafka11 mysql - - - - + hbase + elasticsearch5 + mongo + redis5 launcher rdb - - - + sqlserver + oracle + cassandra From 8a04ec8d3ae9febf1d24c3f16d35de8f3bb71d6f Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 30 Aug 2019 11:43:07 +0800 Subject: [PATCH 360/470] batch insert --- .../com/dtstack/flink/sql/util/JDBCUtils.java | 34 +++++++ .../rdb/format/RetractJDBCOutputFormat.java | 96 +++++++++++++++---- 2 files changed, 112 insertions(+), 18 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/util/JDBCUtils.java diff --git a/core/src/main/java/com/dtstack/flink/sql/util/JDBCUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/JDBCUtils.java new file mode 100644 index 000000000..a63fc0ab4 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/util/JDBCUtils.java @@ -0,0 +1,34 @@ +package com.dtstack.flink.sql.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.DriverManager; + +public class JDBCUtils { + + private static final Logger LOG = LoggerFactory.getLogger(ClassUtil.class); + + public final static String lock_str = "jdbc_lock_str"; + + public static void forName(String clazz, ClassLoader classLoader) { + synchronized (lock_str){ + try { + Class.forName(clazz, true, classLoader); + DriverManager.setLoginTimeout(10); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + + public synchronized static void forName(String clazz) { + try { + Class driverClass = Class.forName(clazz); + driverClass.newInstance(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index d12d1a5e2..b53134d2b 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.sink.rdb.format; import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.util.JDBCUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.tuple.Tuple2; @@ -31,6 +32,8 @@ import java.sql.*; import java.util.*; import java.io.IOException; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import com.dtstack.flink.sql.sink.MetricOutputFormat; @@ -53,6 +56,10 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private String tableName; private String dbType; private RdbSink dbSink; + // trigger preparedStatement execute batch interval + private long batchWaitInterval = 10000l; + + // batchNum private int batchInterval = 5000; private String insertQuery; public int[] typesArray; @@ -60,13 +67,16 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private Connection dbConn; private PreparedStatement upload; - private int batchCount = 0; + /** 存储用于批量写入的数据 */ + protected List rows = new ArrayList(); //index field private Map> realIndexes = Maps.newHashMap(); //full field private List fullField = Lists.newArrayList(); + private transient ScheduledThreadPoolExecutor timerService; + public RetractJDBCOutputFormat() { } @@ -85,6 +95,7 @@ public void configure(Configuration parameters) { @Override public void open(int taskNumber, int numTasks) throws IOException { try { + LOG.info("PreparedStatement execute batch num is {}", batchInterval); dbConn = establishConnection(); initMetric(); if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { @@ -96,6 +107,17 @@ public void open(int taskNumber, int numTasks) throws IOException { throw new SQLException("Table " + tableName + " doesn't exist"); } + if (batchWaitInterval > 0 && batchInterval > 1) { + LOG.info("open batch wait interval scheduled, interval is {} ms", batchWaitInterval); + + timerService = new ScheduledThreadPoolExecutor(1); + timerService.scheduleAtFixedRate(() -> { + submitExecuteBatch(); + }, 0, batchWaitInterval, TimeUnit.MILLISECONDS); + + } + + } catch (SQLException sqe) { throw new IllegalArgumentException("open() failed.", sqe); } catch (ClassNotFoundException cnfe) { @@ -106,12 +128,13 @@ public void open(int taskNumber, int numTasks) throws IOException { private Connection establishConnection() throws SQLException, ClassNotFoundException { Connection connection ; - Class.forName(drivername); + JDBCUtils.forName(drivername, getClass().getClassLoader()); if (username == null) { connection = DriverManager.getConnection(dbURL); } else { connection = DriverManager.getConnection(dbURL, username, password); } + connection.setAutoCommit(false); return connection; } @@ -129,7 +152,7 @@ private Connection establishConnection() throws SQLException, ClassNotFoundExcep * @see PreparedStatement */ @Override - public void writeRecord(Tuple2 tuple2) throws IOException { + public void writeRecord(Tuple2 tuple2) { Tuple2 tupleTrans = tuple2; Boolean retract = tupleTrans.getField(0); @@ -139,27 +162,64 @@ public void writeRecord(Tuple2 tuple2) throws IOException { if (typesArray != null && typesArray.length > 0 && typesArray.length != row.getArity()) { LOG.warn("Column SQL types array doesn't match arity of passed Row! Check the passed array..."); } + + if (retract) { + insertWrite(row); + outRecords.inc(); + } else { + //do nothing + } + + } + + + private void insertWrite(Row row) { + System.out.println("接受到数据row:" +row ); + checkConnectionOpen(dbConn); try { - if (retract) { - insertWrite(row); - outRecords.inc(); + if (batchInterval == 1) { + writeSingleRecord(row); } else { - //do nothing + updatePreparedStmt(row, upload); + rows.add(row); + upload.addBatch(); + if (rows.size() >= batchInterval) { + submitExecuteBatch(); + } } - } catch (SQLException | IllegalArgumentException e) { - throw new IllegalArgumentException("writeRecord() failed", e); + } catch (SQLException e) { + LOG.error("", e); + } + } + + private void writeSingleRecord(Row row) { + try { + updatePreparedStmt(row, upload); + upload.execute(); + System.out.println("单条插入成功:" + row); + } catch (SQLException e) { + System.out.println("单条插入失败:" + row); + LOG.error("record insert failed ..", row.toString()); + LOG.error("", e); } } + private synchronized void submitExecuteBatch() { + try { + LOG.info("submitExecuteBatch start......"); + this.upload.executeBatch(); + dbConn.commit(); + rows.forEach(row -> System.out.println("批量插入成功:"+ row)); + } catch (SQLException e) { + try { + dbConn.rollback(); + } catch (SQLException e1) { + LOG.error("rollback data error !", e); + } - private void insertWrite(Row row) throws SQLException { - checkConnectionOpen(dbConn); - updatePreparedStmt(row, upload); - upload.addBatch(); - batchCount++; - if (batchCount >= batchInterval) { - upload.executeBatch(); - batchCount = 0; + rows.forEach(this::writeSingleRecord); + } finally { + rows.clear(); } } @@ -282,7 +342,7 @@ public void close() throws IOException { LOG.info("Inputformat couldn't be closed - " + se.getMessage()); } finally { upload = null; - batchCount = 0; + } try { From 80f8a848ee08e749fe8e0a920fc2b771b520cf45 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Sat, 31 Aug 2019 14:36:28 +0800 Subject: [PATCH 361/470] convert json timestamp --- .../kafka/CustomerJsonDeserialization.java | 34 ++++++++++++++++++- 1 file changed, 33 insertions(+), 1 deletion(-) diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 3b73f0d51..53837462e 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -24,10 +24,12 @@ import com.dtstack.flink.sql.source.AbsDeserialization; import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; @@ -42,7 +44,11 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.lang.reflect.Array; import java.lang.reflect.Field; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; import java.util.Iterator; import java.util.Map; import java.util.Set; @@ -137,7 +143,8 @@ public Row deserialize(byte[] message) throws IOException { } } else { // Read the value as specified type - Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + + Object value = convert(node, fieldTypes[i]); row.setField(i, value); } } @@ -245,4 +252,29 @@ protected void registerPtMetric(AbstractFetcher fetcher) throws Exceptio private static String partitionLagMetricName(TopicPartition tp) { return tp + ".records-lag"; } + + private Object convert(JsonNode node, TypeInformation info) { + if (info.getTypeClass().equals(Types.BOOLEAN.getTypeClass())) { + return node.asBoolean(); + } else if (info.getTypeClass().equals(Types.STRING.getTypeClass())) { + return node.asText(); + } else if (info.getTypeClass().equals(Types.SQL_DATE.getTypeClass())) { + return Date.valueOf(node.asText()); + } else if (info.getTypeClass().equals(Types.SQL_TIME.getTypeClass())) { + // local zone + return Time.valueOf(node.asText()); + } else if (info.getTypeClass().equals(Types.SQL_TIMESTAMP.getTypeClass())) { + // local zone + return Timestamp.valueOf(node.asText()); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return objectMapper.treeToValue(node, info.getTypeClass()); + } catch (JsonProcessingException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for node: " + node); + } + } + } + } From 7eff6be06308a5ec7711560095271e07d1ae79bb Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Sat, 31 Aug 2019 15:02:01 +0800 Subject: [PATCH 362/470] kafka parse timestamp --- .../kafka/CustomerJsonDeserialization.java | 37 ++++++++++++++++++- .../kafka/CustomerJsonDeserialization.java | 34 ++++++++++++++++- .../kafka/CustomerJsonDeserialization.java | 3 +- 3 files changed, 69 insertions(+), 5 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 28f0e470b..ea7db1fae 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -24,10 +24,12 @@ import com.dtstack.flink.sql.source.AbsDeserialization; import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; @@ -43,6 +45,9 @@ import java.io.IOException; import java.lang.reflect.Field; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; import java.util.Iterator; import java.util.Map; import java.util.Set; @@ -134,17 +139,18 @@ public Row deserialize(byte[] message) throws IOException { } } else { // Read the value as specified type - Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + Object value = convert(node, fieldTypes[i]); row.setField(i, value); } } numInResolveRecord.inc(); return row; - } catch (Throwable t) { + } catch (Exception e) { //add metric of dirty data if (dirtyDataCounter.getCount()%rowLenth == 0){ LOG.info("dirtyData: " + new String(message)); + LOG.info(" " ,e); } dirtyDataCounter.inc(); return null; @@ -243,4 +249,31 @@ protected void registerPtMetric(AbstractFetcher fetcher) throws Exceptio private static String partitionLagMetricName(TopicPartition tp) { return tp + ".records-lag"; } + + private Object convert(JsonNode node, TypeInformation info) { + if (info.getTypeClass().equals(Types.BOOLEAN.getTypeClass())) { + return node.asBoolean(); + } else if (info.getTypeClass().equals(Types.STRING.getTypeClass())) { + return node.asText(); + } else if (info.getTypeClass().equals(Types.SQL_DATE.getTypeClass())) { + return Date.valueOf(node.asText()); + } else if (info.getTypeClass().equals(Types.SQL_TIME.getTypeClass())) { + // local zone + return Time.valueOf(node.asText()); + } else if (info.getTypeClass().equals(Types.SQL_TIMESTAMP.getTypeClass())) { + // local zone + return Timestamp.valueOf(node.asText()); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return objectMapper.treeToValue(node, info.getTypeClass()); + } catch (JsonProcessingException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for node: " + node); + } + } + } + + + } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 66d41e0ba..dc1bf89a2 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -24,10 +24,12 @@ import com.dtstack.flink.sql.source.AbsDeserialization; import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; @@ -43,6 +45,9 @@ import java.io.IOException; import java.lang.reflect.Field; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; import java.util.Iterator; import java.util.Map; import java.util.Set; @@ -134,17 +139,18 @@ public Row deserialize(byte[] message) throws IOException { } } else { // Read the value as specified type - Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + Object value = convert(node, fieldTypes[i]); row.setField(i, value); } } numInResolveRecord.inc(); return row; - } catch (Throwable t) { + } catch (Exception e) { //add metric of dirty data if (dirtyDataCounter.getCount()%rowLenth == 0){ LOG.info("dirtyData: " + new String(message)); + LOG.error(" ", e); } dirtyDataCounter.inc(); return null; @@ -244,4 +250,28 @@ protected void registerPtMetric(AbstractFetcher fetcher) throws Exceptio private static String partitionLagMetricName(TopicPartition tp) { return tp + ".records-lag"; } + + private Object convert(JsonNode node, TypeInformation info) { + if (info.getTypeClass().equals(Types.BOOLEAN.getTypeClass())) { + return node.asBoolean(); + } else if (info.getTypeClass().equals(Types.STRING.getTypeClass())) { + return node.asText(); + } else if (info.getTypeClass().equals(Types.SQL_DATE.getTypeClass())) { + return Date.valueOf(node.asText()); + } else if (info.getTypeClass().equals(Types.SQL_TIME.getTypeClass())) { + // local zone + return Time.valueOf(node.asText()); + } else if (info.getTypeClass().equals(Types.SQL_TIMESTAMP.getTypeClass())) { + // local zone + return Timestamp.valueOf(node.asText()); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return objectMapper.treeToValue(node, info.getTypeClass()); + } catch (JsonProcessingException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for node: " + node); + } + } + } } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 53837462e..60a42cea3 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -151,10 +151,11 @@ public Row deserialize(byte[] message) throws IOException { numInResolveRecord.inc(); return row; - } catch (Throwable t) { + } catch (Exception e) { //add metric of dirty data if (dirtyDataCounter.getCount()%rowLenth == 0){ LOG.info("dirtyData: " + new String(message)); + LOG.error("" , e); } dirtyDataCounter.inc(); return null; From e2f0f1f7366826198c94d6373e81909d02132bd3 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 2 Sep 2019 19:00:45 +0800 Subject: [PATCH 363/470] union parse --- .../java/com/dtstack/flink/sql/side/SideSQLParser.java | 10 ++++++++++ .../java/com/dtstack/flink/sql/side/SideSqlExec.java | 10 ++++++++++ .../sql/sink/rdb/format/RetractJDBCOutputFormat.java | 3 ++- 3 files changed, 22 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 6eb242255..e1c546240 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -110,6 +110,16 @@ private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue aliasInfo.setAlias(alias.toString()); return aliasInfo; + + case UNION: + SqlNode unionLeft = ((SqlBasicCall)sqlNode).getOperands()[0]; + SqlNode unionRight = ((SqlBasicCall)sqlNode).getOperands()[1]; + + parseSql(unionLeft, sideTableSet, queueInfo); + + parseSql(unionRight, sideTableSet, queueInfo); + + break; } return ""; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 69356ee42..b1591e8dc 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -306,6 +306,16 @@ private void replaceFieldName(SqlNode sqlNode, HashBasedTable Date: Mon, 2 Sep 2019 19:50:14 +0800 Subject: [PATCH 364/470] add commit --- .../flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java | 1 + 1 file changed, 1 insertion(+) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 6b2529583..37ead8b0c 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -198,6 +198,7 @@ private void writeSingleRecord(Row row) { try { updatePreparedStmt(row, upload); upload.execute(); + dbConn.commit(); } catch (SQLException e) { outDirtyRecords.inc(); LOG.error("record insert failed ..", row.toString()); From b7fdb2bd7f40b0fc315aa9d3444aa5b68d7be9ba Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 2 Sep 2019 21:01:06 +0800 Subject: [PATCH 365/470] union parse --- .../com/dtstack/flink/sql/side/SideSQLParser.java | 10 ++++++++++ .../java/com/dtstack/flink/sql/side/SideSqlExec.java | 11 +++++++++++ .../sql/sink/rdb/format/RetractJDBCOutputFormat.java | 5 +---- 3 files changed, 22 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 6eb242255..cab400c99 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -110,6 +110,16 @@ private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue aliasInfo.setAlias(alias.toString()); return aliasInfo; + + case UNION: + SqlNode unionLeft = ((SqlBasicCall) sqlNode).getOperands()[0]; + SqlNode unionRight = ((SqlBasicCall) sqlNode).getOperands()[1]; + + parseSql(unionLeft, sideTableSet, queueInfo); + + parseSql(unionRight, sideTableSet, queueInfo); + + break; } return ""; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 2c355e797..092686a24 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -232,6 +232,17 @@ private void replaceFieldName(SqlNode sqlNode, HashBasedTable System.out.println("批量插入成功:"+ row)); } catch (SQLException e) { try { dbConn.rollback(); From 7476661cb8b4650ab84583bc3515056768c211b8 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 3 Sep 2019 10:34:12 +0800 Subject: [PATCH 366/470] modify dirtyDataFrequency name --- .../flink/sql/source/kafka/CustomerJsonDeserialization.java | 6 +++--- .../flink/sql/source/kafka/CustomerJsonDeserialization.java | 6 +++--- .../flink/sql/source/kafka/CustomerJsonDeserialization.java | 6 +++--- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index ea7db1fae..d659b2882 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -67,7 +67,7 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private static final long serialVersionUID = 2385115520960444192L; - private static int rowLenth = 1000; + private static int dirtyDataFrequency = 1000; private final ObjectMapper objectMapper = new ObjectMapper(); @@ -117,7 +117,7 @@ public Row deserialize(byte[] message) throws IOException { try { JsonNode root = objectMapper.readTree(message); - if (numInRecord.getCount()%rowLenth == 0){ + if (numInRecord.getCount() % dirtyDataFrequency == 0) { LOG.info(root.toString()); } @@ -148,7 +148,7 @@ public Row deserialize(byte[] message) throws IOException { return row; } catch (Exception e) { //add metric of dirty data - if (dirtyDataCounter.getCount()%rowLenth == 0){ + if (dirtyDataCounter.getCount() % dirtyDataFrequency == 0) { LOG.info("dirtyData: " + new String(message)); LOG.info(" " ,e); } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index dc1bf89a2..5a717a8dc 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -67,7 +67,7 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private static final long serialVersionUID = 2385115520960444192L; - private static int rowLenth = 1000; + private static int dirtyDataFrequency = 1000; private final ObjectMapper objectMapper = new ObjectMapper(); @@ -117,7 +117,7 @@ public Row deserialize(byte[] message) throws IOException { try { JsonNode root = objectMapper.readTree(message); - if (numInRecord.getCount()%rowLenth == 0){ + if (numInRecord.getCount() % dirtyDataFrequency == 0) { LOG.info(root.toString()); } @@ -148,7 +148,7 @@ public Row deserialize(byte[] message) throws IOException { return row; } catch (Exception e) { //add metric of dirty data - if (dirtyDataCounter.getCount()%rowLenth == 0){ + if (dirtyDataCounter.getCount() % dirtyDataFrequency == 0) { LOG.info("dirtyData: " + new String(message)); LOG.error(" ", e); } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 60a42cea3..1002731e8 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -70,7 +70,7 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private static final long serialVersionUID = 2385115520960444192L; - private static int rowLenth = 1000; + private static int dirtyDataFrequency = 1000; private final ObjectMapper objectMapper = new ObjectMapper(); @@ -121,7 +121,7 @@ public Row deserialize(byte[] message) throws IOException { try { JsonNode root = objectMapper.readTree(message); - if (numInRecord.getCount()%rowLenth == 0){ + if (numInRecord.getCount() % dirtyDataFrequency == 0) { LOG.info(root.toString()); } @@ -153,7 +153,7 @@ public Row deserialize(byte[] message) throws IOException { return row; } catch (Exception e) { //add metric of dirty data - if (dirtyDataCounter.getCount()%rowLenth == 0){ + if (dirtyDataCounter.getCount() % dirtyDataFrequency == 0) { LOG.info("dirtyData: " + new String(message)); LOG.error("" , e); } From f1816929ac105c7531eab29c281a26d71410e512 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 3 Sep 2019 10:50:17 +0800 Subject: [PATCH 367/470] format print log --- .../sink/rdb/format/RetractJDBCOutputFormat.java | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 6b2529583..2e1733f35 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -53,6 +53,10 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private static final Logger LOG = LoggerFactory.getLogger(RetractJDBCOutputFormat.class); + private static int dirtyDataPrintFrequency = 1000; + + private static int receiveDataPrintFrequency = 1000; + private String username; private String password; private String drivername; @@ -167,8 +171,11 @@ public void writeRecord(Tuple2 tuple2) { } if (retract) { - insertWrite(row); outRecords.inc(); + if (outRecords.getCount() % receiveDataPrintFrequency == 0) { + LOG.info("Receive data : {}", row); + } + insertWrite(row); } else { //do nothing } @@ -200,8 +207,10 @@ private void writeSingleRecord(Row row) { upload.execute(); } catch (SQLException e) { outDirtyRecords.inc(); - LOG.error("record insert failed ..", row.toString()); - LOG.error("", e); + if (outDirtyRecords.getCount() % dirtyDataPrintFrequency == 0) { + LOG.error("record insert failed ..", row.toString()); + LOG.error("", e); + } } } @@ -297,7 +306,6 @@ private void updatePreparedStmt(Row row, PreparedStatement pstmt) throws SQLExce private synchronized void submitExecuteBatch() { try { - LOG.info("submitExecuteBatch start......"); this.upload.executeBatch(); dbConn.commit(); } catch (SQLException e) { From c6808f3de689566d5055108c44578a9667336214 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 3 Sep 2019 11:09:45 +0800 Subject: [PATCH 368/470] data output frequency --- .../dtstack/flink/sql/metric/MetricConstant.java | 2 ++ .../flink/sql/sink/MetricOutputFormat.java | 3 +++ .../sink/rdb/format/RetractJDBCOutputFormat.java | 15 +++++++++++---- 3 files changed, 16 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java b/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java index 89c411479..17bb75e82 100644 --- a/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java +++ b/core/src/main/java/com/dtstack/flink/sql/metric/MetricConstant.java @@ -45,6 +45,8 @@ public class MetricConstant { public static final String DT_NUM_RECORDS_OUT = "dtNumRecordsOut"; + public static final String DT_NUM_DIRTY_RECORDS_OUT = "dtNumDirtyRecordsOut"; + public static final String DT_NUM_RECORDS_OUT_RATE = "dtNumRecordsOutRate"; public static final String DT_EVENT_DELAY_GAUGE = "dtEventDelay"; diff --git a/core/src/main/java/com/dtstack/flink/sql/sink/MetricOutputFormat.java b/core/src/main/java/com/dtstack/flink/sql/sink/MetricOutputFormat.java index 4e11280e7..a220940ba 100644 --- a/core/src/main/java/com/dtstack/flink/sql/sink/MetricOutputFormat.java +++ b/core/src/main/java/com/dtstack/flink/sql/sink/MetricOutputFormat.java @@ -34,8 +34,11 @@ public abstract class MetricOutputFormat extends RichOutputFormat{ protected transient Meter outRecordsRate; + protected transient Counter outDirtyRecords; + public void initMetric() { outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + outDirtyRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_DIRTY_RECORDS_OUT); outRecordsRate = getRuntimeContext().getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(outRecords, 20)); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index c15b9f360..cb1654e39 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -49,6 +49,9 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private static final Logger LOG = LoggerFactory.getLogger(RetractJDBCOutputFormat.class); + private static int dirtyDataPrintFrequency = 1000; + private static int receiveDataPrintFrequency = 1000; + private String username; private String password; private String drivername; @@ -164,8 +167,11 @@ public void writeRecord(Tuple2 tuple2) { } if (retract) { - insertWrite(row); outRecords.inc(); + if (outRecords.getCount() % receiveDataPrintFrequency == 0) { + LOG.info("Receive data : {}", row); + } + insertWrite(row); } else { //do nothing } @@ -197,14 +203,15 @@ private void writeSingleRecord(Row row) { upload.execute(); dbConn.commit(); } catch (SQLException e) { - LOG.error("record insert failed ..", row.toString()); - LOG.error("", e); + if (outDirtyRecords.getCount() % dirtyDataPrintFrequency == 0) { + LOG.error("record insert failed ..", row.toString()); + LOG.error("", e); + } } } private synchronized void submitExecuteBatch() { try { - LOG.info("submitExecuteBatch start......"); this.upload.executeBatch(); dbConn.commit(); } catch (SQLException e) { From c04d18fd4f3710a2aecd5fb52937a1a54a9a41c7 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 3 Sep 2019 12:03:29 +0800 Subject: [PATCH 369/470] dirty record inc --- .../flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java | 1 + 1 file changed, 1 insertion(+) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index cb1654e39..0bf8a9614 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -203,6 +203,7 @@ private void writeSingleRecord(Row row) { upload.execute(); dbConn.commit(); } catch (SQLException e) { + outDirtyRecords.inc(); if (outDirtyRecords.getCount() % dirtyDataPrintFrequency == 0) { LOG.error("record insert failed ..", row.toString()); LOG.error("", e); From c88f951282d624c77da3298d69d97ed5112934cc Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 4 Sep 2019 12:02:56 +0800 Subject: [PATCH 370/470] field replace --- core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 092686a24..76ff8261d 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -363,7 +363,7 @@ private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable Date: Wed, 4 Sep 2019 17:35:26 +0800 Subject: [PATCH 371/470] 1.5 merge 3.6.1 --- core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 2 +- .../java/com/dtstack/flink/sql/sink/MetricOutputFormat.java | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index b1591e8dc..dde896484 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -436,7 +436,7 @@ private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable{ protected transient Meter outRecordsRate; - protected transient Counter outDirtyRecords; - public void initMetric() { outRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); outDirtyRecords = getRuntimeContext().getMetricGroup().counter(MetricConstant.DT_NUM_DIRTY_RECORDS_OUT); From 796b8e5f66bc416fd36744dece2fca0fbc37f8ff Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 4 Sep 2019 17:43:26 +0800 Subject: [PATCH 372/470] jdbc safe connect --- .../flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 7ef685774..7131ebdda 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.sink.rdb.format; import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.util.JDBCUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; @@ -136,7 +137,7 @@ public void open(int taskNumber, int numTasks) throws IOException { private Connection establishConnection() throws SQLException, ClassNotFoundException { Connection connection ; - Class.forName(drivername); + JDBCUtils.forName(drivername, getClass().getClassLoader()); if (username == null) { connection = DriverManager.getConnection(dbURL); } else { From 007b1fc8169ccff3d51b38ff24fa9c96e06703aa Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 6 Sep 2019 13:23:53 +0800 Subject: [PATCH 373/470] =?UTF-8?q?[=E6=94=AF=E6=8C=81=E5=AF=B9orderby?= =?UTF-8?q?=E5=AD=97=E6=AE=B5=E7=9A=84=E8=A7=A3=E6=9E=90][=E5=AF=B9orderby?= =?UTF-8?q?=E8=A7=A3=E6=9E=90=E7=BC=BA=E5=A4=B1][18642]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/parser/InsertSqlParser.java | 5 ++++ .../dtstack/flink/sql/side/SideSQLParser.java | 5 ++++ .../dtstack/flink/sql/side/SideSqlExec.java | 29 +++++++++++++++++++ 3 files changed, 39 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index 52541385f..ff2bb9e4b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -26,6 +26,7 @@ import org.apache.calcite.sql.SqlJoin; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOrderBy; import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; @@ -132,6 +133,10 @@ private static void parseNode(SqlNode sqlNode, SqlParseResult sqlParseResult){ parseNode(unionRight, sqlParseResult); } break; + case ORDER_BY: + SqlOrderBy sqlOrderBy = (SqlOrderBy) sqlNode; + parseNode(sqlOrderBy.query, sqlParseResult); + break; default: //do nothing break; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index e1c546240..e2d7c6984 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -31,6 +31,7 @@ import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlOrderBy; import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; @@ -120,6 +121,10 @@ private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue parseSql(unionRight, sideTableSet, queueInfo); break; + + case ORDER_BY: + SqlOrderBy sqlOrderBy = (SqlOrderBy) sqlNode; + parseSql(sqlOrderBy.query, sideTableSet, queueInfo); } return ""; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index dde896484..f3e1f4152 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -37,6 +37,7 @@ import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlOrderBy; import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.fun.SqlCase; import org.apache.calcite.sql.parser.SqlParseException; @@ -317,11 +318,39 @@ private void replaceFieldName(SqlNode sqlNode, HashBasedTable mappingTable, String tableAlias){ if(groupNode.getKind() == IDENTIFIER){ SqlIdentifier sqlIdentifier = (SqlIdentifier) groupNode; From 483223deaa5ad9cf8b993477276b0d7990f2b96b Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Sat, 7 Sep 2019 12:24:28 +0800 Subject: [PATCH 374/470] =?UTF-8?q?[=E5=BC=82=E6=AD=A5=E7=BB=B4=E8=A1=A8?= =?UTF-8?q?=E6=94=AF=E6=8C=81=E9=9D=9E=E7=AD=89=E5=80=BC=E8=BF=9E=E6=8E=A5?= =?UTF-8?q?]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flink/sql/side/SideInfo.java | 4 ++-- .../dtstack/flink/sql/side/SideSqlExec.java | 4 ++-- .../dtstack/flink/sql/util/ParseUtils.java | 24 +++++++++++++++++++ .../sql/side/oracle/OracleAsyncSideInfo.java | 8 +++++-- .../flink/sql/sink/oracle/OracleSink.java | 2 ++ .../sql/side/rdb/async/RdbAsyncSideInfo.java | 10 +++++--- 6 files changed, 43 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java index 063bfd2fd..97e5e555f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java @@ -109,8 +109,8 @@ public void parseSelectFields(JoinInfo joinInfo){ } public void dealOneEqualCon(SqlNode sqlNode, String sideTableName){ - if(sqlNode.getKind() != SqlKind.EQUALS){ - throw new RuntimeException("not equal operator."); + if(!SqlKind.COMPARISON.contains(sqlNode.getKind())){ + throw new RuntimeException("not compare operator."); } SqlIdentifier left = (SqlIdentifier)((SqlBasicCall)sqlNode).getOperands()[0]; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index bd08916f4..2caf91883 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -585,8 +585,8 @@ public List getConditionFields(SqlNode conditionNode, String specifyTabl ParseUtils.parseAnd(conditionNode, sqlNodeList); List conditionFields = Lists.newArrayList(); for(SqlNode sqlNode : sqlNodeList){ - if(sqlNode.getKind() != SqlKind.EQUALS){ - throw new RuntimeException("not equal operator."); + if (!SqlKind.COMPARISON.contains(sqlNode.getKind())) { + throw new RuntimeException("not compare operator."); } SqlIdentifier left = (SqlIdentifier)((SqlBasicCall)sqlNode).getOperands()[0]; diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java index 280594d17..25613d73d 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java @@ -3,8 +3,11 @@ import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; +import org.apache.commons.lang3.StringUtils; +import java.util.HashSet; import java.util.List; +import java.util.Set; /** * @Auther: jiangjunjie @@ -20,4 +23,25 @@ public static void parseAnd(SqlNode conditionNode, List sqlNodeList){ sqlNodeList.add(conditionNode); } } + + public static void parseJoinCompareOperate(SqlNode condition, List sqlJoinCompareOperate) { + SqlBasicCall joinCondition = (SqlBasicCall) condition; + if (joinCondition.getKind() == SqlKind.AND) { + List operandList = joinCondition.getOperandList(); + for (SqlNode sqlNode : operandList) { + parseJoinCompareOperate(sqlNode, sqlJoinCompareOperate); + } + } else { + String operator = parseOperator(joinCondition.getKind()); + sqlJoinCompareOperate.add(operator); + } + } + + public static String parseOperator(SqlKind sqlKind) { + if (StringUtils.equalsIgnoreCase(sqlKind.toString(), "NOT_EQUALS")){ + return "!="; + } + return sqlKind.sql; + } + } diff --git a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java index 61aa6dc14..2c87199f4 100644 --- a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java +++ b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java @@ -24,6 +24,7 @@ import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; +import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -48,7 +49,11 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { SqlNode conditionNode = joinInfo.getCondition(); List sqlNodeList = Lists.newArrayList(); + List sqlJoinCompareOperate= Lists.newArrayList(); + ParseUtils.parseAnd(conditionNode, sqlNodeList); + ParseUtils.parseJoinCompareOperate(conditionNode, sqlJoinCompareOperate); + for (SqlNode sqlNode : sqlNodeList) { dealOneEqualCon(sqlNode, sideTableName); @@ -57,8 +62,7 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { sqlCondition = "select ${selectField} from ${tableName} where "; for (int i = 0; i < equalFieldList.size(); i++) { String equalField = sideTableInfo.getPhysicalFields().getOrDefault(equalFieldList.get(i), equalFieldList.get(i)); - - sqlCondition += dealLowerFiled(equalField) + "=? "; + sqlCondition += dealLowerFiled(equalField) + " " + sqlJoinCompareOperate.get(i) + " " + " ?"; if (i != equalFieldList.size() - 1) { sqlCondition += " and "; } diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index d8669b11f..d899f21a3 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -71,6 +71,8 @@ private void buildInsertSql(String tableName, List fields) { placeholder = placeholder.replaceFirst(",", ""); sqlTmp = sqlTmp.replace("${fields}", fieldsStr).replace("${placeholder}", placeholder); this.sql = sqlTmp; + System.out.println("--------insert sqlTmp--------"); + System.out.println(sql); } @Override diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java index c35ad55d5..d19d8530f 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java @@ -58,7 +58,11 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { SqlNode conditionNode = joinInfo.getCondition(); List sqlNodeList = Lists.newArrayList(); + + List sqlJoinCompareOperate= Lists.newArrayList(); + ParseUtils.parseAnd(conditionNode, sqlNodeList); + ParseUtils.parseJoinCompareOperate(conditionNode, sqlJoinCompareOperate); for (SqlNode sqlNode : sqlNodeList) { dealOneEqualCon(sqlNode, sideTableName); @@ -68,7 +72,7 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { for (int i = 0; i < equalFieldList.size(); i++) { String equalField = sideTableInfo.getPhysicalFields().getOrDefault(equalFieldList.get(i), equalFieldList.get(i)); - sqlCondition += equalField + "=? "; + sqlCondition += equalField + "\t" + sqlJoinCompareOperate.get(i) + " ?"; if (i != equalFieldList.size() - 1) { sqlCondition += " and "; } @@ -81,8 +85,8 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { @Override public void dealOneEqualCon(SqlNode sqlNode, String sideTableName) { - if (sqlNode.getKind() != SqlKind.EQUALS) { - throw new RuntimeException("not equal operator."); + if (!SqlKind.COMPARISON.contains(sqlNode.getKind())) { + throw new RuntimeException("not compare operator."); } SqlIdentifier left = (SqlIdentifier) ((SqlBasicCall) sqlNode).getOperands()[0]; From ff3011814ce0cdba6efa1b6382f39228d7e84014 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Sat, 7 Sep 2019 13:53:56 +0800 Subject: [PATCH 375/470] =?UTF-8?q?[=E7=A7=BB=E9=99=A4SYSTEM=E8=BE=93?= =?UTF-8?q?=E5=87=BA]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java | 1 - .../main/java/com/dtstack/flink/sql/launcher/LauncherMain.java | 1 - .../main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java | 2 -- .../java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java | 1 - .../com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java | 1 - 5 files changed, 6 deletions(-) diff --git a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java index d6c816b97..99a286f9f 100644 --- a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java +++ b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java @@ -54,7 +54,6 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { sqlCondition = "select ${selectField} from ${tableName} "; sqlCondition = sqlCondition.replace("${tableName}", cassandraSideTableInfo.getDatabase() + "." + cassandraSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); - System.out.println("---------side_exe_sql-----\n" + sqlCondition); } @Override diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 61f82b528..415fbce18 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -100,7 +100,6 @@ public static void main(String[] args) throws Exception { clusterClient.shutdown(); } - System.out.println("---submit end----"); } private static String[] parseJson(String[] args) { diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index d899f21a3..d8669b11f 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -71,8 +71,6 @@ private void buildInsertSql(String tableName, List fields) { placeholder = placeholder.replaceFirst(",", ""); sqlTmp = sqlTmp.replace("${fields}", fieldsStr).replace("${placeholder}", placeholder); this.sql = sqlTmp; - System.out.println("--------insert sqlTmp--------"); - System.out.println(sql); } @Override diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java index f50126e4a..9ea79802d 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java @@ -56,7 +56,6 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { sqlCondition = "select ${selectField} from ${tableName} "; sqlCondition = sqlCondition.replace("${tableName}", rdbSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); - System.out.println("---------side_exe_sql-----\n" + sqlCondition); } @Override diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java index d19d8530f..3f3fb323d 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java @@ -79,7 +79,6 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { } sqlCondition = sqlCondition.replace("${tableName}", rdbSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); - System.out.println("---------side_exe_sql-----\n" + sqlCondition); } From 4503f4cf45346b644c0b303df4b9785396eed51e Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 9 Sep 2019 16:20:47 +0800 Subject: [PATCH 376/470] =?UTF-8?q?[=E6=94=AF=E6=8C=81=E5=AF=B9orderby?= =?UTF-8?q?=E5=AD=97=E6=AE=B5=E7=9A=84=E8=A7=A3=E6=9E=90][=E5=AF=B9orderby?= =?UTF-8?q?=E8=A7=A3=E6=9E=90=E7=BC=BA=E5=A4=B1][18642]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/parser/InsertSqlParser.java | 5 ++++ .../dtstack/flink/sql/side/SideSQLParser.java | 5 ++++ .../dtstack/flink/sql/side/SideSqlExec.java | 29 +++++++++++++++++++ 3 files changed, 39 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index 52541385f..ff2bb9e4b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -26,6 +26,7 @@ import org.apache.calcite.sql.SqlJoin; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOrderBy; import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; @@ -132,6 +133,10 @@ private static void parseNode(SqlNode sqlNode, SqlParseResult sqlParseResult){ parseNode(unionRight, sqlParseResult); } break; + case ORDER_BY: + SqlOrderBy sqlOrderBy = (SqlOrderBy) sqlNode; + parseNode(sqlOrderBy.query, sqlParseResult); + break; default: //do nothing break; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index cab400c99..aa7032754 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -31,6 +31,7 @@ import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlOrderBy; import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; @@ -120,6 +121,10 @@ private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue parseSql(unionRight, sideTableSet, queueInfo); break; + + case ORDER_BY: + SqlOrderBy sqlOrderBy = (SqlOrderBy) sqlNode; + parseSql(sqlOrderBy.query, sideTableSet, queueInfo); } return ""; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 76ff8261d..eaae4f0b2 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -35,6 +35,7 @@ import org.apache.calcite.sql.SqlLiteral; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlOrderBy; import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.fun.SqlCase; import org.apache.calcite.sql.parser.SqlParseException; @@ -244,11 +245,39 @@ private void replaceFieldName(SqlNode sqlNode, HashBasedTable mappingTable, String tableAlias){ if(groupNode.getKind() == IDENTIFIER){ SqlIdentifier sqlIdentifier = (SqlIdentifier) groupNode; From dba6bc524be183f537e9507f44495216b7c20319 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 11 Sep 2019 11:05:44 +0800 Subject: [PATCH 377/470] =?UTF-8?q?[=E5=86=B2=E7=AA=81=E8=A7=A3=E5=86=B3][?= =?UTF-8?q?=E4=BF=AE=E6=94=B9BatchNum=E5=90=8D=E7=A7=B0]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../rdb/format/RetractJDBCOutputFormat.java | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 0bf8a9614..198ce2120 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -63,7 +63,7 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private long batchWaitInterval = 10000l; // batchNum - private int batchInterval = 5000; + private int batchNum = 100; private String insertQuery; public int[] typesArray; @@ -98,7 +98,7 @@ public void configure(Configuration parameters) { @Override public void open(int taskNumber, int numTasks) throws IOException { try { - LOG.info("PreparedStatement execute batch num is {}", batchInterval); + LOG.info("PreparedStatement execute batch num is {}", batchNum); dbConn = establishConnection(); initMetric(); if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { @@ -110,7 +110,7 @@ public void open(int taskNumber, int numTasks) throws IOException { throw new SQLException("Table " + tableName + " doesn't exist"); } - if (batchWaitInterval > 0 && batchInterval > 1) { + if (batchWaitInterval > 0 && batchNum > 1) { LOG.info("open batch wait interval scheduled, interval is {} ms", batchWaitInterval); timerService = new ScheduledThreadPoolExecutor(1); @@ -182,13 +182,13 @@ public void writeRecord(Tuple2 tuple2) { private void insertWrite(Row row) { checkConnectionOpen(dbConn); try { - if (batchInterval == 1) { + if (batchNum == 1) { writeSingleRecord(row); } else { updatePreparedStmt(row, upload); rows.add(row); upload.addBatch(); - if (rows.size() >= batchInterval) { + if (rows.size() >= batchNum) { submitExecuteBatch(); } } @@ -413,9 +413,6 @@ public void setDbSink(RdbSink dbSink) { this.dbSink = dbSink; } - public void setBatchInterval(int batchInterval) { - this.batchInterval = batchInterval; - } public void setInsertQuery(String insertQuery) { this.insertQuery = insertQuery; @@ -445,6 +442,15 @@ public Map> getRealIndexes() { return realIndexes; } + + public void setBatchNum(int batchNum) { + this.batchNum = batchNum; + } + + public void setBatchWaitInterval(long batchWaitInterval) { + this.batchWaitInterval = batchWaitInterval; + } + public List getFullField() { return fullField; } From fcb97a029a6931f81c1e68bb54a8f62d955acfd8 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Mon, 16 Sep 2019 16:36:55 +0800 Subject: [PATCH 378/470] optimize parameters --- .../main/java/com/dtstack/flink/sql/Main.java | 44 +---- .../flink/sql/option/OptionParser.java | 120 ++++++++++++ .../flink/sql/option/OptionRequired.java | 42 ++++ .../com/dtstack/flink/sql/option/Options.java | 23 ++- .../dtstack/flink/sql/util/DtStringUtil.java | 4 + .../sql/launcher/ClusterClientFactory.java | 7 +- .../flink/sql/launcher/LauncherMain.java | 7 +- .../sql/launcher/LauncherOptionParser.java | 184 ------------------ .../sql/launcher/perjob/PerJobSubmitter.java | 4 +- 9 files changed, 204 insertions(+), 231 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/option/OptionRequired.java rename launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java => core/src/main/java/com/dtstack/flink/sql/option/Options.java (77%) delete mode 100644 launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index d321de41b..642734444 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -24,6 +24,7 @@ import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.environment.MyLocalStreamEnvironment; import com.dtstack.flink.sql.exec.FlinkSQLExec; +import com.dtstack.flink.sql.option.OptionParser; import com.dtstack.flink.sql.parser.CreateFuncParser; import com.dtstack.flink.sql.parser.CreateTmpTableParser; import com.dtstack.flink.sql.parser.InsertSqlParser; @@ -43,10 +44,6 @@ import org.apache.calcite.config.Lex; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlNode; -import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.CommandLineParser; -import org.apache.commons.cli.DefaultParser; -import org.apache.commons.cli.Options; import org.apache.commons.io.Charsets; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.restartstrategy.RestartStrategies; @@ -56,7 +53,6 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.client.program.ContextEnvironment; import org.apache.flink.configuration.Configuration; -import org.apache.flink.shaded.guava18.com.google.common.base.Preconditions; import org.apache.flink.shaded.guava18.com.google.common.base.Strings; import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; @@ -64,16 +60,13 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamContextEnvironment; - import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; - import org.apache.flink.table.sinks.TableSink; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.io.File; import java.io.IOException; import java.lang.reflect.Field; @@ -112,32 +105,15 @@ public class Main { public static void main(String[] args) throws Exception { - Options options = new Options(); - options.addOption("sql", true, "sql config"); - options.addOption("name", true, "job name"); - options.addOption("addjar", true, "add jar"); - options.addOption("localSqlPluginPath", true, "local sql plugin path"); - options.addOption("remoteSqlPluginPath", true, "remote sql plugin path"); - options.addOption("confProp", true, "env properties"); - options.addOption("mode", true, "deploy mode"); - - options.addOption("savePointPath", true, "Savepoint restore path"); - options.addOption("allowNonRestoredState", true, "Flag indicating whether non restored state is allowed if the savepoint"); - - CommandLineParser parser = new DefaultParser(); - CommandLine cl = parser.parse(options, args); - String sql = cl.getOptionValue("sql"); - String name = cl.getOptionValue("name"); - String addJarListStr = cl.getOptionValue("addjar"); - String localSqlPluginPath = cl.getOptionValue("localSqlPluginPath"); - String remoteSqlPluginPath = cl.getOptionValue("remoteSqlPluginPath"); - String deployMode = cl.getOptionValue("mode"); - String confProp = cl.getOptionValue("confProp"); - - Preconditions.checkNotNull(sql, "parameters of sql is required"); - Preconditions.checkNotNull(name, "parameters of name is required"); - Preconditions.checkNotNull(localSqlPluginPath, "parameters of localSqlPluginPath is required"); - + OptionParser optionParser = new OptionParser(args); + com.dtstack.flink.sql.option.Options options = optionParser.getOptions(); + String sql = options.getSql(); + String name = options.getName(); + String addJarListStr = options.getAddjar(); + String localSqlPluginPath = options.getLocalSqlPluginPath(); + String remoteSqlPluginPath = options.getRemoteSqlPluginPath(); + String deployMode = options.getMode(); + String confProp = options.getConfProp(); sql = URLDecoder.decode(sql, Charsets.UTF_8.name()); SqlParser.setLocalSqlPluginRoot(localSqlPluginPath); diff --git a/core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java b/core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java new file mode 100644 index 000000000..10e34a5e6 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java @@ -0,0 +1,120 @@ +/** + * 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 com.dtstack.flink.sql.option; + +import avro.shaded.com.google.common.collect.Lists; +import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.commons.cli.BasicParser; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.ParseException; +import org.apache.commons.lang.StringUtils; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Field; +import java.util.List; +import java.util.Map; +import java.io.File; +import java.io.FileInputStream; +import java.net.URLEncoder; +import org.apache.commons.codec.Charsets; + + +/** + * The Parser of Launcher commandline options + * + * Company: www.dtstack.com + * @author sishu.yss + */ +public class OptionParser { + + public static final String OPTION_SQL = "sql"; + + private org.apache.commons.cli.Options options = new org.apache.commons.cli.Options(); + + private BasicParser parser = new BasicParser(); + + private Options properties = new Options(); + + public OptionParser(String[] args) throws Exception { + initOptions(addOptions(args)); + } + + private CommandLine addOptions(String[] args) throws NoSuchMethodException, InvocationTargetException, IllegalAccessException, ParseException { + Class cla = properties.getClass(); + Field[] fields = cla.getDeclaredFields(); + for(Field field:fields){ + String name = field.getName(); + OptionRequired optionRequired = field.getAnnotation(OptionRequired.class); + if(optionRequired != null){ + options.addOption(name,optionRequired.hasArg(),optionRequired.description()); + } + } + CommandLine cl = parser.parse(options, args); + return cl; + } + + private void initOptions(CommandLine cl) throws NoSuchMethodException, InvocationTargetException, IllegalAccessException, ParseException { + Class cla = properties.getClass(); + Field[] fields = cla.getDeclaredFields(); + for(Field field:fields){ + String name = field.getName(); + String value = cl.getOptionValue(name); + OptionRequired optionRequired = field.getAnnotation(OptionRequired.class); + if(optionRequired != null){ + if(optionRequired.required()&&StringUtils.isBlank(value)){ + throw new RuntimeException(String.format("parameters of %s is required",name)); + } + } + if(StringUtils.isNotBlank(value)){ + field.setAccessible(true); + field.set(properties,value); + } + } + } + + public Options getOptions(){ + return properties; + } + + public List getProgramExeArgList() throws Exception { + Map mapConf = PluginUtil.ObjectToMap(properties); + List args = Lists.newArrayList(); + for(Map.Entry one : mapConf.entrySet()){ + String key = one.getKey(); + Object value = one.getValue(); + if(value == null){ + continue; + }else if(OPTION_SQL.equalsIgnoreCase(key)){ + File file = new File(value.toString()); + FileInputStream in = new FileInputStream(file); + byte[] filecontent = new byte[(int) file.length()]; + in.read(filecontent); + String content = new String(filecontent, Charsets.UTF_8.name()); + value = URLEncoder.encode(content, Charsets.UTF_8.name()); + } + args.add("-" + key); + args.add(value.toString()); + } + return args; + } + + public static void main(String[] args) throws Exception { + OptionParser OptionParser = new OptionParser(args); + System.out.println(OptionParser.getOptions()); + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/option/OptionRequired.java b/core/src/main/java/com/dtstack/flink/sql/option/OptionRequired.java new file mode 100644 index 000000000..b70c4424b --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/option/OptionRequired.java @@ -0,0 +1,42 @@ +/** + * 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 com.dtstack.flink.sql.option; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * + * Reason: TODO ADD REASON(可选) + * Date: 2019年9月16日 下午1:24:39 + * Company: www.dtstack.com + * @author sishu.yss + * + */ +@Target({ElementType.FIELD}) +@Retention(RetentionPolicy.RUNTIME) +public @interface OptionRequired { + + boolean required() default false; + + boolean hasArg() default true; + + String description() default ""; +} diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java b/core/src/main/java/com/dtstack/flink/sql/option/Options.java similarity index 77% rename from launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java rename to core/src/main/java/com/dtstack/flink/sql/option/Options.java index 571229510..0c33a8323 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptions.java +++ b/core/src/main/java/com/dtstack/flink/sql/option/Options.java @@ -16,44 +16,57 @@ * limitations under the License. */ -package com.dtstack.flink.sql.launcher; +package com.dtstack.flink.sql.option; import com.dtstack.flink.sql.ClusterMode; + /** * This class define commandline options for the Launcher program * * Company: www.dtstack.com * @author huyifan.zju@163.com */ -public class LauncherOptions { +public class Options { + @OptionRequired(description = "Running mode") private String mode = ClusterMode.local.name(); + @OptionRequired(required = true,description = "Job name") private String name; + @OptionRequired(required = true,description = "Job sql file") private String sql; + @OptionRequired(description = "Flink configuration directory") private String flinkconf; + @OptionRequired(description = "Yarn and Hadoop configuration directory") private String yarnconf; + @OptionRequired(required = true,description = "Sql local plugin root") private String localSqlPluginPath; + @OptionRequired(required = true,description = "Sql remote plugin root") private String remoteSqlPluginPath ; + @OptionRequired(description = "sql ext jar,eg udf jar") private String addjar; - private String confProp; + @OptionRequired(description = "sql ref prop,eg specify event time") + private String confProp = "{}"; + @OptionRequired(description = "Savepoint restore path") private String savePointPath; + @OptionRequired(description = "Flag indicating whether non restored state is allowed if the savepoint") private String allowNonRestoredState = "false"; - //just use for per_job mode + @OptionRequired(description = "flink jar path for submit of perjob mode") private String flinkJarPath; - private String queue; + @OptionRequired(description = "yarn queue") + private String queue = "default"; public String getMode() { return mode; diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index 8da889e20..e07f79a4d 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -247,4 +247,8 @@ public static Object parse(String str,Class clazz){ } return object; } + + public static String firstUpperCase(String str) { + return str.substring(0, 1).toUpperCase() + str.substring(1); + } } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 23f6efd10..72dbca3ee 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -18,6 +18,7 @@ package com.dtstack.flink.sql.launcher; +import com.dtstack.flink.sql.option.Options; import com.dtstack.flink.sql.util.PluginUtil; import com.dtstack.flink.yarn.JobParameter; import com.dtstack.flink.yarn.YarnClusterConfiguration; @@ -60,7 +61,7 @@ */ public class ClusterClientFactory { - public static ClusterClient createClusterClient(LauncherOptions launcherOptions) throws Exception { + public static ClusterClient createClusterClient(Options launcherOptions) throws Exception { String mode = launcherOptions.getMode(); if(mode.equals(ClusterMode.standalone.name())) { return createStandaloneClient(launcherOptions); @@ -70,7 +71,7 @@ public static ClusterClient createClusterClient(LauncherOptions launcherOptions) throw new IllegalArgumentException("Unsupported cluster client type: "); } - public static ClusterClient createStandaloneClient(LauncherOptions launcherOptions) throws Exception { + public static ClusterClient createStandaloneClient(Options launcherOptions) throws Exception { String flinkConfDir = launcherOptions.getFlinkconf(); Configuration config = GlobalConfiguration.loadConfiguration(flinkConfDir); MiniClusterConfiguration.Builder configBuilder = new MiniClusterConfiguration.Builder(); @@ -85,7 +86,7 @@ public static ClusterClient createStandaloneClient(LauncherOptions launcherOptio return clusterClient; } - public static ClusterClient createYarnClient(LauncherOptions launcherOptions,String mode) { + public static ClusterClient createYarnClient(Options launcherOptions, String mode) { String flinkConfDir = launcherOptions.getFlinkconf(); Configuration flinkConf = GlobalConfiguration.loadConfiguration(flinkConfDir); String yarnConfDir = launcherOptions.getYarnconf(); diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 415fbce18..33e897114 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -26,6 +26,8 @@ import com.dtstack.flink.sql.ClusterMode; import com.dtstack.flink.sql.Main; import com.dtstack.flink.sql.launcher.perjob.PerJobSubmitter; +import com.dtstack.flink.sql.option.OptionParser; +import com.dtstack.flink.sql.option.Options; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.lang.BooleanUtils; import org.apache.commons.lang.StringUtils; @@ -68,9 +70,8 @@ public static void main(String[] args) throws Exception { if (args.length == 1 && args[0].endsWith(".json")){ args = parseJson(args); } - - LauncherOptionParser optionParser = new LauncherOptionParser(args); - LauncherOptions launcherOptions = optionParser.getLauncherOptions(); + OptionParser optionParser = new OptionParser(args); + Options launcherOptions = optionParser.getOptions(); String mode = launcherOptions.getMode(); List argList = optionParser.getProgramExeArgList(); diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java deleted file mode 100644 index b66f9c32e..000000000 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherOptionParser.java +++ /dev/null @@ -1,184 +0,0 @@ -/** - * 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 com.dtstack.flink.sql.launcher; - -import avro.shaded.com.google.common.collect.Lists; -import com.dtstack.flink.sql.ClusterMode; -import com.dtstack.flink.sql.util.PluginUtil; -import org.apache.commons.cli.BasicParser; -import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.Options; -import org.apache.commons.lang.StringUtils; -import org.apache.flink.hadoop.shaded.com.google.common.base.Charsets; -import org.apache.flink.hadoop.shaded.com.google.common.base.Preconditions; - -import java.io.File; -import java.io.FileInputStream; -import java.net.URLEncoder; -import java.util.List; -import java.util.Map; - - -/** - * The Parser of Launcher commandline options - * - * Company: www.dtstack.com - * @author sishu.yss - */ -public class LauncherOptionParser { - - public static final String OPTION_MODE = "mode"; - - public static final String OPTION_NAME = "name"; - - public static final String OPTION_SQL = "sql"; - - public static final String OPTION_FLINK_CONF_DIR = "flinkconf"; - - public static final String OPTION_YARN_CONF_DIR = "yarnconf"; - - public static final String OPTION_LOCAL_SQL_PLUGIN_PATH = "localSqlPluginPath"; - - public static final String OPTION_REMOTE_SQL_PLUGIN_PATH = "remoteSqlPluginPath"; - - public static final String OPTION_ADDJAR = "addjar"; - - public static final String OPTION_CONF_PROP = "confProp"; - - public static final String OPTION_SAVE_POINT_PATH = "savePointPath"; - - public static final String OPTION_ALLOW_NON_RESTORED_STATE = "allowNonRestoredState"; - - public static final String OPTION_FLINK_JAR_PATH = "flinkJarPath"; - - public static final String OPTION_QUEUE = "queue"; - - private Options options = new Options(); - - private BasicParser parser = new BasicParser(); - - private LauncherOptions properties = new LauncherOptions(); - - public LauncherOptionParser(String[] args) throws Exception { - options.addOption(OPTION_MODE, true, "Running mode"); - options.addOption(OPTION_SQL, true, "Job sql file"); - options.addOption(OPTION_NAME, true, "Job name"); - options.addOption(OPTION_FLINK_CONF_DIR, true, "Flink configuration directory"); - options.addOption(OPTION_LOCAL_SQL_PLUGIN_PATH, true, "sql local plugin root"); - options.addOption(OPTION_REMOTE_SQL_PLUGIN_PATH, true, "sql remote plugin root"); - options.addOption(OPTION_ADDJAR, true, "sql ext jar,eg udf jar"); - options.addOption(OPTION_CONF_PROP, true, "sql ref prop,eg specify event time"); - options.addOption(OPTION_YARN_CONF_DIR, true, "Yarn and hadoop configuration directory"); - - options.addOption(OPTION_SAVE_POINT_PATH, true, "Savepoint restore path"); - options.addOption(OPTION_ALLOW_NON_RESTORED_STATE, true, "Flag indicating whether non restored state is allowed if the savepoint"); - options.addOption(OPTION_FLINK_JAR_PATH, true, "flink jar path for submit of perjob mode"); - - try { - CommandLine cl = parser.parse(options, args); - String mode = cl.getOptionValue(OPTION_MODE, ClusterMode.local.name()); - //check mode - properties.setMode(mode); - - String job = Preconditions.checkNotNull(cl.getOptionValue(OPTION_SQL), - "Must specify job file using option '" + OPTION_SQL + "'"); - File file = new File(job); - FileInputStream in = new FileInputStream(file); - byte[] filecontent = new byte[(int) file.length()]; - in.read(filecontent); - String content = new String(filecontent, "UTF-8"); - String sql = URLEncoder.encode(content, Charsets.UTF_8.name()); - properties.setSql(sql); - String localPlugin = Preconditions.checkNotNull(cl.getOptionValue(OPTION_LOCAL_SQL_PLUGIN_PATH)); - properties.setLocalSqlPluginPath(localPlugin); - String remotePlugin = cl.getOptionValue(OPTION_REMOTE_SQL_PLUGIN_PATH); -// if(!ClusterMode.local.name().equals(mode)){ -// Preconditions.checkNotNull(remotePlugin); - properties.setRemoteSqlPluginPath(remotePlugin); -// } - String name = Preconditions.checkNotNull(cl.getOptionValue(OPTION_NAME)); - properties.setName(name); - String addJar = cl.getOptionValue(OPTION_ADDJAR); - if(StringUtils.isNotBlank(addJar)){ - properties.setAddjar(addJar); - } - String confProp = cl.getOptionValue(OPTION_CONF_PROP); - if(StringUtils.isNotBlank(confProp)){ - properties.setConfProp(confProp); - } - String flinkConfDir = cl.getOptionValue(OPTION_FLINK_CONF_DIR); - if(StringUtils.isNotBlank(flinkConfDir)) { - properties.setFlinkconf(flinkConfDir); - } - - String yarnConfDir = cl.getOptionValue(OPTION_YARN_CONF_DIR); - if(StringUtils.isNotBlank(yarnConfDir)) { - properties.setYarnconf(yarnConfDir); - } - - String savePointPath = cl.getOptionValue(OPTION_SAVE_POINT_PATH); - if(StringUtils.isNotBlank(savePointPath)) { - properties.setSavePointPath(savePointPath); - } - - String allow_non = cl.getOptionValue(OPTION_ALLOW_NON_RESTORED_STATE); - if(StringUtils.isNotBlank(allow_non)) { - properties.setAllowNonRestoredState(allow_non); - } - - String flinkJarPath = cl.getOptionValue(OPTION_FLINK_JAR_PATH); - if(StringUtils.isNotBlank(flinkJarPath)){ - properties.setFlinkJarPath(flinkJarPath); - } - - String queue = cl.getOptionValue(OPTION_QUEUE); - if(StringUtils.isNotBlank(queue)){ - properties.setQueue(queue); - } - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - public LauncherOptions getLauncherOptions(){ - return properties; - } - - public List getProgramExeArgList() throws Exception { - Map mapConf = PluginUtil.ObjectToMap(properties); - List args = Lists.newArrayList(); - for(Map.Entry one : mapConf.entrySet()){ - String key = one.getKey(); - if(OPTION_FLINK_CONF_DIR.equalsIgnoreCase(key) - || OPTION_YARN_CONF_DIR.equalsIgnoreCase(key) - || OPTION_FLINK_JAR_PATH.equalsIgnoreCase(key) - || OPTION_QUEUE.equalsIgnoreCase(key)){ - continue; - } - - if(one.getValue() == null){ - continue; - } - - args.add("-" + key); - args.add(one.getValue().toString()); - } - return args; - } -} diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java index 91550404c..cb5d8482d 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.launcher.perjob; -import com.dtstack.flink.sql.launcher.LauncherOptions; +import com.dtstack.flink.sql.option.Options; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.io.Charsets; import org.apache.commons.lang3.StringUtils; @@ -49,7 +49,7 @@ public class PerJobSubmitter { private static final Logger LOG = LoggerFactory.getLogger(PerJobSubmitter.class); - public static String submit(LauncherOptions launcherOptions, JobGraph jobGraph) throws Exception { + public static String submit(Options launcherOptions, JobGraph jobGraph) throws Exception { fillJobGraphClassPath(jobGraph); if (!StringUtils.isBlank(launcherOptions.getAddjar())) { From f5e9ed80114baa21c633dff0ffd9aa9648d1306e Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 17 Sep 2019 12:39:46 +0800 Subject: [PATCH 379/470] =?UTF-8?q?[RDB=E8=BF=9E=E6=8E=A5=E8=B6=85?= =?UTF-8?q?=E6=97=B6=E6=A3=80=E6=9F=A5=E5=8F=8A=E9=87=8D=E8=BF=9E]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../sql/side/mysql/MysqlAsyncReqRow.java | 8 +- .../sql/side/oracle/OracleAsyncReqRow.java | 6 +- .../sql/side/rdb/async/RdbAsyncReqRow.java | 8 ++ .../provider/DTC3P0DataSourceProvider.java | 95 +++++++++++++++++++ .../dtstack/flink/sql/sink/rdb/RdbSink.java | 27 ++++-- .../rdb/format/RetractJDBCOutputFormat.java | 19 ++-- .../sql/sink/rdb/table/RdbTableInfo.java | 12 +++ .../side/sqlserver/SqlserverAsyncReqRow.java | 6 +- 8 files changed, 161 insertions(+), 20 deletions(-) create mode 100644 rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/provider/DTC3P0DataSourceProvider.java diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java index 39fb6ac56..51f311276 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -63,7 +63,13 @@ public void open(Configuration parameters) throws Exception { .put("driver_class", MYSQL_DRIVER) .put("max_pool_size", DEFAULT_MAX_DB_CONN_POOL_SIZE) .put("user", rdbSideTableInfo.getUserName()) - .put("password", rdbSideTableInfo.getPassword()); + .put("password", rdbSideTableInfo.getPassword()) + .put("provider_class", DT_PROVIDER_CLASS) + .put("preferred_test_query", PREFERRED_TEST_QUERY_SQL) + .put("idle_connection_test_period", DEFAULT_IDLE_CONNECTION_TEST_PEROID) + .put("test_connection_on_checkin", DEFAULT_TEST_CONNECTION_ON_CHECKIN); + + VertxOptions vo = new VertxOptions(); vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); diff --git a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java index c7f42a8dd..e1d2fc69e 100644 --- a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java +++ b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java @@ -55,7 +55,11 @@ public void open(Configuration parameters) throws Exception { .put("driver_class", ORACLE_DRIVER) .put("max_pool_size", DEFAULT_MAX_DB_CONN_POOL_SIZE) .put("user", rdbSideTableInfo.getUserName()) - .put("password", rdbSideTableInfo.getPassword()); + .put("password", rdbSideTableInfo.getPassword()) + .put("provider_class", DT_PROVIDER_CLASS) + .put("preferred_test_query", PREFERRED_TEST_QUERY_SQL) + .put("idle_connection_test_period", DEFAULT_IDLE_CONNECTION_TEST_PEROID) + .put("test_connection_on_checkin", DEFAULT_TEST_CONNECTION_ON_CHECKIN);; VertxOptions vo = new VertxOptions(); vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index e683a40a8..e7612b721 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -57,6 +57,14 @@ public class RdbAsyncReqRow extends AsyncReqRow { public final static int DEFAULT_MAX_DB_CONN_POOL_SIZE = DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE + DEFAULT_VERTX_WORKER_POOL_SIZE; + public final static int DEFAULT_IDLE_CONNECTION_TEST_PEROID = 60; + + public final static boolean DEFAULT_TEST_CONNECTION_ON_CHECKIN = true; + + public final static String DT_PROVIDER_CLASS = "com.dtstack.flink.sql.side.rdb.provider.DTC3P0DataSourceProvider"; + + public final static String PREFERRED_TEST_QUERY_SQL = "select 1 from dual"; + private transient SQLClient rdbSQLClient; public RdbAsyncReqRow(SideInfo sideInfo) { diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/provider/DTC3P0DataSourceProvider.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/provider/DTC3P0DataSourceProvider.java new file mode 100644 index 000000000..d4f5e7032 --- /dev/null +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/provider/DTC3P0DataSourceProvider.java @@ -0,0 +1,95 @@ +package com.dtstack.flink.sql.side.rdb.provider; + +import com.mchange.v2.c3p0.ComboPooledDataSource; +import io.vertx.core.json.JsonObject; +import io.vertx.ext.jdbc.spi.impl.C3P0DataSourceProvider; + +import javax.sql.DataSource; +import java.beans.PropertyVetoException; +import java.sql.SQLException; + +public class DTC3P0DataSourceProvider extends C3P0DataSourceProvider { + + @Override + public DataSource getDataSource(JsonObject config) throws SQLException { + String url = config.getString("url"); + if (url == null) throw new NullPointerException("url cannot be null"); + String driverClass = config.getString("driver_class"); + String user = config.getString("user"); + String password = config.getString("password"); + Integer maxPoolSize = config.getInteger("max_pool_size"); + Integer initialPoolSize = config.getInteger("initial_pool_size"); + Integer minPoolSize = config.getInteger("min_pool_size"); + Integer maxStatements = config.getInteger("max_statements"); + Integer maxStatementsPerConnection = config.getInteger("max_statements_per_connection"); + Integer maxIdleTime = config.getInteger("max_idle_time"); + Integer acquireRetryAttempts = config.getInteger("acquire_retry_attempts"); + Integer acquireRetryDelay = config.getInteger("acquire_retry_delay"); + Boolean breakAfterAcquireFailure = config.getBoolean("break_after_acquire_failure"); + + //add c3p0 params + String preferredTestQuery = config.getString("preferred_test_query"); + Integer idleConnectionTestPeriod = config.getInteger("idle_connection_test_period"); + Boolean testConnectionOnCheckin = config.getBoolean("test_connection_on_checkin"); + + + + // If you want to configure any other C3P0 properties you can add a file c3p0.properties to the classpath + ComboPooledDataSource cpds = new ComboPooledDataSource(); + cpds.setJdbcUrl(url); + if (driverClass != null) { + try { + cpds.setDriverClass(driverClass); + } catch (PropertyVetoException e) { + throw new IllegalArgumentException(e); + } + } + if (user != null) { + cpds.setUser(user); + } + if (password != null) { + cpds.setPassword(password); + } + if (maxPoolSize != null) { + cpds.setMaxPoolSize(maxPoolSize); + } + if (minPoolSize != null) { + cpds.setMinPoolSize(minPoolSize); + } + if (initialPoolSize != null) { + cpds.setInitialPoolSize(initialPoolSize); + } + if (maxStatements != null) { + cpds.setMaxStatements(maxStatements); + } + if (maxStatementsPerConnection != null) { + cpds.setMaxStatementsPerConnection(maxStatementsPerConnection); + } + if (maxIdleTime != null) { + cpds.setMaxIdleTime(maxIdleTime); + } + if(acquireRetryAttempts != null){ + cpds.setAcquireRetryAttempts(acquireRetryAttempts); + } + if(acquireRetryDelay != null){ + cpds.setAcquireRetryDelay(acquireRetryDelay); + } + if(breakAfterAcquireFailure != null){ + cpds.setBreakAfterAcquireFailure(breakAfterAcquireFailure); + } + + if (preferredTestQuery != null) { + cpds.setPreferredTestQuery(preferredTestQuery); + } + + if (idleConnectionTestPeriod != null) { + cpds.setIdleConnectionTestPeriod(idleConnectionTestPeriod); + } + + if (testConnectionOnCheckin != null) { + cpds.setTestConnectionOnCheckin(testConnectionOnCheckin); + } + + return cpds; + } +} diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java index 5ee2c44c4..c1401c745 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java @@ -61,7 +61,9 @@ public abstract class RdbSink implements RetractStreamTableSink, Serializab protected String dbType; - protected int batchInterval = 1; + protected int batchNum = 100; + + protected long batchWaitInterval = 10000; protected int[] sqlTypes; @@ -89,7 +91,8 @@ public RichSinkFunction createJdbcSinkFunc() { outputFormat.setUsername(userName); outputFormat.setPassword(password); outputFormat.setInsertQuery(sql); - outputFormat.setBatchInterval(batchInterval); + outputFormat.setBatchNum(batchNum); + outputFormat.setBatchWaitInterval(batchWaitInterval); outputFormat.setTypesArray(sqlTypes); outputFormat.setTableName(tableName); outputFormat.setDbType(dbType); @@ -112,7 +115,12 @@ public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { Integer tmpSqlBatchSize = rdbTableInfo.getBatchSize(); if (tmpSqlBatchSize != null) { - setBatchInterval(tmpSqlBatchSize); + setBatchNum(tmpSqlBatchSize); + } + + Long batchWaitInterval = rdbTableInfo.getBatchWaitInterval(); + if (batchWaitInterval != null) { + setBatchWaitInterval(batchWaitInterval); } Integer tmpSinkParallelism = rdbTableInfo.getParallelism(); @@ -198,13 +206,12 @@ public TableSink> configure(String[] fieldNames, TypeInform return this; } - /** - * Set the default frequency submit updated every submission - * - * @param batchInterval - */ - public void setBatchInterval(int batchInterval) { - this.batchInterval = batchInterval; + public void setBatchNum(int batchNum) { + this.batchNum = batchNum; + } + + public void setBatchWaitInterval(long batchWaitInterval) { + this.batchWaitInterval = batchWaitInterval; } @Override diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 0bf8a9614..a2a173a47 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -63,7 +63,7 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private long batchWaitInterval = 10000l; // batchNum - private int batchInterval = 5000; + private int batchNum = 100; private String insertQuery; public int[] typesArray; @@ -98,7 +98,7 @@ public void configure(Configuration parameters) { @Override public void open(int taskNumber, int numTasks) throws IOException { try { - LOG.info("PreparedStatement execute batch num is {}", batchInterval); + LOG.info("PreparedStatement execute batch num is {}", batchNum); dbConn = establishConnection(); initMetric(); if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { @@ -110,7 +110,7 @@ public void open(int taskNumber, int numTasks) throws IOException { throw new SQLException("Table " + tableName + " doesn't exist"); } - if (batchWaitInterval > 0 && batchInterval > 1) { + if (batchWaitInterval > 0 && batchNum > 1) { LOG.info("open batch wait interval scheduled, interval is {} ms", batchWaitInterval); timerService = new ScheduledThreadPoolExecutor(1); @@ -182,13 +182,13 @@ public void writeRecord(Tuple2 tuple2) { private void insertWrite(Row row) { checkConnectionOpen(dbConn); try { - if (batchInterval == 1) { + if (batchNum == 1) { writeSingleRecord(row); } else { updatePreparedStmt(row, upload); rows.add(row); upload.addBatch(); - if (rows.size() >= batchInterval) { + if (rows.size() >= batchNum) { submitExecuteBatch(); } } @@ -234,6 +234,7 @@ private void checkConnectionOpen(Connection dbConn) { LOG.info("db connection reconnect.."); dbConn= establishConnection(); upload = dbConn.prepareStatement(insertQuery); + this.dbConn = dbConn; } } catch (SQLException e) { LOG.error("check connection open failed..", e); @@ -413,8 +414,12 @@ public void setDbSink(RdbSink dbSink) { this.dbSink = dbSink; } - public void setBatchInterval(int batchInterval) { - this.batchInterval = batchInterval; + public void setBatchNum(int batchNum) { + this.batchNum = batchNum; + } + + public void setBatchWaitInterval(long batchWaitInterval) { + this.batchWaitInterval = batchWaitInterval; } public void setInsertQuery(String insertQuery) { diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java index 9c2bbc9e1..6b505c837 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java @@ -41,6 +41,8 @@ public class RdbTableInfo extends TargetTableInfo { public static final String BUFFER_SIZE_KEY = "bufferSize"; + public static final String BATCH_WAIT_INTERVAL_KEY = "batchWaitInterval"; + public static final String FLUSH_INTERVALMS_KEY = "flushIntervalMs"; private String url; @@ -53,6 +55,8 @@ public class RdbTableInfo extends TargetTableInfo { private Integer batchSize; + private Long batchWaitInterval; + private String bufferSize; private String flushIntervalMs; @@ -113,6 +117,14 @@ public void setFlushIntervalMs(String flushIntervalMs) { this.flushIntervalMs = flushIntervalMs; } + public Long getBatchWaitInterval() { + return batchWaitInterval; + } + + public void setBatchWaitInterval(Long batchWaitInterval) { + this.batchWaitInterval = batchWaitInterval; + } + @Override public boolean check() { Preconditions.checkNotNull(url, "rdb field of URL is required"); diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java index 8abae552e..9fee31171 100644 --- a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java @@ -53,7 +53,11 @@ public void open(Configuration parameters) throws Exception { .put("driver_class", SQLSERVER_DRIVER) .put("max_pool_size", DEFAULT_MAX_DB_CONN_POOL_SIZE) .put("user", rdbSideTableInfo.getUserName()) - .put("password", rdbSideTableInfo.getPassword()); + .put("password", rdbSideTableInfo.getPassword()) + .put("provider_class", DT_PROVIDER_CLASS) + .put("preferred_test_query", PREFERRED_TEST_QUERY_SQL) + .put("idle_connection_test_period", DEFAULT_IDLE_CONNECTION_TEST_PEROID) + .put("test_connection_on_checkin", DEFAULT_TEST_CONNECTION_ON_CHECKIN);; VertxOptions vo = new VertxOptions(); vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); From 386efb35cb723e1e23d8ed8f32f290c1d8a03425 Mon Sep 17 00:00:00 2001 From: toutian <953372946@qq.com> Date: Tue, 17 Sep 2019 16:38:40 +0800 Subject: [PATCH 380/470] =?UTF-8?q?[kafka=20not=20null][flinkStreamSQL?= =?UTF-8?q?=E6=94=AF=E6=8C=81not=20null=20=E8=AF=AD=E6=B3=95][17872]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- cassandra/cassandra-sink/pom.xml | 2 +- console/console-sink/pom.xml | 2 +- .../main/java/com/dtstack/flink/sql/Main.java | 22 +++--------- .../dtstack/flink/sql/parser/SqlParser.java | 1 + .../flink/sql/table/AbsTableParser.java | 1 + .../dtstack/flink/sql/table/TableInfo.java | 31 +++++++++++++++++ .../com/dtstack/flink/sql/util/FlinkUtil.java | 13 +------ hbase/hbase-sink/pom.xml | 2 +- .../kafka/CustomerJsonDeserialization.java | 19 ++++------- .../flink/sql/source/kafka/KafkaSource.java | 5 ++- .../source/kafka/table/KafkaSourceParser.java | 9 +++-- launcher/job/kafkaNotNull.txt | 34 +++++++++++++++++++ serversocket/serversocket-source/pom.xml | 2 +- 13 files changed, 92 insertions(+), 51 deletions(-) create mode 100644 launcher/job/kafkaNotNull.txt diff --git a/cassandra/cassandra-sink/pom.xml b/cassandra/cassandra-sink/pom.xml index 4fb20c373..b60ff2e3f 100644 --- a/cassandra/cassandra-sink/pom.xml +++ b/cassandra/cassandra-sink/pom.xml @@ -71,7 +71,7 @@ + tofile="${basedir}/../../plugins/cassandrasink/${project.name}.jar-${git.branch}.jar" /> diff --git a/console/console-sink/pom.xml b/console/console-sink/pom.xml index a5ca2629f..0579f68cf 100644 --- a/console/console-sink/pom.xml +++ b/console/console-sink/pom.xml @@ -67,7 +67,7 @@ + tofile="${basedir}/../../plugins/consolesink/${project.name}.jar-${git.branch}.jar" /> diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index d321de41b..0cecdaa61 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -78,7 +78,6 @@ import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; import java.net.URL; import java.net.URLClassLoader; import java.net.URLDecoder; @@ -251,14 +250,11 @@ private static void addEnvClassPath(StreamExecutionEnvironment env, Set cla private static void registerUDF(SqlTree sqlTree, List jarURList, URLClassLoader parentClassloader, StreamTableEnvironment tableEnv) throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException { + //load jar + URLClassLoader classLoader = FlinkUtil.loadExtraJar(jarURList, parentClassloader); //register urf - URLClassLoader classLoader = null; List funcList = sqlTree.getFunctionList(); for (CreateFuncParser.SqlParserResult funcInfo : funcList) { - //classloader - if (classLoader == null) { - classLoader = FlinkUtil.loadExtraJar(jarURList, parentClassloader); - } FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName(), tableEnv, classLoader); } @@ -335,19 +331,9 @@ private static StreamExecutionEnvironment getStreamExeEnv(Properties confPropert new MyLocalStreamEnvironment(); env.getConfig().disableClosureCleaner(); env.setParallelism(FlinkUtil.getEnvParallelism(confProperties)); + Configuration globalJobParameters = new Configuration(); - Method method = Configuration.class.getDeclaredMethod("setValueInternal", String.class, Object.class); - method.setAccessible(true); - - confProperties.forEach((key,val) -> { - try { - method.invoke(globalJobParameters, key, val); - } catch (IllegalAccessException e) { - e.printStackTrace(); - } catch (InvocationTargetException e) { - e.printStackTrace(); - } - }); + globalJobParameters.addAllToProperties(confProperties); ExecutionConfig exeConfig = env.getConfig(); if(exeConfig.getGlobalJobParameters() == null){ diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java index 0f9f8ffd7..e9fb68cfe 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java @@ -88,6 +88,7 @@ public static SqlTree parseSql(String sql) throws Exception { sqlParser.parseSql(childSql, sqlTree); result = true; + break; } if(!result){ diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index 7eee0490e..fc29932bf 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -105,6 +105,7 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ tableInfo.addField(fieldName); tableInfo.addFieldClass(fieldClass); tableInfo.addFieldType(fieldType); + tableInfo.addFieldExtraInfo(null); } tableInfo.finish(); diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java index 57f94e2e0..491b7bca3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java @@ -57,6 +57,8 @@ public abstract class TableInfo implements Serializable { private final List fieldClassList = Lists.newArrayList(); + private final List fieldExtraInfoList = Lists.newArrayList(); + private List primaryKeys; private Integer parallelism = 1; @@ -143,6 +145,10 @@ public void setFieldClasses(Class[] fieldClasses) { this.fieldClasses = fieldClasses; } + public void addFieldExtraInfo(FieldExtraInfo extraInfo) { + fieldExtraInfoList.add(extraInfo); + } + public List getFieldList() { return fieldList; } @@ -159,6 +165,10 @@ public Map getPhysicalFields() { return physicalFields; } + public List getFieldExtraInfoList() { + return fieldExtraInfoList; + } + public void setPhysicalFields(Map physicalFields) { this.physicalFields = physicalFields; } @@ -168,4 +178,25 @@ public void finish(){ this.fieldClasses = fieldClassList.toArray(new Class[fieldClassList.size()]); this.fieldTypes = fieldTypeList.toArray(new String[fieldTypeList.size()]); } + + /** + * field extra info,used to store `not null` `default 0`..., + * + * now, only support not null + */ + public static class FieldExtraInfo implements Serializable { + + /** + * default false:allow field is null + */ + boolean notNull; + + public boolean getNotNull() { + return notNull; + } + + public void setNotNull(boolean notNull) { + this.notNull = notNull; + } + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java index 89476380e..2dc2a0bf3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java @@ -136,6 +136,7 @@ public static void setStreamTimeCharacteristic(StreamExecutionEnvironment env, P if(characteristicStr.equalsIgnoreCase(tmp.toString())){ env.setStreamTimeCharacteristic(tmp); flag = true; + break; } } @@ -244,21 +245,9 @@ public static long getBufferTimeoutMillis(Properties properties){ } public static URLClassLoader loadExtraJar(List jarURLList, URLClassLoader classLoader) throws NoSuchMethodException, IllegalAccessException, InvocationTargetException { - - int size = 0; - for(URL url : jarURLList){ - if(url.toString().endsWith(".jar")){ - size++; - } - } - - URL[] urlArray = new URL[size]; - int i=0; for(URL url : jarURLList){ if(url.toString().endsWith(".jar")){ - urlArray[i] = url; urlClassLoaderAddUrl(classLoader, url); - i++; } } diff --git a/hbase/hbase-sink/pom.xml b/hbase/hbase-sink/pom.xml index 26ead3574..132739f2f 100644 --- a/hbase/hbase-sink/pom.xml +++ b/hbase/hbase-sink/pom.xml @@ -68,7 +68,7 @@ + tofile="${basedir}/../../plugins/hbasesink/${project.name}.jar-${git.branch}.jar" /> diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 1002731e8..c6ce5fefe 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.source.AbsDeserialization; import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import com.dtstack.flink.sql.table.TableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -50,6 +51,7 @@ import java.sql.Time; import java.sql.Timestamp; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Set; @@ -83,9 +85,6 @@ public class CustomerJsonDeserialization extends AbsDeserialization { /** Types to parse fields as. Indices match fieldNames indices. */ private final TypeInformation[] fieldTypes; - /** Flag indicating whether to fail on a missing field. */ - private boolean failOnMissingField; - private AbstractFetcher fetcher; private boolean firstMsg = true; @@ -94,15 +93,14 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private Map rowAndFieldMapping; + private List fieldExtraInfos; - public CustomerJsonDeserialization(TypeInformation typeInfo, Map rowAndFieldMapping){ + public CustomerJsonDeserialization(TypeInformation typeInfo, Map rowAndFieldMapping, List fieldExtraInfos){ this.typeInfo = typeInfo; - this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); - this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); - this.rowAndFieldMapping= rowAndFieldMapping; + this.fieldExtraInfos = fieldExtraInfos; } @Override @@ -133,9 +131,10 @@ public Row deserialize(byte[] message) throws IOException { for (int i = 0; i < fieldNames.length; i++) { JsonNode node = getIgnoreCase(fieldNames[i]); + TableInfo.FieldExtraInfo fieldExtraInfo = fieldExtraInfos.get(i); if (node == null) { - if (failOnMissingField) { + if (fieldExtraInfo != null && fieldExtraInfo.getNotNull()) { throw new IllegalStateException("Failed to find field with name '" + fieldNames[i] + "'."); } else { @@ -180,10 +179,6 @@ public JsonNode getIgnoreCase(String key) { return node; } - public void setFailOnMissingField(boolean failOnMissingField) { - this.failOnMissingField = failOnMissingField; - } - private void parseTree(JsonNode jsonNode, String prefix){ Iterator iterator = jsonNode.fieldNames(); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 54e4190ac..10a5615d5 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -89,13 +89,12 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv FlinkKafkaConsumer011 kafkaSrc; if (BooleanUtils.isTrue(kafka011SourceTableInfo.getTopicIsPattern())) { kafkaSrc = new CustomerKafka011Consumer(Pattern.compile(topicName), - new CustomerJsonDeserialization(typeInformation, kafka011SourceTableInfo.getPhysicalFields()), props); + new CustomerJsonDeserialization(typeInformation, kafka011SourceTableInfo.getPhysicalFields(), kafka011SourceTableInfo.getFieldExtraInfoList()), props); } else { kafkaSrc = new CustomerKafka011Consumer(topicName, - new CustomerJsonDeserialization(typeInformation, kafka011SourceTableInfo.getPhysicalFields()), props); + new CustomerJsonDeserialization(typeInformation, kafka011SourceTableInfo.getPhysicalFields(), kafka011SourceTableInfo.getFieldExtraInfoList()), props); } - //earliest,latest if ("earliest".equalsIgnoreCase(kafka011SourceTableInfo.getOffsetReset())) { kafkaSrc.setStartFromEarliest(); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 85f6cd189..4d5cce30c 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -44,7 +44,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)(\\s+AS\\s+(\\w+))?(\\s+NOT\\s+NULL)?$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); @@ -60,13 +60,18 @@ public class KafkaSourceParser extends AbsSourceParser { static void dealNestField(Matcher matcher, TableInfo tableInfo) { String physicalField = matcher.group(1); String fieldType = matcher.group(3); - String mappingField = matcher.group(4); + String mappingField = matcher.group(5); + mappingField = mappingField != null ? mappingField : physicalField; Class fieldClass= ClassUtil.stringConvertClass(fieldType); + boolean notNull = matcher.group(6) != null; + TableInfo.FieldExtraInfo fieldExtraInfo = new TableInfo.FieldExtraInfo(); + fieldExtraInfo.setNotNull(notNull); tableInfo.addPhysicalMappings(mappingField, physicalField); tableInfo.addField(mappingField); tableInfo.addFieldClass(fieldClass); tableInfo.addFieldType(fieldType); + tableInfo.addFieldExtraInfo(fieldExtraInfo); if(LOG.isInfoEnabled()){ LOG.info(physicalField + "--->" + mappingField + " Class: " + fieldClass.toString()); } diff --git a/launcher/job/kafkaNotNull.txt b/launcher/job/kafkaNotNull.txt new file mode 100644 index 000000000..913d30f51 --- /dev/null +++ b/launcher/job/kafkaNotNull.txt @@ -0,0 +1,34 @@ +CREATE TABLE MyTable( + channel varchar not null, + pv INT, + xctime bigint, + CHARACTER_LENGTH(channel) as timeLeng, + WATERMARK FOR xctime AS withOffset(xctime,1000) + )WITH( + type='kafka11', + bootstrapServers='172.16.8.107:9092', + offsetReset='latest', + topic='toutou' + ); + +CREATE TABLE MyResult( + channel varchar, + pv INT + )WITH( + type='mysql', + url='jdbc:mysql://172.16.8.109:3306/test?charset=utf8', + userName='dtstack', + password='abc123', + tableName='pv' + ); + + +insert +into + MyResult + select + a.channel, + a.pv + from + MyTable a + diff --git a/serversocket/serversocket-source/pom.xml b/serversocket/serversocket-source/pom.xml index 4df2c3849..12e623063 100644 --- a/serversocket/serversocket-source/pom.xml +++ b/serversocket/serversocket-source/pom.xml @@ -70,7 +70,7 @@ + tofile="${basedir}/../../plugins/serversocketsource/${project.name}.jar-${git.branch}.jar" /> From e51e66c1d13b9903f5ac5e8b8523252cf5647067 Mon Sep 17 00:00:00 2001 From: toutian <953372946@qq.com> Date: Tue, 17 Sep 2019 20:22:46 +0800 Subject: [PATCH 381/470] =?UTF-8?q?[kafka=20not=20null][flinkStreamSQL?= =?UTF-8?q?=E6=94=AF=E6=8C=81not=20null=20=E8=AF=AD=E6=B3=95][17872]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../kafka/CustomerJsonDeserialization.java | 19 +++++++---------- .../flink/sql/source/kafka/KafkaSource.java | 4 ++-- .../source/kafka/table/KafkaSourceParser.java | 9 ++++++-- .../kafka/CustomerJsonDeserialization.java | 21 +++++++------------ .../flink/sql/source/kafka/KafkaSource.java | 4 ++-- .../source/kafka/table/KafkaSourceParser.java | 9 ++++++-- 6 files changed, 33 insertions(+), 33 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index d659b2882..ba8cc62a9 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.source.AbsDeserialization; import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import com.dtstack.flink.sql.table.TableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -49,6 +50,7 @@ import java.sql.Time; import java.sql.Timestamp; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Set; @@ -80,9 +82,6 @@ public class CustomerJsonDeserialization extends AbsDeserialization { /** Types to parse fields as. Indices match fieldNames indices. */ private final TypeInformation[] fieldTypes; - /** Flag indicating whether to fail on a missing field. */ - private boolean failOnMissingField; - private AbstractFetcher fetcher; private boolean firstMsg = true; @@ -91,15 +90,14 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private Map rowAndFieldMapping; + private List fieldExtraInfos; - public CustomerJsonDeserialization(TypeInformation typeInfo, Map rowAndFieldMapping){ + public CustomerJsonDeserialization(TypeInformation typeInfo, Map rowAndFieldMapping, List fieldExtraInfos){ this.typeInfo = typeInfo; - this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); - this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); - this.rowAndFieldMapping= rowAndFieldMapping; + this.fieldExtraInfos = fieldExtraInfos; } @Override @@ -129,9 +127,10 @@ public Row deserialize(byte[] message) throws IOException { for (int i = 0; i < fieldNames.length; i++) { JsonNode node = getIgnoreCase(fieldNames[i]); + TableInfo.FieldExtraInfo fieldExtraInfo = fieldExtraInfos.get(i); if (node == null) { - if (failOnMissingField) { + if (fieldExtraInfo != null && fieldExtraInfo.getNotNull()) { throw new IllegalStateException("Failed to find field with name '" + fieldNames[i] + "'."); } else { @@ -159,10 +158,6 @@ public Row deserialize(byte[] message) throws IOException { } } - public void setFailOnMissingField(boolean failOnMissingField) { - this.failOnMissingField = failOnMissingField; - } - private JsonNode getIgnoreCase(String key) { String nodeMappingKey = rowAndFieldMapping.getOrDefault(key, key); JsonNode node = nodeAndJsonNodeMapping.get(nodeMappingKey); diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 9b410b204..cb14f3af8 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -87,10 +87,10 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv FlinkKafkaConsumer09 kafkaSrc; if (BooleanUtils.isTrue(kafka09SourceTableInfo.getTopicIsPattern())) { kafkaSrc = new CustomerKafka09Consumer(Pattern.compile(topicName), - new CustomerJsonDeserialization(typeInformation, kafka09SourceTableInfo.getPhysicalFields()), props); + new CustomerJsonDeserialization(typeInformation, kafka09SourceTableInfo.getPhysicalFields(), kafka09SourceTableInfo.getFieldExtraInfoList()), props); } else { kafkaSrc = new CustomerKafka09Consumer(topicName, - new CustomerJsonDeserialization(typeInformation, kafka09SourceTableInfo.getPhysicalFields()), props); + new CustomerJsonDeserialization(typeInformation, kafka09SourceTableInfo.getPhysicalFields(), kafka09SourceTableInfo.getFieldExtraInfoList()), props); } //earliest,latest diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index c2fe6cb21..a6ffd9927 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -45,7 +45,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)(\\s+AS\\s+(\\w+))?(\\s+NOT\\s+NULL)?$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); @@ -56,13 +56,18 @@ public class KafkaSourceParser extends AbsSourceParser { static void dealNestField(Matcher matcher, TableInfo tableInfo) { String physicalField = matcher.group(1); String fieldType = matcher.group(3); - String mappingField = matcher.group(4); + String mappingField = matcher.group(5); + mappingField = mappingField != null ? mappingField : physicalField; Class fieldClass= ClassUtil.stringConvertClass(fieldType); + boolean notNull = matcher.group(6) != null; + TableInfo.FieldExtraInfo fieldExtraInfo = new TableInfo.FieldExtraInfo(); + fieldExtraInfo.setNotNull(notNull); tableInfo.addPhysicalMappings(mappingField, physicalField); tableInfo.addField(mappingField); tableInfo.addFieldClass(fieldClass); tableInfo.addFieldType(fieldType); + tableInfo.addFieldExtraInfo(fieldExtraInfo); if(LOG.isInfoEnabled()){ LOG.info(physicalField + "--->" + mappingField + " Class: " + fieldClass.toString()); } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 5a717a8dc..66cd6efb3 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.source.AbsDeserialization; import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import com.dtstack.flink.sql.table.TableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -49,6 +50,7 @@ import java.sql.Time; import java.sql.Timestamp; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Set; @@ -80,9 +82,6 @@ public class CustomerJsonDeserialization extends AbsDeserialization { /** Types to parse fields as. Indices match fieldNames indices. */ private final TypeInformation[] fieldTypes; - /** Flag indicating whether to fail on a missing field. */ - private boolean failOnMissingField; - private AbstractFetcher fetcher; private boolean firstMsg = true; @@ -91,14 +90,14 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private Map rowAndFieldMapping; - public CustomerJsonDeserialization(TypeInformation typeInfo, Map rowAndFieldMapping){ - this.typeInfo = typeInfo; + private List fieldExtraInfos; + public CustomerJsonDeserialization(TypeInformation typeInfo, Map rowAndFieldMapping, List fieldExtraInfos){ + this.typeInfo = typeInfo; this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); - this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); - this.rowAndFieldMapping= rowAndFieldMapping; + this.fieldExtraInfos = fieldExtraInfos; } @Override @@ -129,9 +128,10 @@ public Row deserialize(byte[] message) throws IOException { for (int i = 0; i < fieldNames.length; i++) { JsonNode node = getIgnoreCase(fieldNames[i]); + TableInfo.FieldExtraInfo fieldExtraInfo = fieldExtraInfos.get(i); if (node == null) { - if (failOnMissingField) { + if (fieldExtraInfo != null && fieldExtraInfo.getNotNull()) { throw new IllegalStateException("Failed to find field with name '" + fieldNames[i] + "'."); } else { @@ -176,11 +176,6 @@ public JsonNode getIgnoreCase(String key) { return node; } - - public void setFailOnMissingField(boolean failOnMissingField) { - this.failOnMissingField = failOnMissingField; - } - private void parseTree(JsonNode jsonNode, String prefix){ Iterator iterator = jsonNode.fieldNames(); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java index 779cc2f71..f875329e6 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java @@ -89,10 +89,10 @@ public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnv FlinkKafkaConsumer010 kafkaSrc; if (BooleanUtils.isTrue(kafka010SourceTableInfo.getTopicIsPattern())) { kafkaSrc = new CustomerKafka010Consumer(Pattern.compile(topicName), - new CustomerJsonDeserialization(typeInformation, kafka010SourceTableInfo.getPhysicalFields()), props); + new CustomerJsonDeserialization(typeInformation, kafka010SourceTableInfo.getPhysicalFields(), kafka010SourceTableInfo.getFieldExtraInfoList()), props); } else { kafkaSrc = new CustomerKafka010Consumer(topicName, - new CustomerJsonDeserialization(typeInformation, kafka010SourceTableInfo.getPhysicalFields()), props); + new CustomerJsonDeserialization(typeInformation, kafka010SourceTableInfo.getPhysicalFields(), kafka010SourceTableInfo.getFieldExtraInfoList()), props); } //earliest,latest diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 020d54781..806f3a65d 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -44,7 +44,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)(\\s+AS\\s+(\\w+))?(\\s+NOT\\s+NULL)?$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); @@ -60,13 +60,18 @@ public class KafkaSourceParser extends AbsSourceParser { static void dealNestField(Matcher matcher, TableInfo tableInfo) { String physicalField = matcher.group(1); String fieldType = matcher.group(3); - String mappingField = matcher.group(4); + String mappingField = matcher.group(5); + mappingField = mappingField != null ? mappingField : physicalField; Class fieldClass= ClassUtil.stringConvertClass(fieldType); + boolean notNull = matcher.group(6) != null; + TableInfo.FieldExtraInfo fieldExtraInfo = new TableInfo.FieldExtraInfo(); + fieldExtraInfo.setNotNull(notNull); tableInfo.addPhysicalMappings(mappingField, physicalField); tableInfo.addField(mappingField); tableInfo.addFieldClass(fieldClass); tableInfo.addFieldType(fieldType); + tableInfo.addFieldExtraInfo(fieldExtraInfo); if(LOG.isInfoEnabled()){ LOG.info(physicalField + "--->" + mappingField + " Class: " + fieldClass.toString()); } From dfc825444f4ead58d9b96f087134d6e1a911c054 Mon Sep 17 00:00:00 2001 From: toutian <953372946@qq.com> Date: Wed, 18 Sep 2019 09:58:11 +0800 Subject: [PATCH 382/470] =?UTF-8?q?[pom=E6=96=87=E4=BB=B6=E5=8A=A0?= =?UTF-8?q?=E5=85=A5branch][flinkStreamSQL=E6=94=AF=E6=8C=81not=20null=20?= =?UTF-8?q?=E8=AF=AD=E6=B3=95][17872]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- cassandra/cassandra-sink/pom.xml | 2 +- console/console-sink/pom.xml | 2 +- hbase/hbase-sink/pom.xml | 2 +- serversocket/serversocket-source/pom.xml | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/cassandra/cassandra-sink/pom.xml b/cassandra/cassandra-sink/pom.xml index b60ff2e3f..0d9b10361 100644 --- a/cassandra/cassandra-sink/pom.xml +++ b/cassandra/cassandra-sink/pom.xml @@ -71,7 +71,7 @@ + tofile="${basedir}/../../plugins/cassandrasink/${project.name}-${git.branch}.jar" /> diff --git a/console/console-sink/pom.xml b/console/console-sink/pom.xml index 0579f68cf..2f8ad9ef9 100644 --- a/console/console-sink/pom.xml +++ b/console/console-sink/pom.xml @@ -67,7 +67,7 @@ + tofile="${basedir}/../../plugins/consolesink/${project.name}-${git.branch}.jar" /> diff --git a/hbase/hbase-sink/pom.xml b/hbase/hbase-sink/pom.xml index 132739f2f..11ecf0a35 100644 --- a/hbase/hbase-sink/pom.xml +++ b/hbase/hbase-sink/pom.xml @@ -68,7 +68,7 @@ + tofile="${basedir}/../../plugins/hbasesink/${project.name}-${git.branch}.jar" /> diff --git a/serversocket/serversocket-source/pom.xml b/serversocket/serversocket-source/pom.xml index 12e623063..9a1f274c0 100644 --- a/serversocket/serversocket-source/pom.xml +++ b/serversocket/serversocket-source/pom.xml @@ -70,7 +70,7 @@ + tofile="${basedir}/../../plugins/serversocketsource/${project.name}-${git.branch}.jar" /> From c2927187cfbc1b42e305ad00f38ba8d305d01ff1 Mon Sep 17 00:00:00 2001 From: toutian <953372946@qq.com> Date: Wed, 18 Sep 2019 10:02:45 +0800 Subject: [PATCH 383/470] =?UTF-8?q?[registerUDF=20=E4=BC=98=E5=8C=96?= =?UTF-8?q?=E5=86=99=E6=B3=95][flinkStreamSQL=E6=94=AF=E6=8C=81not=20null?= =?UTF-8?q?=20=E8=AF=AD=E6=B3=95][17872]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/src/main/java/com/dtstack/flink/sql/Main.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 0cecdaa61..4a4e0a44e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -250,10 +250,13 @@ private static void addEnvClassPath(StreamExecutionEnvironment env, Set cla private static void registerUDF(SqlTree sqlTree, List jarURList, URLClassLoader parentClassloader, StreamTableEnvironment tableEnv) throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException { + List funcList = sqlTree.getFunctionList(); + if (funcList.isEmpty()) { + return; + } //load jar URLClassLoader classLoader = FlinkUtil.loadExtraJar(jarURList, parentClassloader); //register urf - List funcList = sqlTree.getFunctionList(); for (CreateFuncParser.SqlParserResult funcInfo : funcList) { FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName(), tableEnv, classLoader); From 6815e1f08b90c24d427f4fb0b53b6e32d49660c1 Mon Sep 17 00:00:00 2001 From: toutian <953372946@qq.com> Date: Wed, 18 Sep 2019 13:09:39 +0800 Subject: [PATCH 384/470] =?UTF-8?q?[not=20null][flinkStreamSQL=E6=94=AF?= =?UTF-8?q?=E6=8C=81not=20null=20=E8=AF=AD=E6=B3=95][17872]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/Main.java | 11 +++++++--- .../flink/sql/table/AbsSourceParser.java | 22 +++++++++++++++++-- .../dtstack/flink/sql/table/TableInfo.java | 2 +- .../source/kafka/table/KafkaSourceParser.java | 8 +++---- .../source/kafka/table/KafkaSourceParser.java | 7 +++--- .../source/kafka/table/KafkaSourceParser.java | 7 +++--- 6 files changed, 38 insertions(+), 19 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 4a4e0a44e..80c81eb71 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -75,9 +75,9 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.io.IOException; import java.lang.reflect.Field; import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.net.URL; import java.net.URLClassLoader; import java.net.URLDecoder; @@ -328,7 +328,7 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en } } - private static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws IOException, NoSuchMethodException { + private static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws Exception { StreamExecutionEnvironment env = !ClusterMode.local.name().equals(deployMode) ? StreamExecutionEnvironment.getExecutionEnvironment() : new MyLocalStreamEnvironment(); @@ -336,7 +336,12 @@ private static StreamExecutionEnvironment getStreamExeEnv(Properties confPropert env.setParallelism(FlinkUtil.getEnvParallelism(confProperties)); Configuration globalJobParameters = new Configuration(); - globalJobParameters.addAllToProperties(confProperties); + //Configuration unsupported set properties key-value + Method method = Configuration.class.getDeclaredMethod("setValueInternal", String.class, Object.class); + method.setAccessible(true); + for (Map.Entry prop : confProperties.entrySet()) { + method.invoke(globalJobParameters, prop.getKey(), prop.getValue()); + } ExecutionConfig exeConfig = env.getConfig(); if(exeConfig.getGlobalJobParameters() == null){ diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java index 1b676ff8d..09e7ab6b5 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java @@ -20,6 +20,7 @@ package com.dtstack.flink.sql.table; +import com.dtstack.flink.sql.util.ClassUtil; import com.dtstack.flink.sql.util.MathUtil; import java.util.regex.Matcher; @@ -36,19 +37,21 @@ public abstract class AbsSourceParser extends AbsTableParser { private static final String VIRTUAL_KEY = "virtualFieldKey"; - private static final String WATERMARK_KEY = "waterMarkKey"; + private static final String NOTNULL_KEY = "notNullKey"; private static Pattern virtualFieldKeyPattern = Pattern.compile("(?i)^(\\S+\\([^\\)]+\\))\\s+AS\\s+(\\w+)$"); - private static Pattern waterMarkKeyPattern = Pattern.compile("(?i)^\\s*WATERMARK\\s+FOR\\s+(\\S+)\\s+AS\\s+withOffset\\(\\s*(\\S+)\\s*,\\s*(\\d+)\\s*\\)$"); + private static Pattern notNullKeyPattern = Pattern.compile("(?i)(\\w+)\\s+(\\w+)(\\s+NOT\\s+NULL)?$"); static { keyPatternMap.put(VIRTUAL_KEY, virtualFieldKeyPattern); keyPatternMap.put(WATERMARK_KEY, waterMarkKeyPattern); + keyPatternMap.put(NOTNULL_KEY, notNullKeyPattern); keyHandlerMap.put(VIRTUAL_KEY, AbsSourceParser::dealVirtualField); keyHandlerMap.put(WATERMARK_KEY, AbsSourceParser::dealWaterMark); + keyHandlerMap.put(NOTNULL_KEY, AbsSourceParser::dealNotNull); } static void dealVirtualField(Matcher matcher, TableInfo tableInfo){ @@ -66,4 +69,19 @@ static void dealWaterMark(Matcher matcher, TableInfo tableInfo){ sourceTableInfo.setEventTimeField(eventTimeField); sourceTableInfo.setMaxOutOrderness(offset); } + + static void dealNotNull(Matcher matcher, TableInfo tableInfo) { + String fieldName = matcher.group(1); + String fieldType = matcher.group(2); + Class fieldClass= ClassUtil.stringConvertClass(fieldType); + boolean notNull = matcher.group(3) != null; + TableInfo.FieldExtraInfo fieldExtraInfo = new TableInfo.FieldExtraInfo(); + fieldExtraInfo.setNotNull(notNull); + + tableInfo.addPhysicalMappings(fieldName, fieldName); + tableInfo.addField(fieldName); + tableInfo.addFieldClass(fieldClass); + tableInfo.addFieldType(fieldType); + tableInfo.addFieldExtraInfo(fieldExtraInfo); + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java index 491b7bca3..ebda7d120 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java @@ -189,7 +189,7 @@ public static class FieldExtraInfo implements Serializable { /** * default false:allow field is null */ - boolean notNull; + boolean notNull = false; public boolean getNotNull() { return notNull; diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index a6ffd9927..d147e4a19 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -21,7 +21,6 @@ package com.dtstack.flink.sql.source.kafka.table; import com.dtstack.flink.sql.table.AbsSourceParser; -import com.dtstack.flink.sql.table.SourceTableInfo; import com.dtstack.flink.sql.table.TableInfo; import com.dtstack.flink.sql.util.ClassUtil; import com.dtstack.flink.sql.util.MathUtil; @@ -45,7 +44,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)(\\s+AS\\s+(\\w+))?(\\s+NOT\\s+NULL)?$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)(\\s+NOT\\s+NULL)?$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); @@ -56,10 +55,9 @@ public class KafkaSourceParser extends AbsSourceParser { static void dealNestField(Matcher matcher, TableInfo tableInfo) { String physicalField = matcher.group(1); String fieldType = matcher.group(3); - String mappingField = matcher.group(5); - mappingField = mappingField != null ? mappingField : physicalField; + String mappingField = matcher.group(4); Class fieldClass= ClassUtil.stringConvertClass(fieldType); - boolean notNull = matcher.group(6) != null; + boolean notNull = matcher.group(5) != null; TableInfo.FieldExtraInfo fieldExtraInfo = new TableInfo.FieldExtraInfo(); fieldExtraInfo.setNotNull(notNull); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 806f3a65d..44f22497f 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -44,7 +44,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)(\\s+AS\\s+(\\w+))?(\\s+NOT\\s+NULL)?$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)(\\s+NOT\\s+NULL)?$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); @@ -60,10 +60,9 @@ public class KafkaSourceParser extends AbsSourceParser { static void dealNestField(Matcher matcher, TableInfo tableInfo) { String physicalField = matcher.group(1); String fieldType = matcher.group(3); - String mappingField = matcher.group(5); - mappingField = mappingField != null ? mappingField : physicalField; + String mappingField = matcher.group(4); Class fieldClass= ClassUtil.stringConvertClass(fieldType); - boolean notNull = matcher.group(6) != null; + boolean notNull = matcher.group(5) != null; TableInfo.FieldExtraInfo fieldExtraInfo = new TableInfo.FieldExtraInfo(); fieldExtraInfo.setNotNull(notNull); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 4d5cce30c..79686921f 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -44,7 +44,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)(\\s+AS\\s+(\\w+))?(\\s+NOT\\s+NULL)?$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)(\\s+NOT\\s+NULL)?$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); @@ -60,10 +60,9 @@ public class KafkaSourceParser extends AbsSourceParser { static void dealNestField(Matcher matcher, TableInfo tableInfo) { String physicalField = matcher.group(1); String fieldType = matcher.group(3); - String mappingField = matcher.group(5); - mappingField = mappingField != null ? mappingField : physicalField; + String mappingField = matcher.group(4); Class fieldClass= ClassUtil.stringConvertClass(fieldType); - boolean notNull = matcher.group(6) != null; + boolean notNull = matcher.group(5) != null; TableInfo.FieldExtraInfo fieldExtraInfo = new TableInfo.FieldExtraInfo(); fieldExtraInfo.setNotNull(notNull); From 61b24f024ef2d3d824635f9b19d1ac577adf5e2a Mon Sep 17 00:00:00 2001 From: toutian <953372946@qq.com> Date: Wed, 18 Sep 2019 14:06:07 +0800 Subject: [PATCH 385/470] =?UTF-8?q?[not=20null][flinkStreamSQL=E6=94=AF?= =?UTF-8?q?=E6=8C=81not=20null=20=E8=AF=AD=E6=B3=95][17872]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flink/sql/table/AbsSourceParser.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java index 09e7ab6b5..347151fe3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java @@ -42,7 +42,7 @@ public abstract class AbsSourceParser extends AbsTableParser { private static Pattern virtualFieldKeyPattern = Pattern.compile("(?i)^(\\S+\\([^\\)]+\\))\\s+AS\\s+(\\w+)$"); private static Pattern waterMarkKeyPattern = Pattern.compile("(?i)^\\s*WATERMARK\\s+FOR\\s+(\\S+)\\s+AS\\s+withOffset\\(\\s*(\\S+)\\s*,\\s*(\\d+)\\s*\\)$"); - private static Pattern notNullKeyPattern = Pattern.compile("(?i)(\\w+)\\s+(\\w+)(\\s+NOT\\s+NULL)?$"); + private static Pattern notNullKeyPattern = Pattern.compile("(?i)(\\w+)\\s+(\\w+)\\s+NOT\\s+NULL?$"); static { keyPatternMap.put(VIRTUAL_KEY, virtualFieldKeyPattern); @@ -74,9 +74,8 @@ static void dealNotNull(Matcher matcher, TableInfo tableInfo) { String fieldName = matcher.group(1); String fieldType = matcher.group(2); Class fieldClass= ClassUtil.stringConvertClass(fieldType); - boolean notNull = matcher.group(3) != null; TableInfo.FieldExtraInfo fieldExtraInfo = new TableInfo.FieldExtraInfo(); - fieldExtraInfo.setNotNull(notNull); + fieldExtraInfo.setNotNull(true); tableInfo.addPhysicalMappings(fieldName, fieldName); tableInfo.addField(fieldName); From c548076cc946c8033ec8471b64b4a13fd5557641 Mon Sep 17 00:00:00 2001 From: toutian <953372946@qq.com> Date: Wed, 18 Sep 2019 14:10:28 +0800 Subject: [PATCH 386/470] =?UTF-8?q?[not=20null][flinkStreamSQL=E6=94=AF?= =?UTF-8?q?=E6=8C=81not=20null=20=E8=AF=AD=E6=B3=95][17872]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flink/sql/table/AbsSourceParser.java | 2 +- launcher/job/kafkaNotNull.txt | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java index 347151fe3..82000b386 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsSourceParser.java @@ -42,7 +42,7 @@ public abstract class AbsSourceParser extends AbsTableParser { private static Pattern virtualFieldKeyPattern = Pattern.compile("(?i)^(\\S+\\([^\\)]+\\))\\s+AS\\s+(\\w+)$"); private static Pattern waterMarkKeyPattern = Pattern.compile("(?i)^\\s*WATERMARK\\s+FOR\\s+(\\S+)\\s+AS\\s+withOffset\\(\\s*(\\S+)\\s*,\\s*(\\d+)\\s*\\)$"); - private static Pattern notNullKeyPattern = Pattern.compile("(?i)(\\w+)\\s+(\\w+)\\s+NOT\\s+NULL?$"); + private static Pattern notNullKeyPattern = Pattern.compile("(?i)^(\\w+)\\s+(\\w+)\\s+NOT\\s+NULL?$"); static { keyPatternMap.put(VIRTUAL_KEY, virtualFieldKeyPattern); diff --git a/launcher/job/kafkaNotNull.txt b/launcher/job/kafkaNotNull.txt index 913d30f51..0515f2036 100644 --- a/launcher/job/kafkaNotNull.txt +++ b/launcher/job/kafkaNotNull.txt @@ -1,6 +1,6 @@ CREATE TABLE MyTable( - channel varchar not null, - pv INT, + channel varchar as sss not null, + pv INT not null, xctime bigint, CHARACTER_LENGTH(channel) as timeLeng, WATERMARK FOR xctime AS withOffset(xctime,1000) From 74b04888c53a4ad2264675ac69182291ae755e95 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 18 Sep 2019 14:39:00 +0800 Subject: [PATCH 387/470] [mian method optimize] --- .../main/java/com/dtstack/flink/sql/Main.java | 43 ++++++++----------- .../flink/sql/{ => enums}/ClusterMode.java | 2 +- .../com/dtstack/flink/sql/option/Options.java | 2 +- .../sql/launcher/ClusterClientFactory.java | 2 +- .../flink/sql/launcher/LauncherMain.java | 2 +- 5 files changed, 22 insertions(+), 29 deletions(-) rename core/src/main/java/com/dtstack/flink/sql/{ => enums}/ClusterMode.java (96%) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 642734444..4bb90e737 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql; import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.enums.ClusterMode; import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.environment.MyLocalStreamEnvironment; import com.dtstack.flink.sql.exec.FlinkSQLExec; @@ -82,6 +83,7 @@ import java.util.Properties; import java.util.Set; import java.util.concurrent.TimeUnit; +import com.dtstack.flink.sql.option.Options; /** * Date: 2018/6/26 @@ -106,7 +108,7 @@ public class Main { public static void main(String[] args) throws Exception { OptionParser optionParser = new OptionParser(args); - com.dtstack.flink.sql.option.Options options = optionParser.getOptions(); + Options options = optionParser.getOptions(); String sql = options.getSql(); String name = options.getName(); String addJarListStr = options.getAddjar(); @@ -149,9 +151,20 @@ public static void main(String[] args) throws Exception { //register table schema registerTable(sqlTree, env, tableEnv, localSqlPluginPath, remoteSqlPluginPath, sideTableMap, registerTableCache); - SideSqlExec sideSqlExec = new SideSqlExec(); - sideSqlExec.setLocalSqlPluginPath(localSqlPluginPath); + sqlTranslation(options,tableEnv,sqlTree,sideTableMap,registerTableCache); + + if(env instanceof MyLocalStreamEnvironment) { + List urlList = new ArrayList<>(); + urlList.addAll(Arrays.asList(parentClassloader.getURLs())); + ((MyLocalStreamEnvironment) env).setClasspaths(urlList); + } + env.execute(name); + } + + private static void sqlTranslation(Options options,StreamTableEnvironment tableEnv,SqlTree sqlTree,Map sideTableMap,Map registerTableCache) throws Exception { + SideSqlExec sideSqlExec = new SideSqlExec(); + sideSqlExec.setLocalSqlPluginPath(options.getLocalSqlPluginPath()); for (CreateTmpTableParser.SqlParserResult result : sqlTree.getTmpSqlList()) { sideSqlExec.registerTmpTable(result, sideTableMap, tableEnv, registerTableCache); } @@ -160,9 +173,7 @@ public static void main(String[] args) throws Exception { if(LOG.isInfoEnabled()){ LOG.info("exe-sql:\n" + result.getExecSql()); } - boolean isSide = false; - for (String tableName : result.getTargetTableList()) { if (sqlTree.getTmpTableMap().containsKey(tableName)) { CreateTmpTableParser.SqlParserResult tmp = sqlTree.getTmpTableMap().get(tableName); @@ -183,7 +194,6 @@ public static void main(String[] args) throws Exception { break; } } - if(isSide){ //sql-dimensional table contains the dimension table of execution sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache); @@ -197,15 +207,8 @@ public static void main(String[] args) throws Exception { } } - if(env instanceof MyLocalStreamEnvironment) { - List urlList = new ArrayList<>(); - urlList.addAll(Arrays.asList(parentClassloader.getURLs())); - ((MyLocalStreamEnvironment) env).setClasspaths(urlList); - } - env.execute(name); } - /** * This part is just to add classpath for the jar when reading remote execution, and will not submit jar from a local * @param env @@ -314,42 +317,32 @@ private static StreamExecutionEnvironment getStreamExeEnv(Properties confPropert Configuration globalJobParameters = new Configuration(); Method method = Configuration.class.getDeclaredMethod("setValueInternal", String.class, Object.class); method.setAccessible(true); - confProperties.forEach((key,val) -> { try { method.invoke(globalJobParameters, key, val); - } catch (IllegalAccessException e) { - e.printStackTrace(); - } catch (InvocationTargetException e) { - e.printStackTrace(); + } catch (Exception e) { + LOG.error("set Configuration key:{},value:{} error:{}",key,val,e); } }); - ExecutionConfig exeConfig = env.getConfig(); if(exeConfig.getGlobalJobParameters() == null){ exeConfig.setGlobalJobParameters(globalJobParameters); }else if(exeConfig.getGlobalJobParameters() instanceof Configuration){ ((Configuration) exeConfig.getGlobalJobParameters()).addAll(globalJobParameters); } - - if(FlinkUtil.getMaxEnvParallelism(confProperties) > 0){ env.setMaxParallelism(FlinkUtil.getMaxEnvParallelism(confProperties)); } - if(FlinkUtil.getBufferTimeoutMillis(confProperties) > 0){ env.setBufferTimeout(FlinkUtil.getBufferTimeoutMillis(confProperties)); } - env.setRestartStrategy(RestartStrategies.failureRateRestart( failureRate, Time.of(failureInterval, TimeUnit.MINUTES), Time.of(delayInterval, TimeUnit.SECONDS) )); - FlinkUtil.setStreamTimeCharacteristic(env, confProperties); FlinkUtil.openCheckpoint(env, confProperties); - return env; } } diff --git a/core/src/main/java/com/dtstack/flink/sql/ClusterMode.java b/core/src/main/java/com/dtstack/flink/sql/enums/ClusterMode.java similarity index 96% rename from core/src/main/java/com/dtstack/flink/sql/ClusterMode.java rename to core/src/main/java/com/dtstack/flink/sql/enums/ClusterMode.java index 82105d9e7..341258a43 100644 --- a/core/src/main/java/com/dtstack/flink/sql/ClusterMode.java +++ b/core/src/main/java/com/dtstack/flink/sql/enums/ClusterMode.java @@ -17,7 +17,7 @@ * limitations under the License. */ -package com.dtstack.flink.sql; +package com.dtstack.flink.sql.enums; /** * Created by sishu.yss on 2018/10/10. diff --git a/core/src/main/java/com/dtstack/flink/sql/option/Options.java b/core/src/main/java/com/dtstack/flink/sql/option/Options.java index 0c33a8323..d90d6c4a6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/option/Options.java +++ b/core/src/main/java/com/dtstack/flink/sql/option/Options.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.option; -import com.dtstack.flink.sql.ClusterMode; +import com.dtstack.flink.sql.enums.ClusterMode; /** diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 72dbca3ee..3596b2a3c 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -49,7 +49,7 @@ import java.net.URLDecoder; import java.util.*; -import com.dtstack.flink.sql.ClusterMode; +import com.dtstack.flink.sql.enums.ClusterMode; import org.apache.hadoop.yarn.exceptions.YarnException; import java.io.IOException; import java.util.stream.Collectors; diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index 33e897114..ce48b388c 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -23,7 +23,7 @@ import avro.shaded.com.google.common.collect.Lists; import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.TypeReference; -import com.dtstack.flink.sql.ClusterMode; +import com.dtstack.flink.sql.enums.ClusterMode; import com.dtstack.flink.sql.Main; import com.dtstack.flink.sql.launcher.perjob.PerJobSubmitter; import com.dtstack.flink.sql.option.OptionParser; From fc5bb4d567b0a6a289aedac0a2d75923807279d8 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 18 Sep 2019 15:20:32 +0800 Subject: [PATCH 388/470] [code optimize] --- core/src/main/java/com/dtstack/flink/sql/Main.java | 13 ++++--------- .../sql/{util => constrant}/ConfigConstrant.java | 9 ++++++++- .../{threadFactory => factory}/DTThreadFactory.java | 2 +- .../java/com/dtstack/flink/sql/side/AllReqRow.java | 2 +- .../java/com/dtstack/flink/sql/util/FlinkUtil.java | 1 + .../flink/sql/side/hbase/HbaseAsyncReqRow.java | 2 +- .../flink/sql/launcher/ClusterClientFactory.java | 6 +++--- .../com/dtstack/flink/sql}/yarn/JobParameter.java | 2 +- .../flink/sql}/yarn/YarnClusterConfiguration.java | 2 +- .../flink/sql}/yarn/YarnClusterDescriptor.java | 3 +-- 10 files changed, 22 insertions(+), 20 deletions(-) rename core/src/main/java/com/dtstack/flink/sql/{util => constrant}/ConfigConstrant.java (89%) rename core/src/main/java/com/dtstack/flink/sql/{threadFactory => factory}/DTThreadFactory.java (97%) rename {core/src/main/java/com/dtstack/flink => launcher/src/main/java/com/dtstack/flink/sql}/yarn/JobParameter.java (99%) rename {core/src/main/java/com/dtstack/flink => launcher/src/main/java/com/dtstack/flink/sql}/yarn/YarnClusterConfiguration.java (98%) rename {core/src/main/java/com/dtstack/flink => launcher/src/main/java/com/dtstack/flink/sql}/yarn/YarnClusterDescriptor.java (99%) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 4bb90e737..b14a87912 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql; import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.constrant.ConfigConstrant; import com.dtstack.flink.sql.enums.ClusterMode; import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.environment.MyLocalStreamEnvironment; @@ -99,12 +100,6 @@ public class Main { private static final Logger LOG = LoggerFactory.getLogger(Main.class); - private static final int failureRate = 3; - - private static final int failureInterval = 6; //min - - private static final int delayInterval = 10; //sec - public static void main(String[] args) throws Exception { OptionParser optionParser = new OptionParser(args); @@ -337,9 +332,9 @@ private static StreamExecutionEnvironment getStreamExeEnv(Properties confPropert env.setBufferTimeout(FlinkUtil.getBufferTimeoutMillis(confProperties)); } env.setRestartStrategy(RestartStrategies.failureRateRestart( - failureRate, - Time.of(failureInterval, TimeUnit.MINUTES), - Time.of(delayInterval, TimeUnit.SECONDS) + ConfigConstrant.failureRate, + Time.of(ConfigConstrant.failureInterval, TimeUnit.MINUTES), + Time.of(ConfigConstrant.delayInterval, TimeUnit.SECONDS) )); FlinkUtil.setStreamTimeCharacteristic(env, confProperties); FlinkUtil.openCheckpoint(env, confProperties); diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ConfigConstrant.java b/core/src/main/java/com/dtstack/flink/sql/constrant/ConfigConstrant.java similarity index 89% rename from core/src/main/java/com/dtstack/flink/sql/util/ConfigConstrant.java rename to core/src/main/java/com/dtstack/flink/sql/constrant/ConfigConstrant.java index 160d8f411..6b6551f9a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ConfigConstrant.java +++ b/core/src/main/java/com/dtstack/flink/sql/constrant/ConfigConstrant.java @@ -18,7 +18,7 @@ -package com.dtstack.flink.sql.util; +package com.dtstack.flink.sql.constrant; /** @@ -51,4 +51,11 @@ public class ConfigConstrant { public static final String FLINK_TIME_CHARACTERISTIC_KEY = "time.characteristic"; + // restart plocy + public static final int failureRate = 3; + + public static final int failureInterval = 6; //min + + public static final int delayInterval = 10; //sec + } diff --git a/core/src/main/java/com/dtstack/flink/sql/threadFactory/DTThreadFactory.java b/core/src/main/java/com/dtstack/flink/sql/factory/DTThreadFactory.java similarity index 97% rename from core/src/main/java/com/dtstack/flink/sql/threadFactory/DTThreadFactory.java rename to core/src/main/java/com/dtstack/flink/sql/factory/DTThreadFactory.java index 1c16581ef..e9f9dfa64 100644 --- a/core/src/main/java/com/dtstack/flink/sql/threadFactory/DTThreadFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/factory/DTThreadFactory.java @@ -18,7 +18,7 @@ -package com.dtstack.flink.sql.threadFactory; +package com.dtstack.flink.sql.factory; import java.util.concurrent.ThreadFactory; import java.util.concurrent.atomic.AtomicInteger; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java b/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java index a185da1bd..e788cf139 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/AllReqRow.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.side; -import com.dtstack.flink.sql.threadFactory.DTThreadFactory; +import com.dtstack.flink.sql.factory.DTThreadFactory; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Row; diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java index 89476380e..8884c47b3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.util; +import com.dtstack.flink.sql.constrant.ConfigConstrant; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.runtime.state.filesystem.FsStateBackend; diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java index ed5cfa558..05751d61e 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncReqRow.java @@ -30,7 +30,7 @@ import com.dtstack.flink.sql.side.hbase.rowkeydealer.PreRowKeyModeDealerDealer; import com.dtstack.flink.sql.side.hbase.rowkeydealer.RowKeyEqualModeDealer; import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo; -import com.dtstack.flink.sql.threadFactory.DTThreadFactory; +import com.dtstack.flink.sql.factory.DTThreadFactory; import com.google.common.collect.Maps; import com.stumbleupon.async.Deferred; import org.apache.flink.api.java.typeutils.RowTypeInfo; diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 3596b2a3c..125ee301a 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -20,8 +20,8 @@ import com.dtstack.flink.sql.option.Options; import com.dtstack.flink.sql.util.PluginUtil; -import com.dtstack.flink.yarn.JobParameter; -import com.dtstack.flink.yarn.YarnClusterConfiguration; +import com.dtstack.flink.sql.yarn.JobParameter; +import com.dtstack.flink.sql.yarn.YarnClusterConfiguration; import org.apache.commons.io.Charsets; import org.apache.commons.lang.StringUtils; import org.apache.flink.client.program.ClusterClient; @@ -129,7 +129,7 @@ public static ClusterClient createYarnClient(Options launcherOptions, String mod //jobmanager+taskmanager param JobParameter appConf = new JobParameter(confProperties); - com.dtstack.flink.yarn.YarnClusterDescriptor clusterDescriptor = new com.dtstack.flink.yarn.YarnClusterDescriptor( + com.dtstack.flink.sql.yarn.YarnClusterDescriptor clusterDescriptor = new com.dtstack.flink.sql.yarn.YarnClusterDescriptor( clusterConf, yarnClient, appConf,applicationId, launcherOptions.getName(),null ); clusterClient = clusterDescriptor.deploy(); diff --git a/core/src/main/java/com/dtstack/flink/yarn/JobParameter.java b/launcher/src/main/java/com/dtstack/flink/sql/yarn/JobParameter.java similarity index 99% rename from core/src/main/java/com/dtstack/flink/yarn/JobParameter.java rename to launcher/src/main/java/com/dtstack/flink/sql/yarn/JobParameter.java index 1879c6ea1..abfa0a8ad 100644 --- a/core/src/main/java/com/dtstack/flink/yarn/JobParameter.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/yarn/JobParameter.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flink.yarn; +package com.dtstack.flink.sql.yarn; import java.util.Objects; import java.util.Properties; diff --git a/core/src/main/java/com/dtstack/flink/yarn/YarnClusterConfiguration.java b/launcher/src/main/java/com/dtstack/flink/sql/yarn/YarnClusterConfiguration.java similarity index 98% rename from core/src/main/java/com/dtstack/flink/yarn/YarnClusterConfiguration.java rename to launcher/src/main/java/com/dtstack/flink/sql/yarn/YarnClusterConfiguration.java index 79ecbcc36..2189e9a9a 100644 --- a/core/src/main/java/com/dtstack/flink/yarn/YarnClusterConfiguration.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/yarn/YarnClusterConfiguration.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flink.yarn; +package com.dtstack.flink.sql.yarn; import org.apache.flink.configuration.Configuration; import org.apache.hadoop.fs.Path; diff --git a/core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java b/launcher/src/main/java/com/dtstack/flink/sql/yarn/YarnClusterDescriptor.java similarity index 99% rename from core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java rename to launcher/src/main/java/com/dtstack/flink/sql/yarn/YarnClusterDescriptor.java index 4e8987057..989a4400a 100644 --- a/core/src/main/java/com/dtstack/flink/yarn/YarnClusterDescriptor.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/yarn/YarnClusterDescriptor.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.flink.yarn; +package com.dtstack.flink.sql.yarn; import org.apache.flink.client.deployment.ClusterDeploymentException; import org.apache.flink.client.deployment.ClusterSpecification; @@ -42,7 +42,6 @@ import java.util.*; import java.util.concurrent.*; import java.util.stream.Collectors; -import java.util.stream.Stream; import static java.util.Objects.requireNonNull; import static org.apache.hadoop.yarn.api.records.YarnApplicationState.NEW; From 7ced9a07519a3bc3171599c3ca84ae6f993a4cc8 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 18 Sep 2019 16:42:41 +0800 Subject: [PATCH 389/470] =?UTF-8?q?[dbconn=E6=9B=BF=E6=8D=A2]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java | 1 + 1 file changed, 1 insertion(+) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 7131ebdda..40cb712a9 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -329,6 +329,7 @@ private void checkConnectionOpen(Connection dbConn) { LOG.info("db connection reconnect.."); dbConn= establishConnection(); upload = dbConn.prepareStatement(insertQuery); + this.dbConn = dbConn; } } catch (SQLException e) { LOG.error("check connection open failed..", e); From ebaae7ea3219869186a7951dca332a294bcde572 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Thu, 19 Sep 2019 11:50:20 +0800 Subject: [PATCH 390/470] =?UTF-8?q?[=E7=A6=81=E7=94=A8vertx=E7=BC=93?= =?UTF-8?q?=E5=AD=98][flinkSQL=E4=BB=BB=E5=8A=A1=EF=BC=8C=E8=AE=BE?= =?UTF-8?q?=E7=BD=AE=E7=BB=93=E6=9E=9C=E8=A1=A8=EF=BC=8C=E6=8F=90=E4=BA=A4?= =?UTF-8?q?=E8=BF=90=E8=A1=8C=E4=B8=AD=EF=BC=8C=E5=81=9C=E6=AD=A2=E8=AF=A5?= =?UTF-8?q?=E4=BB=BB=E5=8A=A1=E4=BF=AE=E6=94=B9=E7=BB=93=E6=9E=9C=E8=A1=A8?= =?UTF-8?q?=EF=BC=8C=E5=86=8D=E6=AC=A1=E6=8F=90=E4=BA=A4=E8=BF=90=E8=A1=8C?= =?UTF-8?q?=E4=B8=AD=EF=BC=8C=E6=95=B0=E6=8D=AE=E6=9B=B2=E7=BA=BF=E5=8F=98?= =?UTF-8?q?=E6=88=90=E4=B8=A4=E6=9D=A1=EF=BC=8C=E4=B8=94=E6=96=B0=E7=9A=84?= =?UTF-8?q?=E7=BB=93=E6=9E=9C=E8=A1=A8=E6=95=B0=E6=8D=AE=E4=B8=8D=E6=AD=A3?= =?UTF-8?q?=E7=A1=AE][18063]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java | 3 ++- .../com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java | 5 ++++- .../flink/sql/side/sqlserver/SqlserverAsyncReqRow.java | 5 ++++- 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java index 51f311276..6120767ef 100644 --- a/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java +++ b/mysql/mysql-side/mysql-async-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAsyncReqRow.java @@ -69,11 +69,12 @@ public void open(Configuration parameters) throws Exception { .put("idle_connection_test_period", DEFAULT_IDLE_CONNECTION_TEST_PEROID) .put("test_connection_on_checkin", DEFAULT_TEST_CONNECTION_ON_CHECKIN); - + System.setProperty("vertx.disableFileCPResolving", "true"); VertxOptions vo = new VertxOptions(); vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); vo.setWorkerPoolSize(DEFAULT_VERTX_WORKER_POOL_SIZE); + vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); setRdbSQLClient(JDBCClient.createNonShared(vertx, mysqlClientConfig)); } diff --git a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java index e1d2fc69e..f973ffee2 100644 --- a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java +++ b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncReqRow.java @@ -59,11 +59,14 @@ public void open(Configuration parameters) throws Exception { .put("provider_class", DT_PROVIDER_CLASS) .put("preferred_test_query", PREFERRED_TEST_QUERY_SQL) .put("idle_connection_test_period", DEFAULT_IDLE_CONNECTION_TEST_PEROID) - .put("test_connection_on_checkin", DEFAULT_TEST_CONNECTION_ON_CHECKIN);; + .put("test_connection_on_checkin", DEFAULT_TEST_CONNECTION_ON_CHECKIN); + + System.setProperty("vertx.disableFileCPResolving", "true"); VertxOptions vo = new VertxOptions(); vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); vo.setWorkerPoolSize(DEFAULT_VERTX_WORKER_POOL_SIZE); + vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); setRdbSQLClient(JDBCClient.createNonShared(vertx, oracleClientConfig)); } diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java index 9fee31171..0f94106d7 100644 --- a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java @@ -57,11 +57,14 @@ public void open(Configuration parameters) throws Exception { .put("provider_class", DT_PROVIDER_CLASS) .put("preferred_test_query", PREFERRED_TEST_QUERY_SQL) .put("idle_connection_test_period", DEFAULT_IDLE_CONNECTION_TEST_PEROID) - .put("test_connection_on_checkin", DEFAULT_TEST_CONNECTION_ON_CHECKIN);; + .put("test_connection_on_checkin", DEFAULT_TEST_CONNECTION_ON_CHECKIN); + + System.setProperty("vertx.disableFileCPResolving", "true"); VertxOptions vo = new VertxOptions(); vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); vo.setWorkerPoolSize(DEFAULT_VERTX_WORKER_POOL_SIZE); + vo.setFileResolverCachingEnabled(false); Vertx vertx = Vertx.vertx(vo); setRdbSQLClient(JDBCClient.createNonShared(vertx, sqlserverClientConfig)); } From 441651cabe22f1dbd45a1f3391402f531dc86506 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Thu, 19 Sep 2019 15:20:17 +0800 Subject: [PATCH 391/470] =?UTF-8?q?[rdbsink=E8=A1=A8=E5=90=8D=E4=B8=8E?= =?UTF-8?q?=E6=B3=A8=E5=86=8C=E8=A1=A8=E5=90=8D=E7=A7=B0=E4=B8=80=E8=87=B4?= =?UTF-8?q?][flinkSQL=E4=BB=BB=E5=8A=A1=EF=BC=8C=E8=AE=BE=E7=BD=AE?= =?UTF-8?q?=E7=BB=93=E6=9E=9C=E8=A1=A8=EF=BC=8C=E6=8F=90=E4=BA=A4=E8=BF=90?= =?UTF-8?q?=E8=A1=8C=E4=B8=AD=EF=BC=8C=E5=81=9C=E6=AD=A2=E8=AF=A5=E4=BB=BB?= =?UTF-8?q?=E5=8A=A1=E4=BF=AE=E6=94=B9=E7=BB=93=E6=9E=9C=E8=A1=A8=EF=BC=8C?= =?UTF-8?q?=E5=86=8D=E6=AC=A1=E6=8F=90=E4=BA=A4=E8=BF=90=E8=A1=8C=E4=B8=AD?= =?UTF-8?q?=EF=BC=8C=E6=95=B0=E6=8D=AE=E6=9B=B2=E7=BA=BF=E5=8F=98=E6=88=90?= =?UTF-8?q?=E4=B8=A4=E6=9D=A1=EF=BC=8C=E4=B8=94=E6=96=B0=E7=9A=84=E7=BB=93?= =?UTF-8?q?=E6=9E=9C=E8=A1=A8=E6=95=B0=E6=8D=AE=E4=B8=8D=E6=AD=A3=E7=A1=AE?= =?UTF-8?q?][18063]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java index c1401c745..2e154a9b0 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/RdbSink.java @@ -69,6 +69,8 @@ public abstract class RdbSink implements RetractStreamTableSink, Serializab protected String tableName; + protected String registerTabName; + protected String sql; protected List primaryKeys; @@ -112,6 +114,7 @@ public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { String tmpUserName = rdbTableInfo.getUserName(); String tmpPassword = rdbTableInfo.getPassword(); String tmpTableName = rdbTableInfo.getTableName(); + String tmpRegisterName = rdbTableInfo.getName(); Integer tmpSqlBatchSize = rdbTableInfo.getBatchSize(); if (tmpSqlBatchSize != null) { @@ -136,6 +139,7 @@ public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { this.userName = tmpUserName; this.password = tmpPassword; this.tableName = tmpTableName; + this.registerTabName = tmpRegisterName; this.primaryKeys = rdbTableInfo.getPrimaryKeys(); this.dbType = rdbTableInfo.getType(); @@ -193,7 +197,7 @@ protected void buildSqlTypes(List fieldTypeArray) { public void emitDataStream(DataStream> dataStream) { RichSinkFunction richSinkFunction = createJdbcSinkFunc(); DataStreamSink streamSink = dataStream.addSink(richSinkFunction); - streamSink.name(tableName); + streamSink.name(registerTabName); if (parallelism > 0) { streamSink.setParallelism(parallelism); } From 900241accaac6309fed558f0d1806ec14decc0ad Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 19 Sep 2019 16:05:36 +0800 Subject: [PATCH 392/470] [code optimize] --- core/src/main/java/com/dtstack/flink/sql/Main.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index b14a87912..dbd82c6f0 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -85,6 +85,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import com.dtstack.flink.sql.option.Options; +import org.apache.calcite.sql.parser.SqlParser.Config; /** * Date: 2018/6/26 @@ -100,6 +101,11 @@ public class Main { private static final Logger LOG = LoggerFactory.getLogger(Main.class); + private static Config config = org.apache.calcite.sql.parser.SqlParser + .configBuilder() + .setLex(Lex.MYSQL) + .build(); + public static void main(String[] args) throws Exception { OptionParser optionParser = new OptionParser(args); @@ -174,10 +180,6 @@ private static void sqlTranslation(Options options,StreamTableEnvironment tableE CreateTmpTableParser.SqlParserResult tmp = sqlTree.getTmpTableMap().get(tableName); String realSql = DtStringUtil.replaceIgnoreQuota(result.getExecSql(), "`", ""); - org.apache.calcite.sql.parser.SqlParser.Config config = org.apache.calcite.sql.parser.SqlParser - .configBuilder() - .setLex(Lex.MYSQL) - .build(); SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql,config).parseStmt(); String tmpSql = ((SqlInsert) sqlNode).getSource().toString(); tmp.setExecSql(tmpSql); From 4c270319486a2034e2c3a5b48acf72d71b506377 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 19 Sep 2019 16:23:43 +0800 Subject: [PATCH 393/470] [code opt] --- .../dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java index 9fee31171..41c1dbd4a 100644 --- a/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-async-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAsyncReqRow.java @@ -57,7 +57,7 @@ public void open(Configuration parameters) throws Exception { .put("provider_class", DT_PROVIDER_CLASS) .put("preferred_test_query", PREFERRED_TEST_QUERY_SQL) .put("idle_connection_test_period", DEFAULT_IDLE_CONNECTION_TEST_PEROID) - .put("test_connection_on_checkin", DEFAULT_TEST_CONNECTION_ON_CHECKIN);; + .put("test_connection_on_checkin", DEFAULT_TEST_CONNECTION_ON_CHECKIN); VertxOptions vo = new VertxOptions(); vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); From 70b7a3c11aef090b8ca081a3c86d1f7b9d1fc05e Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Fri, 20 Sep 2019 16:26:07 +0800 Subject: [PATCH 394/470] =?UTF-8?q?flinksql=E5=A2=9E=E5=8A=A0udaf=E5=87=BD?= =?UTF-8?q?=E6=95=B0=E5=8A=9F=E8=83=BD?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flink/sql/util/FlinkUtil.java | 25 +++++++++++++++---- 1 file changed, 20 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java index 89476380e..fe342fe5b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java @@ -16,7 +16,7 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.util; @@ -33,6 +33,9 @@ import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.TableFunction; +import org.apache.flink.table.functions.UserDefinedFunction; +import org.apache.flink.table.functions.AggregateFunction; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -192,13 +195,25 @@ public static void registerScalaUDF(String classPath, String funcName, TableEnvi public static void registerTableUDF(String classPath, String funcName, TableEnvironment tableEnv, ClassLoader classLoader){ try { - TableFunction udfFunc = Class.forName(classPath, false, classLoader) - .asSubclass(TableFunction.class).newInstance(); + UserDefinedFunction udfFunc = Class.forName(classPath,false, classLoader) + .asSubclass(UserDefinedFunction.class).newInstance(); if(tableEnv instanceof StreamTableEnvironment){ - ((StreamTableEnvironment)tableEnv).registerFunction(funcName, udfFunc); + if (udfFunc instanceof AggregateFunction){ + ((StreamTableEnvironment) tableEnv).registerFunction(funcName, (AggregateFunction)udfFunc); + }else if (udfFunc instanceof TableFunction) { + ((StreamTableEnvironment) tableEnv).registerFunction(funcName, (TableFunction)udfFunc); + }else{ + throw new RuntimeException("no support UserDefinedFunction class for " + udfFunc.getClass().getName()); + } }else if(tableEnv instanceof BatchTableEnvironment){ - ((BatchTableEnvironment)tableEnv).registerFunction(funcName, udfFunc); + if (udfFunc instanceof AggregateFunction){ + ((BatchTableEnvironment) tableEnv).registerFunction(funcName, (AggregateFunction)udfFunc); + }else if (udfFunc instanceof TableFunction) { + ((BatchTableEnvironment) tableEnv).registerFunction(funcName, (TableFunction)udfFunc); + }else{ + throw new RuntimeException("no support UserDefinedFunction class for " + udfFunc.getClass().getName()); + } }else{ throw new RuntimeException("no support tableEnvironment class for " + tableEnv.getClass().getName()); } From 1924159c6ba567e1d3f9578cd59cdcee980bf856 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 23 Sep 2019 17:27:04 +0800 Subject: [PATCH 395/470] =?UTF-8?q?[LauncherMain=E5=90=AF=E5=8A=A8?= =?UTF-8?q?=E4=BE=9D=E8=B5=96=E9=85=8D=E7=BD=AE=E8=B0=83=E6=95=B4]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flink/sql/exec/FlinkSQLExec.java | 18 ++++++++ .../perjob/PerJobClusterClientBuilder.java | 8 ++-- .../sql/launcher/perjob/PerJobSubmitter.java | 42 ++++++------------- .../provider/DTC3P0DataSourceProvider.java | 18 ++++++++ 4 files changed, 51 insertions(+), 35 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/exec/FlinkSQLExec.java b/core/src/main/java/com/dtstack/flink/sql/exec/FlinkSQLExec.java index e1162bc78..6bcc25251 100644 --- a/core/src/main/java/com/dtstack/flink/sql/exec/FlinkSQLExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/exec/FlinkSQLExec.java @@ -1,3 +1,21 @@ +/* + * 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 com.dtstack.flink.sql.exec; import org.apache.calcite.sql.SqlIdentifier; diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java index 1f1b7d679..9157d219e 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java @@ -30,7 +30,6 @@ import java.io.File; import java.net.MalformedURLException; -import java.net.URL; import java.util.ArrayList; import java.util.List; import java.util.Properties; @@ -75,7 +74,7 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties co } - List classpaths = new ArrayList<>(); + List shipFiles = new ArrayList<>(); if (flinkJarPath != null) { File[] jars = new File(flinkJarPath).listFiles(); @@ -83,15 +82,14 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties co if (file.toURI().toURL().toString().contains("flink-dist")){ clusterDescriptor.setLocalJarPath(new Path(file.toURI().toURL().toString())); } else { - classpaths.add(file.toURI().toURL()); + shipFiles.add(file); } } } else { throw new RuntimeException("The Flink jar path is null"); } - - //clusterDescriptor.setProvidedUserJarFiles(classpaths); + clusterDescriptor.addShipFiles(shipFiles); if(!Strings.isNullOrEmpty(queue)){ clusterDescriptor.setQueue(queue); diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java index cb5d8482d..ccc696a54 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java @@ -22,12 +22,11 @@ import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.io.Charsets; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.shaded.guava18.com.google.common.base.Strings; -import org.apache.flink.shaded.guava18.com.google.common.collect.Sets; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.slf4j.Logger; @@ -36,7 +35,10 @@ import java.net.MalformedURLException; import java.net.URL; import java.net.URLDecoder; -import java.util.*; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Properties; /** * per job mode submitter @@ -54,7 +56,7 @@ public static String submit(Options launcherOptions, JobGraph jobGraph) throws E fillJobGraphClassPath(jobGraph); if (!StringUtils.isBlank(launcherOptions.getAddjar())) { String addjarPath = URLDecoder.decode(launcherOptions.getAddjar(), Charsets.UTF_8.toString()); - List paths = getJarPaths(addjarPath); + List paths = getJarPaths(addjarPath); paths.forEach( path -> { jobGraph.addJar(new Path("file://" + path)); }); @@ -94,31 +96,11 @@ private static List getJarPaths(String addjarPath) { } private static void fillJobGraphClassPath(JobGraph jobGraph) throws MalformedURLException { - Map jobCacheFileConfig = jobGraph.getJobConfiguration().toMap(); - Set classPathKeySet = Sets.newHashSet(); - - for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ - if(Strings.isNullOrEmpty(tmp.getValue())){ - continue; - } - - if(tmp.getValue().startsWith("class_path")){ - //DISTRIBUTED_CACHE_FILE_NAME_1 - //DISTRIBUTED_CACHE_FILE_PATH_1 - String key = tmp.getKey(); - String[] array = key.split("_"); - if(array.length < 5){ - continue; - } - - array[3] = "PATH"; - classPathKeySet.add(StringUtils.join(array, "_")); - } - } - - for(String key : classPathKeySet){ - String pathStr = jobCacheFileConfig.get(key); - jobGraph.getClasspaths().add(new URL("file:" + pathStr)); - } + Map jobCacheFileConfig = jobGraph.getUserArtifacts(); + for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ + if(tmp.getKey().startsWith("class_path")){ + jobGraph.getClasspaths().add(new URL("file:" + tmp.getValue().filePath)); + } + } } } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/provider/DTC3P0DataSourceProvider.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/provider/DTC3P0DataSourceProvider.java index d4f5e7032..fcca4f0ef 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/provider/DTC3P0DataSourceProvider.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/provider/DTC3P0DataSourceProvider.java @@ -1,3 +1,21 @@ +/* + * 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 com.dtstack.flink.sql.side.rdb.provider; import com.mchange.v2.c3p0.ComboPooledDataSource; From ce1f81c8a65787baacc69bcf448f3b9c124a571a Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 23 Sep 2019 19:46:37 +0800 Subject: [PATCH 396/470] modify README --- README.md | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index 572b604a5..55b6e5cba 100644 --- a/README.md +++ b/README.md @@ -7,18 +7,24 @@ > > * 支持原生FLinkSQL所有的语法 > > * 扩展了输入和输出的性能指标到promethus + ## 新特性: + 1.kafka源表支持not null语法,支持字符串类型的时间转换。 + 2.rdb维表与DB建立连接时,周期进行连接,防止连接断开。rdbsink写入时,对连接进行检查。 + 3.异步维表支持非等值连接,比如:<>,<,>。 + + ## BUG修复: + 1.修复不能解析sql中orderby,union语法。 + 2.修复yarnPer模式提交失败的异常。 # 已支持 * 源表:kafka 0.9,1.x版本 - * 维表:mysql,SQlServer,oracle,hbase,mongo,redis,cassandra - * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra + * 维表:mysql,SQlServer,oracle,hbase,mongo,redis,cassandra,serversocket + * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra,console # 后续开发计划 * 增加SQL支持CEP * 维表快照 * sql优化(谓词下移等) - * serverSocket 源表 - * console 结果表 * kafka avro格式 * topN From 517b1073d54bfb600bcb83a2e8a0b328db6bc173 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 24 Sep 2019 12:14:22 +0800 Subject: [PATCH 397/470] kafka parallelism --- .../com/dtstack/flink/sql/sink/kafka/KafkaSink.java | 10 +++++++++- .../flink/sql/sink/kafka/table/KafkaSinkParser.java | 4 ++++ .../dtstack/flink/sql/source/kafka/KafkaSource.java | 9 ++++++++- .../com/dtstack/flink/sql/sink/kafka/KafkaSink.java | 9 ++++++++- .../flink/sql/sink/kafka/table/KafkaSinkParser.java | 4 ++++ .../dtstack/flink/sql/source/kafka/KafkaSource.java | 9 ++++++++- .../com/dtstack/flink/sql/sink/kafka/KafkaSink.java | 8 +++++++- .../flink/sql/sink/kafka/table/KafkaSinkParser.java | 4 ++++ .../dtstack/flink/sql/source/kafka/KafkaSource.java | 9 ++++++++- 9 files changed, 60 insertions(+), 6 deletions(-) diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 29b97e6d2..227a300f7 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -53,6 +53,9 @@ public class KafkaSink implements RetractStreamTableSink, IStreamSinkGener< /** Serialization schema for encoding records to Kafka. */ protected SerializationSchema serializationSchema; + protected int parallelism; + + @Override public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { KafkaSinkTableInfo kafka09SinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; @@ -64,6 +67,11 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { } this.fieldTypes = types; + Integer parallelism = kafka09SinkTableInfo.getParallelism(); + if (parallelism != null) { + this.parallelism = parallelism; + } + properties = new Properties(); for (String key : kafka09SinkTableInfo.getKafkaParamKeys()) { properties.setProperty(key, kafka09SinkTableInfo.getKafkaParam(key)); @@ -90,7 +98,7 @@ public void emitDataStream(DataStream> dataStream) { DataStream ds = dataStream.map((Tuple2 record) -> { return record.f1; - }).returns(getOutputType().getTypeAt(1)); + }).returns(getOutputType().getTypeAt(1)).setParallelism(parallelism); kafkaTableSink.emitDataStream(ds); } diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index 3e1f707e2..dd4d4450f 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -42,6 +42,10 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map, IStreamSinkGener< protected Properties properties; + protected int parallelism; + /** Serialization schema for encoding records to Kafka. */ protected SerializationSchema serializationSchema; @@ -69,6 +71,11 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { } this.fieldTypes = types; + Integer parallelism = kafka10SinkTableInfo.getParallelism(); + if (parallelism != null) { + this.parallelism = parallelism; + } + properties = new Properties(); for (String key : kafka10SinkTableInfo.getKafkaParamKeys()) { properties.setProperty(key, kafka10SinkTableInfo.getKafkaParam(key)); @@ -95,7 +102,7 @@ public void emitDataStream(DataStream> dataStream) { DataStream ds = dataStream.map((Tuple2 record) -> { return record.f1; - }).returns(getOutputType().getTypeAt(1)); + }).returns(getOutputType().getTypeAt(1)).setParallelism(parallelism); kafkaTableSink.emitDataStream(ds); } diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index 9f250fc03..073fe9111 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -42,6 +42,10 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map, IStreamSinkGener protected String topic; + protected int parallelism; + protected Properties properties; /** Serialization schema for encoding records to Kafka. */ @@ -67,6 +69,11 @@ public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { } this.fieldTypes = types; + Integer parallelism = kafka11SinkTableInfo.getParallelism(); + if (parallelism != null) { + this.parallelism = parallelism; + } + properties = new Properties(); for (String key : kafka11SinkTableInfo.getKafkaParamKeys()) { properties.setProperty(key, kafka11SinkTableInfo.getKafkaParam(key)); @@ -89,7 +96,6 @@ public void emitDataStream(DataStream> dataStream) { serializationSchema ); - DataStream ds = dataStream.map((Tuple2 record) -> { return record.f1; }).returns(getOutputType().getTypeAt(1)); diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java index 4ee9d99f0..5a6e8fa79 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -42,6 +42,10 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Tue, 24 Sep 2019 12:16:58 +0800 Subject: [PATCH 398/470] kafka11 parallelism --- .../main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java index 3a814ca1d..450a30b21 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -98,7 +98,7 @@ public void emitDataStream(DataStream> dataStream) { DataStream ds = dataStream.map((Tuple2 record) -> { return record.f1; - }).returns(getOutputType().getTypeAt(1)); + }).returns(getOutputType().getTypeAt(1)).setParallelism(parallelism); kafkaTableSink.emitDataStream(ds); } From d65e7db2264fb7e4120825bfaadf99b2ef3cb645 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 24 Sep 2019 15:00:28 +0800 Subject: [PATCH 399/470] oracle sink code opt --- .../flink/sql/sink/oracle/table/OracleSinkParser.java | 6 +++--- .../flink/sql/sink/rdb/format/ExtendOutputFormat.java | 11 ++++++----- .../sql/sink/rdb/format/RetractJDBCOutputFormat.java | 10 +++++++++- 3 files changed, 18 insertions(+), 9 deletions(-) diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java index 6db2c9c06..aff096bd3 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/table/OracleSinkParser.java @@ -34,8 +34,8 @@ public class OracleSinkParser extends RdbSinkParser { @Override public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { - TableInfo sqlserverTableInfo = super.getTableInfo(tableName, fieldsInfo, props); - sqlserverTableInfo.setType(CURR_TYPE); - return sqlserverTableInfo; + TableInfo oracleTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + oracleTableInfo.setType(CURR_TYPE); + return oracleTableInfo; } } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index 93391b713..9d6c39338 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -66,25 +66,26 @@ public void fillRealIndexes() throws SQLException { while (rs.next()) { String indexName = rs.getString("INDEX_NAME"); - if (!map.containsKey(indexName)) { + if (StringUtils.isNotBlank(indexName) && !map.containsKey(indexName)) { map.put(indexName, new ArrayList<>()); } String column_name = rs.getString("COLUMN_NAME"); if (StringUtils.isNotBlank(column_name)) { - column_name = column_name.toUpperCase(); + map.get(indexName).add(column_name); } - map.get(indexName).add(column_name); } for (Map.Entry> entry : map.entrySet()) { String k = entry.getKey(); List v = entry.getValue(); if (v != null && v.size() != 0 && v.get(0) != null) { - getRealIndexes().put(k, v); + realIndexesAdd(k, v); } } } + + /** * get db all column name * @@ -95,7 +96,7 @@ public void fillFullColumns() throws SQLException { while (rs.next()) { String columnName = rs.getString("COLUMN_NAME"); if (StringUtils.isNotBlank(columnName)) { - getFullField().add(columnName.toUpperCase()); + fullFieldAdd(columnName); } } } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 40cb712a9..cc5065913 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -452,7 +452,11 @@ public String getTableName() { return tableName; } - public Map> getRealIndexes() { + public void realIndexesAdd(String index, List fieldes) { + this.realIndexes.put(index, fieldes); + } + + public Map> getRealIndexes() { return realIndexes; } @@ -464,4 +468,8 @@ public void setBatchWaitInterval(long batchWaitInterval) { public List getFullField() { return fullField; } + + public void fullFieldAdd(String colName) { + this.fullField.add(colName); + } } From ab3ad9b19aac4687d4e3609dfedc68c6ac807bce Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Wed, 25 Sep 2019 11:43:10 +0800 Subject: [PATCH 400/470] =?UTF-8?q?udaf=E5=87=BD=E6=95=B0=E6=94=B9?= =?UTF-8?q?=E4=B8=BA=E4=B8=8Escala=E5=92=8Ctable=E5=87=BD=E6=95=B0?= =?UTF-8?q?=E5=B9=B6=E8=A1=8C=E5=AD=98=E5=9C=A8=E7=9A=84=E6=96=B9=E5=BC=8F?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/parser/CreateFuncParser.java | 2 +- .../com/dtstack/flink/sql/util/FlinkUtil.java | 67 +++++++++++-------- 2 files changed, 41 insertions(+), 28 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java index 793dd6baa..670d98a7e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateFuncParser.java @@ -32,7 +32,7 @@ public class CreateFuncParser implements IParser { - private static final String funcPatternStr = "(?i)\\s*create\\s+(scala|table)\\s+function\\s+(\\S+)\\s+WITH\\s+(\\S+)"; + private static final String funcPatternStr = "(?i)\\s*create\\s+(scala|table|aggregate)\\s+function\\s+(\\S+)\\s+WITH\\s+(\\S+)"; private static final Pattern funcPattern = Pattern.compile(funcPatternStr); diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java index fe342fe5b..d4524ceca 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java @@ -33,7 +33,6 @@ import org.apache.flink.table.api.java.StreamTableEnvironment; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.TableFunction; -import org.apache.flink.table.functions.UserDefinedFunction; import org.apache.flink.table.functions.AggregateFunction; import org.slf4j.Logger; @@ -148,10 +147,8 @@ public static void setStreamTimeCharacteristic(StreamExecutionEnvironment env, P } - /** - * FIXME 暂时不支持 UDF 实现类--有参构造方法 - * TABLE|SCALA + * TABLE|SCALA|AGGREGATE * 注册UDF到table env */ public static void registerUDF(String type, String classPath, String funcName, TableEnvironment tableEnv, @@ -160,10 +157,11 @@ public static void registerUDF(String type, String classPath, String funcName, T registerScalaUDF(classPath, funcName, tableEnv, classLoader); }else if("TABLE".equalsIgnoreCase(type)){ registerTableUDF(classPath, funcName, tableEnv, classLoader); + }else if("AGGREGATE".equalsIgnoreCase(type)){ + registerAggregateUDF(classPath, funcName, tableEnv, classLoader); }else{ - throw new RuntimeException("not support of UDF which is not in (TABLE, SCALA)"); + throw new RuntimeException("not support of UDF which is not in (TABLE, SCALA, AGGREGATE)"); } - } /** @@ -187,7 +185,7 @@ public static void registerScalaUDF(String classPath, String funcName, TableEnvi /** * 注册自定义TABLEFFUNC方法到env上 - * TODO 对User-Defined Aggregate Functions的支持 + * * @param classPath * @param funcName * @param tableEnv @@ -195,26 +193,14 @@ public static void registerScalaUDF(String classPath, String funcName, TableEnvi public static void registerTableUDF(String classPath, String funcName, TableEnvironment tableEnv, ClassLoader classLoader){ try { - UserDefinedFunction udfFunc = Class.forName(classPath,false, classLoader) - .asSubclass(UserDefinedFunction.class).newInstance(); - - if(tableEnv instanceof StreamTableEnvironment){ - if (udfFunc instanceof AggregateFunction){ - ((StreamTableEnvironment) tableEnv).registerFunction(funcName, (AggregateFunction)udfFunc); - }else if (udfFunc instanceof TableFunction) { - ((StreamTableEnvironment) tableEnv).registerFunction(funcName, (TableFunction)udfFunc); - }else{ - throw new RuntimeException("no support UserDefinedFunction class for " + udfFunc.getClass().getName()); - } - }else if(tableEnv instanceof BatchTableEnvironment){ - if (udfFunc instanceof AggregateFunction){ - ((BatchTableEnvironment) tableEnv).registerFunction(funcName, (AggregateFunction)udfFunc); - }else if (udfFunc instanceof TableFunction) { - ((BatchTableEnvironment) tableEnv).registerFunction(funcName, (TableFunction)udfFunc); - }else{ - throw new RuntimeException("no support UserDefinedFunction class for " + udfFunc.getClass().getName()); - } - }else{ + TableFunction udfFunc = Class.forName(classPath, false, classLoader) + .asSubclass(TableFunction.class).newInstance(); + + if (tableEnv instanceof StreamTableEnvironment) { + ((StreamTableEnvironment) tableEnv).registerFunction(funcName, udfFunc); + } else if (tableEnv instanceof BatchTableEnvironment) { + ((BatchTableEnvironment) tableEnv).registerFunction(funcName, udfFunc); + } else { throw new RuntimeException("no support tableEnvironment class for " + tableEnv.getClass().getName()); } @@ -225,6 +211,33 @@ public static void registerTableUDF(String classPath, String funcName, TableEnvi } } + /** + * 注册自定义Aggregate FUNC方法到env上 + * + * @param classPath + * @param funcName + * @param tableEnv + */ + public static void registerAggregateUDF(String classPath, String funcName, TableEnvironment tableEnv, + ClassLoader classLoader) { + try { + AggregateFunction udfFunc = Class.forName(classPath, false, classLoader) + .asSubclass(AggregateFunction.class).newInstance(); + + if (tableEnv instanceof StreamTableEnvironment) { + ((StreamTableEnvironment) tableEnv).registerFunction(funcName, udfFunc); + } else if (tableEnv instanceof BatchTableEnvironment) { + ((BatchTableEnvironment) tableEnv).registerFunction(funcName, udfFunc); + } else { + throw new RuntimeException("no support tableEnvironment class for " + tableEnv.getClass().getName()); + } + + logger.info("register Aggregate function:{} success.", funcName); + } catch (Exception e) { + logger.error("", e); + throw new RuntimeException("register Aggregate UDF exception:", e); + } + } /** * From bf2d82dce20c79d98efb11c1496afbdf85deeec5 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 25 Sep 2019 14:41:49 +0800 Subject: [PATCH 401/470] import class --- core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index ab5d821f1..ab3c47a8a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -38,6 +38,7 @@ import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlOrderBy; import org.apache.calcite.sql.SqlSelect; import org.apache.calcite.sql.fun.SqlCase; import org.apache.calcite.sql.parser.SqlParseException; From c823549fb434effd976e7901576e0f98e2a2af27 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 25 Sep 2019 21:16:10 +0800 Subject: [PATCH 402/470] rdb side sql error --- .../dtstack/flink/sql/util/ConfigConstrant.java | 10 ++++++++-- .../com/dtstack/flink/sql/util/FlinkUtil.java | 17 +++++++++++++---- .../sql/side/rdb/async/RdbAsyncSideInfo.java | 2 +- 3 files changed, 22 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ConfigConstrant.java b/core/src/main/java/com/dtstack/flink/sql/util/ConfigConstrant.java index 160d8f411..dd1755621 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ConfigConstrant.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ConfigConstrant.java @@ -29,7 +29,9 @@ */ public class ConfigConstrant { - public static final String FLINK_CHECKPOINT_INTERVAL_KEY = "sql.checkpoint.interval"; + public static final String SQL_CHECKPOINT_INTERVAL_KEY = "sql.checkpoint.interval"; + // 兼容上层 + public static final String FLINK_CHECKPOINT_INTERVAL_KEY = "flink.checkpoint.interval"; public static final String FLINK_CHECKPOINT_MODE_KEY = "sql.checkpoint.mode"; @@ -37,7 +39,11 @@ public class ConfigConstrant { public static final String FLINK_MAXCONCURRENTCHECKPOINTS_KEY = "sql.max.concurrent.checkpoints"; - public static final String FLINK_CHECKPOINT_CLEANUPMODE_KEY = "sql.checkpoint.cleanup.mode"; + public static final String SQL_CHECKPOINT_CLEANUPMODE_KEY = "sql.checkpoint.cleanup.mode"; + + public static final String FLINK_CHECKPOINT_CLEANUPMODE_KEY = "flink.checkpoint.cleanup.mode"; + + public static final String FLINK_CHECKPOINT_DATAURI_KEY = "flinkCheckpointDataURI"; diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java index 2dc2a0bf3..b14cf809f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java @@ -67,12 +67,14 @@ public static void openCheckpoint(StreamExecutionEnvironment env, Properties pro } //设置了时间间隔才表明开启了checkpoint - if(properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_INTERVAL_KEY) == null){ + if(properties.getProperty(ConfigConstrant.SQL_CHECKPOINT_INTERVAL_KEY) == null && properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_INTERVAL_KEY) == null){ return; }else{ - Long interval = Long.valueOf(properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_INTERVAL_KEY)); + Long sql_interval = Long.valueOf(properties.getProperty(ConfigConstrant.SQL_CHECKPOINT_INTERVAL_KEY,"0")); + Long flink_interval = Long.valueOf(properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_INTERVAL_KEY, "0")); + long checkpointInterval = Math.max(sql_interval, flink_interval); //start checkpoint every ${interval} - env.enableCheckpointing(interval); + env.enableCheckpointing(checkpointInterval); } String checkMode = properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_MODE_KEY); @@ -100,7 +102,14 @@ public static void openCheckpoint(StreamExecutionEnvironment env, Properties pro env.getCheckpointConfig().setMaxConcurrentCheckpoints(maxConcurrCheckpoints); } - String cleanupModeStr = properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_CLEANUPMODE_KEY); + Boolean sqlCleanMode = MathUtil.getBoolean(properties.getProperty(ConfigConstrant.SQL_CHECKPOINT_CLEANUPMODE_KEY), false); + Boolean flinkCleanMode = MathUtil.getBoolean(properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_CLEANUPMODE_KEY), false); + + String cleanupModeStr = "false"; + if (sqlCleanMode || flinkCleanMode ){ + cleanupModeStr = "true"; + } + if ("true".equalsIgnoreCase(cleanupModeStr)){ env.getCheckpointConfig().enableExternalizedCheckpoints( CheckpointConfig.ExternalizedCheckpointCleanup.DELETE_ON_CANCELLATION); diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java index 3f3fb323d..aba1d6214 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java @@ -72,7 +72,7 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { for (int i = 0; i < equalFieldList.size(); i++) { String equalField = sideTableInfo.getPhysicalFields().getOrDefault(equalFieldList.get(i), equalFieldList.get(i)); - sqlCondition += equalField + "\t" + sqlJoinCompareOperate.get(i) + " ?"; + sqlCondition += equalField + " " + sqlJoinCompareOperate.get(i) + " ? "; if (i != equalFieldList.size() - 1) { sqlCondition += " and "; } From a0454cb3132f70264568cff02df372b21e42d29c Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Thu, 26 Sep 2019 14:41:51 +0800 Subject: [PATCH 403/470] properties field trim --- core/src/main/java/com/dtstack/flink/sql/Main.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 80c81eb71..6cb6c9f0f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -329,6 +329,8 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en } private static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws Exception { + confProperties = propertiesTrim(confProperties); + StreamExecutionEnvironment env = !ClusterMode.local.name().equals(deployMode) ? StreamExecutionEnvironment.getExecutionEnvironment() : new MyLocalStreamEnvironment(); @@ -370,4 +372,14 @@ private static StreamExecutionEnvironment getStreamExeEnv(Properties confPropert return env; } + + private static Properties propertiesTrim(Properties confProperties) { + Properties properties = new Properties(); + confProperties.forEach( + (k, v) -> { + properties.put(k.toString().trim(), v.toString().trim()); + } + ); + return properties; + } } From 1900e734a2487fe9df1fd50d3b902826f8e1ced4 Mon Sep 17 00:00:00 2001 From: gezhihui Date: Thu, 26 Sep 2019 17:55:51 +0800 Subject: [PATCH 404/470] =?UTF-8?q?yarn=20session=E6=A8=A1=E5=BC=8F?= =?UTF-8?q?=E4=B8=8B=E5=8F=AF=E4=BB=A5=E6=8C=87=E5=AE=9AapplicationID?= =?UTF-8?q?=E6=8F=90=E4=BA=A4=E4=BB=BB=E5=8A=A1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 7 ++++- .../com/dtstack/flink/sql/option/Options.java | 11 +++++++ .../sql/launcher/ClusterClientFactory.java | 29 ++++++++++++++++++- 3 files changed, 45 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 55b6e5cba..3c4c527a8 100644 --- a/README.md +++ b/README.md @@ -58,7 +58,7 @@ mvn clean package -Dmaven.test.skip #### 1.4.1 启动命令 ``` -sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack/150_flinkplugin/sqlplugin -localSqlPluginPath D:\gitspace\flinkStreamSQL\plugins -addjar \["udf.jar\"\] -mode yarn -flinkconf D:\flink_home\kudu150etc -yarnconf D:\hadoop\etc\hadoopkudu -confProp \{\"time.characteristic\":\"EventTime\",\"sql.checkpoint.interval\":10000\} +sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack/150_flinkplugin/sqlplugin -localSqlPluginPath D:\gitspace\flinkStreamSQL\plugins -addjar \["udf.jar\"\] -mode yarn -flinkconf D:\flink_home\kudu150etc -yarnconf D:\hadoop\etc\hadoopkudu -confProp \{\"time.characteristic\":\"EventTime\",\"sql.checkpoint.interval\":10000\} -yarnSessionConf \{\"yid\":\"application_1564971615273_38182\"} ``` #### 1.4.2 命令行参数选项 @@ -149,6 +149,11 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * 描述:per_job 模式下指定的yarn queue * 必选:否 * 默认值:false + +* **yarnSessionConf** + * 描述:yarn session 模式下指定的运行的一些参数,[可参考](https://ci.apache.org/projects/flink/flink-docs-release-1.8/ops/cli.html),目前只支持指定yid + * 必选:否 + * 默认值:false ## 2 结构 ### 2.1 源表插件 diff --git a/core/src/main/java/com/dtstack/flink/sql/option/Options.java b/core/src/main/java/com/dtstack/flink/sql/option/Options.java index d90d6c4a6..eef54a617 100644 --- a/core/src/main/java/com/dtstack/flink/sql/option/Options.java +++ b/core/src/main/java/com/dtstack/flink/sql/option/Options.java @@ -68,6 +68,9 @@ public class Options { @OptionRequired(description = "yarn queue") private String queue = "default"; + @OptionRequired(description = "yarn session configuration,such as yid") + private String yarnSessionConf = "{}"; + public String getMode() { return mode; } @@ -171,4 +174,12 @@ public String getQueue() { public void setQueue(String queue) { this.queue = queue; } + + public String getYarnSessionConf() { + return yarnSessionConf; + } + + public void setYarnSessionConf(String yarnSessionConf) { + this.yarnSessionConf = yarnSessionConf; + } } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java index 125ee301a..ffabc7002 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/ClusterClientFactory.java @@ -51,6 +51,8 @@ import com.dtstack.flink.sql.enums.ClusterMode; import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.util.StringHelper; + import java.io.IOException; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -113,7 +115,15 @@ public static ClusterClient createYarnClient(Options launcherOptions, String mod ApplicationId applicationId = null; ClusterClient clusterClient = null; if(mode.equals(ClusterMode.yarn.name())) {//on yarn cluster mode - applicationId = getYarnClusterApplicationId(yarnClient); + String yarnSessionConf = launcherOptions.getYarnSessionConf(); + yarnSessionConf = URLDecoder.decode(yarnSessionConf, Charsets.UTF_8.toString()); + Properties yarnSessionConfProperties = PluginUtil.jsonStrToObject(yarnSessionConf, Properties.class); + String yid = yarnSessionConfProperties.get("yid").toString(); + if(StringUtils.isNotBlank(yid)){ + applicationId = toApplicationId(yid); + }else{ + applicationId = getYarnClusterApplicationId(yarnClient); + } System.out.println("applicationId="+applicationId.toString()); AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( @@ -249,4 +259,21 @@ private static org.apache.hadoop.conf.Configuration haYarnConf(org.apache.hadoop return yarnConf; } + private static ApplicationId toApplicationId(String appIdStr) { + Iterator it = StringHelper._split(appIdStr).iterator(); + if (!(it.next()).equals("application")) { + throw new IllegalArgumentException("Invalid ApplicationId prefix: " + appIdStr + ". The valid ApplicationId should start with prefix " + "application"); + } else { + try { + return toApplicationId(it); + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Invalid AppAttemptId: " + appIdStr, e); + } + } + } + + private static ApplicationId toApplicationId(Iterator it) throws NumberFormatException { + return ApplicationId.newInstance(Long.parseLong(it.next()), Integer.parseInt(it.next())); + } + } \ No newline at end of file From d742bdf1e3abaf6efba0d7464012adc5f7377087 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 27 Sep 2019 10:12:20 +0800 Subject: [PATCH 405/470] modify oracle mergeinto sql --- .../flink/sql/sink/oracle/OracleSink.java | 21 +++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index d8669b11f..49390dc64 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -76,12 +76,25 @@ private void buildInsertSql(String tableName, List fields) { @Override public String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField) { tableName = quoteTable(tableName); - return "MERGE INTO " + tableName + " T1 USING " + StringBuilder sb = new StringBuilder(); + + sb.append("MERGE INTO " + tableName + " T1 USING " + "(" + makeValues(fieldNames) + ") T2 ON (" - + updateKeySql(realIndexes) + ") WHEN MATCHED THEN UPDATE SET " - + getUpdateSql(fieldNames, fullField, "T1", "T2", keyColList(realIndexes)) + " WHEN NOT MATCHED THEN " + + updateKeySql(realIndexes) + ") "); + + + String updateSql = getUpdateSql(fieldNames, fullField, "T1", "T2", keyColList(realIndexes)); + + if (StringUtils.isNotEmpty(updateSql)) { + sb.append(" WHEN MATCHED THEN UPDATE SET "); + sb.append(updateSql); + } + + sb.append(" WHEN NOT MATCHED THEN " + "INSERT (" + quoteColumns(fieldNames) + ") VALUES (" - + quoteColumns(fieldNames, "T2") + ")"; + + quoteColumns(fieldNames, "T2") + ")"); + + return sb.toString(); } From 6bdcab215f20cbef8faf8d5ba6c4e989ac24bd62 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 27 Sep 2019 11:35:10 +0800 Subject: [PATCH 406/470] add PropertiesUtils --- .../main/java/com/dtstack/flink/sql/Main.java | 12 +----- .../dtstack/flink/sql/udf/TimestampUdf.java | 19 ++++++++++ .../com/dtstack/flink/sql/util/JDBCUtils.java | 19 ++++++++++ .../dtstack/flink/sql/util/ParseUtils.java | 38 ++++++++++++++++++- .../flink/sql/util/PropertiesUtils.java | 34 +++++++++++++++++ .../watermarker/AbsCustomerWaterMarker.java | 19 ++++++++++ 6 files changed, 129 insertions(+), 12 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/util/PropertiesUtils.java diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 6cb6c9f0f..c81f04e21 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -37,6 +37,7 @@ import com.dtstack.flink.sql.sink.StreamSinkFactory; import com.dtstack.flink.sql.source.StreamSourceFactory; import com.dtstack.flink.sql.util.DtStringUtil; +import com.dtstack.flink.sql.util.PropertiesUtils; import com.dtstack.flink.sql.watermarker.WaterMarkerAssigner; import com.dtstack.flink.sql.util.FlinkUtil; import com.dtstack.flink.sql.util.PluginUtil; @@ -329,7 +330,7 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en } private static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws Exception { - confProperties = propertiesTrim(confProperties); + confProperties = PropertiesUtils.propertiesTrim(confProperties); StreamExecutionEnvironment env = !ClusterMode.local.name().equals(deployMode) ? StreamExecutionEnvironment.getExecutionEnvironment() : @@ -373,13 +374,4 @@ private static StreamExecutionEnvironment getStreamExeEnv(Properties confPropert return env; } - private static Properties propertiesTrim(Properties confProperties) { - Properties properties = new Properties(); - confProperties.forEach( - (k, v) -> { - properties.put(k.toString().trim(), v.toString().trim()); - } - ); - return properties; - } } diff --git a/core/src/main/java/com/dtstack/flink/sql/udf/TimestampUdf.java b/core/src/main/java/com/dtstack/flink/sql/udf/TimestampUdf.java index 9f605dde3..33cba2952 100644 --- a/core/src/main/java/com/dtstack/flink/sql/udf/TimestampUdf.java +++ b/core/src/main/java/com/dtstack/flink/sql/udf/TimestampUdf.java @@ -1,3 +1,22 @@ +/* + * 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 com.dtstack.flink.sql.udf; import org.apache.flink.table.functions.FunctionContext; diff --git a/core/src/main/java/com/dtstack/flink/sql/util/JDBCUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/JDBCUtils.java index a63fc0ab4..fde2f166e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/JDBCUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/JDBCUtils.java @@ -1,3 +1,22 @@ +/* + * 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 com.dtstack.flink.sql.util; import org.slf4j.Logger; diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java index 25613d73d..accbcdd1a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java @@ -1,3 +1,39 @@ +/* + * 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. + */ + +/* + * 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 com.dtstack.flink.sql.util; import org.apache.calcite.sql.SqlBasicCall; @@ -5,9 +41,7 @@ import org.apache.calcite.sql.SqlNode; import org.apache.commons.lang3.StringUtils; -import java.util.HashSet; import java.util.List; -import java.util.Set; /** * @Auther: jiangjunjie diff --git a/core/src/main/java/com/dtstack/flink/sql/util/PropertiesUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/PropertiesUtils.java new file mode 100644 index 000000000..dcb2a081a --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/util/PropertiesUtils.java @@ -0,0 +1,34 @@ +/* + * 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 com.dtstack.flink.sql.util; + +import java.util.Properties; + +public class PropertiesUtils { + public static Properties propertiesTrim(Properties confProperties) { + Properties properties = new Properties(); + confProperties.forEach( + (k, v) -> { + properties.put(k.toString().trim(), v.toString().trim()); + } + ); + return properties; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/AbsCustomerWaterMarker.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/AbsCustomerWaterMarker.java index b990bdd82..c9fbc0f44 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/AbsCustomerWaterMarker.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/AbsCustomerWaterMarker.java @@ -1,3 +1,22 @@ +/* + * 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 com.dtstack.flink.sql.watermarker; import com.dtstack.flink.sql.metric.EventDelayGauge; From 2f9bce17ee74360094ab6d99460660be0acaed38 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 27 Sep 2019 14:31:47 +0800 Subject: [PATCH 407/470] oracle merge into midify --- .../flink/sql/sink/oracle/OracleSink.java | 46 ++++++++++++++++--- 1 file changed, 39 insertions(+), 7 deletions(-) diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index 49390dc64..0945e223d 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -73,6 +73,14 @@ private void buildInsertSql(String tableName, List fields) { this.sql = sqlTmp; } + /** + * use MERGE INTO build oracle replace into sql + * @param tableName + * @param fieldNames create table contained column columns + * @param realIndexes + * @param fullField real columns , query from db + * @return + */ @Override public String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField) { tableName = quoteTable(tableName); @@ -111,9 +119,14 @@ public String quoteColumns(List column, String table) { return StringUtils.join(list, ","); } - protected List keyColList(Map> updateKey) { + /** + * extract all distinct index column + * @param realIndexes + * @return + */ + protected List keyColList(Map> realIndexes) { List keyCols = new ArrayList<>(); - for (Map.Entry> entry : updateKey.entrySet()) { + for (Map.Entry> entry : realIndexes.entrySet()) { List list = entry.getValue(); for (String col : list) { if (!containsIgnoreCase(keyCols,col)) { @@ -124,15 +137,25 @@ protected List keyColList(Map> updateKey) { return keyCols; } - public String getUpdateSql(List column, List fullColumn, String leftTable, String rightTable, List keyCols) { + /** + * build update sql , such as UPDATE SET "T1".A="T2".A + * @param updateColumn create table contained column columns + * @param fullColumn real columns , query from db + * @param leftTable alias + * @param rightTable alias + * @param indexCols index column + * @return + */ + public String getUpdateSql(List updateColumn, List fullColumn, String leftTable, String rightTable, List indexCols) { String prefixLeft = StringUtils.isBlank(leftTable) ? "" : quoteTable(leftTable) + "."; String prefixRight = StringUtils.isBlank(rightTable) ? "" : quoteTable(rightTable) + "."; List list = new ArrayList<>(); for (String col : fullColumn) { - if (keyCols == null || keyCols.size() == 0 || containsIgnoreCase(keyCols,col)) { + // filter index column + if (indexCols == null || indexCols.size() == 0 || containsIgnoreCase(indexCols,col)) { continue; } - if (fullColumn == null ||containsIgnoreCase(column,col)) { + if (containsIgnoreCase(updateColumn,col)) { list.add(prefixLeft + col + "=" + prefixRight + col); } else { list.add(prefixLeft + col + "=null"); @@ -153,7 +176,11 @@ public String quoteTable(String table) { return sb.toString(); } - + /** + * build connect sql by index column, such as T1."A"=T2."A" + * @param updateKey + * @return + */ public String updateKeySql(Map> updateKey) { List exprList = new ArrayList<>(); for (Map.Entry> entry : updateKey.entrySet()) { @@ -166,7 +193,12 @@ public String updateKeySql(Map> updateKey) { return StringUtils.join(exprList, " OR "); } - + /** + * build select sql , such as (SELECT ? "A",? "B" FROM DUAL) + * + * @param column destination column + * @return + */ public String makeValues(List column) { StringBuilder sb = new StringBuilder("SELECT "); for (int i = 0; i < column.size(); ++i) { From 6a16bd30940c8dbb54118931935e73816c87ec6c Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Sun, 29 Sep 2019 17:23:24 +0800 Subject: [PATCH 408/470] deal oracle table name --- .../dtstack/flink/sql/util/DtStringUtil.java | 25 +++++++++++ .../flink/sql/sink/oracle/OracleSink.java | 45 +++++-------------- .../sink/rdb/format/ExtendOutputFormat.java | 4 +- 3 files changed, 40 insertions(+), 34 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index e07f79a4d..50af1c47c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -248,7 +248,32 @@ public static Object parse(String str,Class clazz){ return object; } + public static String firstUpperCase(String str) { return str.substring(0, 1).toUpperCase() + str.substring(1); } + + public static String quoteTable(String table) { + String[] parts = table.split("\\."); + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < parts.length; ++i) { + if (i != 0) { + sb.append("."); + } + sb.append(DtStringUtil.quoteColumn(parts[i])); + } + return sb.toString(); + } + + public static String quoteColumn(String column) { + return getStartQuote() + column + getEndQuote(); + } + + public static String getStartQuote() { + return "\""; + } + + public static String getEndQuote() { + return "\""; + } } diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index 0945e223d..978359f73 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -21,6 +21,7 @@ import com.dtstack.flink.sql.sink.rdb.RdbSink; import com.dtstack.flink.sql.sink.rdb.format.ExtendOutputFormat; import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; +import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; @@ -56,11 +57,11 @@ public void buildSql(String tableName, List fields) { private void buildInsertSql(String tableName, List fields) { - tableName = quoteTable(tableName); + tableName = DtStringUtil.quoteTable(tableName); String sqlTmp = "insert into " + tableName + " (${fields}) values (${placeholder})"; List adaptFields = Lists.newArrayList(); - fields.forEach(field -> adaptFields.add(quoteColumn(field))); + fields.forEach(field -> adaptFields.add(DtStringUtil.quoteColumn(field))); String fieldsStr = StringUtils.join(adaptFields, ","); String placeholder = ""; @@ -83,7 +84,7 @@ private void buildInsertSql(String tableName, List fields) { */ @Override public String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField) { - tableName = quoteTable(tableName); + tableName = DtStringUtil.quoteTable(tableName); StringBuilder sb = new StringBuilder(); sb.append("MERGE INTO " + tableName + " T1 USING " @@ -111,10 +112,10 @@ public String quoteColumns(List column) { } public String quoteColumns(List column, String table) { - String prefix = StringUtils.isBlank(table) ? "" : quoteTable(table) + "."; + String prefix = StringUtils.isBlank(table) ? "" : DtStringUtil.quoteTable(table) + "."; List list = new ArrayList<>(); for (String col : column) { - list.add(prefix + quoteColumn(col)); + list.add(prefix + DtStringUtil.quoteColumn(col)); } return StringUtils.join(list, ","); } @@ -147,8 +148,8 @@ protected List keyColList(Map> realIndexes) { * @return */ public String getUpdateSql(List updateColumn, List fullColumn, String leftTable, String rightTable, List indexCols) { - String prefixLeft = StringUtils.isBlank(leftTable) ? "" : quoteTable(leftTable) + "."; - String prefixRight = StringUtils.isBlank(rightTable) ? "" : quoteTable(rightTable) + "."; + String prefixLeft = StringUtils.isBlank(leftTable) ? "" : DtStringUtil.quoteTable(leftTable) + "."; + String prefixRight = StringUtils.isBlank(rightTable) ? "" : DtStringUtil.quoteTable(rightTable) + "."; List list = new ArrayList<>(); for (String col : fullColumn) { // filter index column @@ -156,25 +157,14 @@ public String getUpdateSql(List updateColumn, List fullColumn, S continue; } if (containsIgnoreCase(updateColumn,col)) { - list.add(prefixLeft + col + "=" + prefixRight + col); + list.add(prefixLeft + DtStringUtil.quoteColumn(col) + "=" + prefixRight + DtStringUtil.quoteColumn(col)); } else { - list.add(prefixLeft + col + "=null"); + list.add(prefixLeft + DtStringUtil.quoteColumn(col) + "=null"); } } return StringUtils.join(list, ","); } - public String quoteTable(String table) { - String[] parts = table.split("\\."); - StringBuilder sb = new StringBuilder(); - for (int i = 0; i < parts.length; ++i) { - if (i != 0) { - sb.append("."); - } - sb.append(getStartQuote() + parts[i] + getEndQuote()); - } - return sb.toString(); - } /** * build connect sql by index column, such as T1."A"=T2."A" @@ -186,7 +176,7 @@ public String updateKeySql(Map> updateKey) { for (Map.Entry> entry : updateKey.entrySet()) { List colList = new ArrayList<>(); for (String col : entry.getValue()) { - colList.add("T1." + quoteColumn(col) + "=T2." + quoteColumn(col)); + colList.add("T1." + DtStringUtil.quoteColumn(col) + "=T2." + DtStringUtil.quoteColumn(col)); } exprList.add(StringUtils.join(colList, " AND ")); } @@ -205,7 +195,7 @@ public String makeValues(List column) { if (i != 0) { sb.append(","); } - sb.append("? " + quoteColumn(column.get(i))); + sb.append("? " + DtStringUtil.quoteColumn(column.get(i))); } sb.append(" FROM DUAL"); return sb.toString(); @@ -220,17 +210,6 @@ public boolean containsIgnoreCase(List l, String s) { return false; } - public String quoteColumn(String column) { - return getStartQuote() + column + getEndQuote(); - } - - public String getStartQuote() { - return "\""; - } - - public String getEndQuote() { - return "\""; - } } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index ba8a526ef..3abba2a42 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -18,6 +18,7 @@ package com.dtstack.flink.sql.sink.rdb.format; +import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; @@ -62,7 +63,7 @@ public boolean isReplaceInsertQuery() throws SQLException { */ public void fillRealIndexes() throws SQLException { Map> map = Maps.newHashMap(); - ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, null, getTableName(), true, false); + ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, null, DtStringUtil.quoteTable(getTableName()), true, false); while (rs.next()) { String indexName = rs.getString("INDEX_NAME"); @@ -92,6 +93,7 @@ public void fillRealIndexes() throws SQLException { * @throws SQLException */ public void fillFullColumns() throws SQLException { + // table name not quote ResultSet rs = getDbConn().getMetaData().getColumns(null, null, getTableName(), null); while (rs.next()) { String columnName = rs.getString("COLUMN_NAME"); From fa1996472ce702ef5b1cd0fff6b3adac2a301944 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Sun, 29 Sep 2019 18:09:37 +0800 Subject: [PATCH 409/470] modify method name --- .../dtstack/flink/sql/util/DtStringUtil.java | 6 ++--- .../flink/sql/sink/oracle/OracleSink.java | 22 +++++++++---------- .../sink/rdb/format/ExtendOutputFormat.java | 2 +- 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index 50af1c47c..46a9c5c0e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -253,19 +253,19 @@ public static String firstUpperCase(String str) { return str.substring(0, 1).toUpperCase() + str.substring(1); } - public static String quoteTable(String table) { + public static String addQuoteForTableName(String table) { String[] parts = table.split("\\."); StringBuilder sb = new StringBuilder(); for (int i = 0; i < parts.length; ++i) { if (i != 0) { sb.append("."); } - sb.append(DtStringUtil.quoteColumn(parts[i])); + sb.append(DtStringUtil.addQuoteForColumn(parts[i])); } return sb.toString(); } - public static String quoteColumn(String column) { + public static String addQuoteForColumn(String column) { return getStartQuote() + column + getEndQuote(); } diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index 978359f73..fad3238df 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -57,11 +57,11 @@ public void buildSql(String tableName, List fields) { private void buildInsertSql(String tableName, List fields) { - tableName = DtStringUtil.quoteTable(tableName); + tableName = DtStringUtil.addQuoteForTableName(tableName); String sqlTmp = "insert into " + tableName + " (${fields}) values (${placeholder})"; List adaptFields = Lists.newArrayList(); - fields.forEach(field -> adaptFields.add(DtStringUtil.quoteColumn(field))); + fields.forEach(field -> adaptFields.add(DtStringUtil.addQuoteForColumn(field))); String fieldsStr = StringUtils.join(adaptFields, ","); String placeholder = ""; @@ -84,7 +84,7 @@ private void buildInsertSql(String tableName, List fields) { */ @Override public String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField) { - tableName = DtStringUtil.quoteTable(tableName); + tableName = DtStringUtil.addQuoteForTableName(tableName); StringBuilder sb = new StringBuilder(); sb.append("MERGE INTO " + tableName + " T1 USING " @@ -112,10 +112,10 @@ public String quoteColumns(List column) { } public String quoteColumns(List column, String table) { - String prefix = StringUtils.isBlank(table) ? "" : DtStringUtil.quoteTable(table) + "."; + String prefix = StringUtils.isBlank(table) ? "" : DtStringUtil.addQuoteForTableName(table) + "."; List list = new ArrayList<>(); for (String col : column) { - list.add(prefix + DtStringUtil.quoteColumn(col)); + list.add(prefix + DtStringUtil.addQuoteForColumn(col)); } return StringUtils.join(list, ","); } @@ -148,8 +148,8 @@ protected List keyColList(Map> realIndexes) { * @return */ public String getUpdateSql(List updateColumn, List fullColumn, String leftTable, String rightTable, List indexCols) { - String prefixLeft = StringUtils.isBlank(leftTable) ? "" : DtStringUtil.quoteTable(leftTable) + "."; - String prefixRight = StringUtils.isBlank(rightTable) ? "" : DtStringUtil.quoteTable(rightTable) + "."; + String prefixLeft = StringUtils.isBlank(leftTable) ? "" : DtStringUtil.addQuoteForTableName(leftTable) + "."; + String prefixRight = StringUtils.isBlank(rightTable) ? "" : DtStringUtil.addQuoteForTableName(rightTable) + "."; List list = new ArrayList<>(); for (String col : fullColumn) { // filter index column @@ -157,9 +157,9 @@ public String getUpdateSql(List updateColumn, List fullColumn, S continue; } if (containsIgnoreCase(updateColumn,col)) { - list.add(prefixLeft + DtStringUtil.quoteColumn(col) + "=" + prefixRight + DtStringUtil.quoteColumn(col)); + list.add(prefixLeft + DtStringUtil.addQuoteForColumn(col) + "=" + prefixRight + DtStringUtil.addQuoteForColumn(col)); } else { - list.add(prefixLeft + DtStringUtil.quoteColumn(col) + "=null"); + list.add(prefixLeft + DtStringUtil.addQuoteForColumn(col) + "=null"); } } return StringUtils.join(list, ","); @@ -176,7 +176,7 @@ public String updateKeySql(Map> updateKey) { for (Map.Entry> entry : updateKey.entrySet()) { List colList = new ArrayList<>(); for (String col : entry.getValue()) { - colList.add("T1." + DtStringUtil.quoteColumn(col) + "=T2." + DtStringUtil.quoteColumn(col)); + colList.add("T1." + DtStringUtil.addQuoteForColumn(col) + "=T2." + DtStringUtil.addQuoteForColumn(col)); } exprList.add(StringUtils.join(colList, " AND ")); } @@ -195,7 +195,7 @@ public String makeValues(List column) { if (i != 0) { sb.append(","); } - sb.append("? " + DtStringUtil.quoteColumn(column.get(i))); + sb.append("? " + DtStringUtil.addQuoteForColumn(column.get(i))); } sb.append(" FROM DUAL"); return sb.toString(); diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index 3abba2a42..d5bdc8cd9 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -63,7 +63,7 @@ public boolean isReplaceInsertQuery() throws SQLException { */ public void fillRealIndexes() throws SQLException { Map> map = Maps.newHashMap(); - ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, null, DtStringUtil.quoteTable(getTableName()), true, false); + ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, null, DtStringUtil.addQuoteForTableName(getTableName()), true, false); while (rs.next()) { String indexName = rs.getString("INDEX_NAME"); From 6a6a9528052923d128761e831c675995276f318e Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 30 Sep 2019 12:57:10 +0800 Subject: [PATCH 410/470] =?UTF-8?q?[mysql=E7=BB=B4=E8=A1=A8=E5=85=A8?= =?UTF-8?q?=E9=87=8F=E7=BC=93=E5=AD=98=E6=8A=A5=E7=A9=BA=E6=8C=87=E9=92=88?= =?UTF-8?q?=E5=BC=82=E5=B8=B8=E9=94=99=E8=AF=AF][19145]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java index 26ed7648f..70fe17b84 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java @@ -119,9 +119,8 @@ public void flatMap(Row value, Collector out) throws Exception { for (Integer conValIndex : sideInfo.getEqualValIndex()) { Object equalObj = value.getField(conValIndex); if (equalObj == null) { - out.collect(null); + return; } - inputParams.add(equalObj); } From 6e8d735f0c37ac85a6d5c0de7cd5f1b2a83f858c Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Tue, 8 Oct 2019 14:05:15 +0800 Subject: [PATCH 411/470] =?UTF-8?q?=E5=88=A0=E9=99=A4timestampudf?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flink/sql/udf/TimestampUdf.java | 43 ------------------- 1 file changed, 43 deletions(-) delete mode 100644 core/src/main/java/com/dtstack/flink/sql/udf/TimestampUdf.java diff --git a/core/src/main/java/com/dtstack/flink/sql/udf/TimestampUdf.java b/core/src/main/java/com/dtstack/flink/sql/udf/TimestampUdf.java deleted file mode 100644 index 33cba2952..000000000 --- a/core/src/main/java/com/dtstack/flink/sql/udf/TimestampUdf.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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 com.dtstack.flink.sql.udf; - -import org.apache.flink.table.functions.FunctionContext; -import org.apache.flink.table.functions.ScalarFunction; - -import java.sql.Timestamp; - -public class TimestampUdf extends ScalarFunction { - @Override - public void open(FunctionContext context) { - } - public static Timestamp eval(String timestamp) { - if (timestamp.length() == 13){ - return new Timestamp(Long.parseLong(timestamp)); - }else if (timestamp.length() == 10){ - return new Timestamp(Long.parseLong(timestamp)*1000); - } else{ - return Timestamp.valueOf(timestamp); - } - } - @Override - public void close() { - } -} From d365b4a47e35be6a9c838e41b4da5e24a2cbc077 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Tue, 8 Oct 2019 14:20:43 +0800 Subject: [PATCH 412/470] =?UTF-8?q?=E5=88=A0=E9=99=A4timestampudf?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flink/sql/udf/TimestampUdf.java | 24 ------------------- 1 file changed, 24 deletions(-) delete mode 100644 core/src/main/java/com/dtstack/flink/sql/udf/TimestampUdf.java diff --git a/core/src/main/java/com/dtstack/flink/sql/udf/TimestampUdf.java b/core/src/main/java/com/dtstack/flink/sql/udf/TimestampUdf.java deleted file mode 100644 index 9f605dde3..000000000 --- a/core/src/main/java/com/dtstack/flink/sql/udf/TimestampUdf.java +++ /dev/null @@ -1,24 +0,0 @@ -package com.dtstack.flink.sql.udf; - -import org.apache.flink.table.functions.FunctionContext; -import org.apache.flink.table.functions.ScalarFunction; - -import java.sql.Timestamp; - -public class TimestampUdf extends ScalarFunction { - @Override - public void open(FunctionContext context) { - } - public static Timestamp eval(String timestamp) { - if (timestamp.length() == 13){ - return new Timestamp(Long.parseLong(timestamp)); - }else if (timestamp.length() == 10){ - return new Timestamp(Long.parseLong(timestamp)*1000); - } else{ - return Timestamp.valueOf(timestamp); - } - } - @Override - public void close() { - } -} From 912b1f36aba5dbe3aa35ea5c98ee32ad90786f64 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 9 Oct 2019 15:57:45 +0800 Subject: [PATCH 413/470] oracle schema --- .../dtstack/flink/sql/util/DtStringUtil.java | 13 +---------- .../flink/sql/sink/oracle/OracleSink.java | 22 +++++++++---------- .../sink/rdb/format/ExtendOutputFormat.java | 9 ++++++-- .../rdb/format/RetractJDBCOutputFormat.java | 14 +++++++++++- 4 files changed, 32 insertions(+), 26 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index 46a9c5c0e..f8ade7ec0 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -253,19 +253,8 @@ public static String firstUpperCase(String str) { return str.substring(0, 1).toUpperCase() + str.substring(1); } - public static String addQuoteForTableName(String table) { - String[] parts = table.split("\\."); - StringBuilder sb = new StringBuilder(); - for (int i = 0; i < parts.length; ++i) { - if (i != 0) { - sb.append("."); - } - sb.append(DtStringUtil.addQuoteForColumn(parts[i])); - } - return sb.toString(); - } - public static String addQuoteForColumn(String column) { + public static String addQuoteForStr(String column) { return getStartQuote() + column + getEndQuote(); } diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index fad3238df..68c6efabf 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -57,11 +57,11 @@ public void buildSql(String tableName, List fields) { private void buildInsertSql(String tableName, List fields) { - tableName = DtStringUtil.addQuoteForTableName(tableName); + tableName = DtStringUtil.addQuoteForStr(tableName); String sqlTmp = "insert into " + tableName + " (${fields}) values (${placeholder})"; List adaptFields = Lists.newArrayList(); - fields.forEach(field -> adaptFields.add(DtStringUtil.addQuoteForColumn(field))); + fields.forEach(field -> adaptFields.add(DtStringUtil.addQuoteForStr(field))); String fieldsStr = StringUtils.join(adaptFields, ","); String placeholder = ""; @@ -84,7 +84,7 @@ private void buildInsertSql(String tableName, List fields) { */ @Override public String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField) { - tableName = DtStringUtil.addQuoteForTableName(tableName); + tableName = DtStringUtil.addQuoteForStr(tableName); StringBuilder sb = new StringBuilder(); sb.append("MERGE INTO " + tableName + " T1 USING " @@ -112,10 +112,10 @@ public String quoteColumns(List column) { } public String quoteColumns(List column, String table) { - String prefix = StringUtils.isBlank(table) ? "" : DtStringUtil.addQuoteForTableName(table) + "."; + String prefix = StringUtils.isBlank(table) ? "" : DtStringUtil.addQuoteForStr(table) + "."; List list = new ArrayList<>(); for (String col : column) { - list.add(prefix + DtStringUtil.addQuoteForColumn(col)); + list.add(prefix + DtStringUtil.addQuoteForStr(col)); } return StringUtils.join(list, ","); } @@ -148,8 +148,8 @@ protected List keyColList(Map> realIndexes) { * @return */ public String getUpdateSql(List updateColumn, List fullColumn, String leftTable, String rightTable, List indexCols) { - String prefixLeft = StringUtils.isBlank(leftTable) ? "" : DtStringUtil.addQuoteForTableName(leftTable) + "."; - String prefixRight = StringUtils.isBlank(rightTable) ? "" : DtStringUtil.addQuoteForTableName(rightTable) + "."; + String prefixLeft = StringUtils.isBlank(leftTable) ? "" : DtStringUtil.addQuoteForStr(leftTable) + "."; + String prefixRight = StringUtils.isBlank(rightTable) ? "" : DtStringUtil.addQuoteForStr(rightTable) + "."; List list = new ArrayList<>(); for (String col : fullColumn) { // filter index column @@ -157,9 +157,9 @@ public String getUpdateSql(List updateColumn, List fullColumn, S continue; } if (containsIgnoreCase(updateColumn,col)) { - list.add(prefixLeft + DtStringUtil.addQuoteForColumn(col) + "=" + prefixRight + DtStringUtil.addQuoteForColumn(col)); + list.add(prefixLeft + DtStringUtil.addQuoteForStr(col) + "=" + prefixRight + DtStringUtil.addQuoteForStr(col)); } else { - list.add(prefixLeft + DtStringUtil.addQuoteForColumn(col) + "=null"); + list.add(prefixLeft + DtStringUtil.addQuoteForStr(col) + "=null"); } } return StringUtils.join(list, ","); @@ -176,7 +176,7 @@ public String updateKeySql(Map> updateKey) { for (Map.Entry> entry : updateKey.entrySet()) { List colList = new ArrayList<>(); for (String col : entry.getValue()) { - colList.add("T1." + DtStringUtil.addQuoteForColumn(col) + "=T2." + DtStringUtil.addQuoteForColumn(col)); + colList.add("T1." + DtStringUtil.addQuoteForStr(col) + "=T2." + DtStringUtil.addQuoteForStr(col)); } exprList.add(StringUtils.join(colList, " AND ")); } @@ -195,7 +195,7 @@ public String makeValues(List column) { if (i != 0) { sb.append(","); } - sb.append("? " + DtStringUtil.addQuoteForColumn(column.get(i))); + sb.append("? " + DtStringUtil.addQuoteForStr(column.get(i))); } sb.append(" FROM DUAL"); return sb.toString(); diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index d5bdc8cd9..fd6c4ace0 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -63,7 +63,8 @@ public boolean isReplaceInsertQuery() throws SQLException { */ public void fillRealIndexes() throws SQLException { Map> map = Maps.newHashMap(); - ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, null, DtStringUtil.addQuoteForTableName(getTableName()), true, false); + + ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, getSchema().toUpperCase(), DtStringUtil.addQuoteForStr(getTableName()), true, false); while (rs.next()) { String indexName = rs.getString("INDEX_NAME"); @@ -94,7 +95,7 @@ public void fillRealIndexes() throws SQLException { */ public void fillFullColumns() throws SQLException { // table name not quote - ResultSet rs = getDbConn().getMetaData().getColumns(null, null, getTableName(), null); + ResultSet rs = getDbConn().getMetaData().getColumns(null, getSchema().toUpperCase(), getTableName(), null); while (rs.next()) { String columnName = rs.getString("COLUMN_NAME"); if (StringUtils.isNotBlank(columnName)) { @@ -111,4 +112,8 @@ public boolean containsIgnoreCase(List l, String s) { } return false; } + + public boolean existTabname() throws SQLException { + return getDbConn().getMetaData().getTables(null, getSchema().toUpperCase(), getTableName(), null).next(); + } } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index cc5065913..7969ce10e 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -106,7 +106,7 @@ public void open(int taskNumber, int numTasks) throws IOException { dbConn = establishConnection(); initMetric(); - if (dbConn.getMetaData().getTables(null, null, tableName, null).next()) { + if (existTabname()) { if (isReplaceInsertQuery()) { insertQuery = dbSink.buildUpdateSql(tableName, Arrays.asList(dbSink.getFieldNames()), realIndexes, fullField); } @@ -400,6 +400,18 @@ public void setUsername(String username) { this.username = username; } + /** + * username as default schema + * @return + */ + public String getSchema() { + return username; + } + + public boolean existTabname() throws SQLException { + return dbConn.getMetaData().getTables(null, null, tableName, null).next(); + } + public void setPassword(String password) { this.password = password; } From 066c89e1492249476e8b9e3f94c362bff0113a69 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Thu, 10 Oct 2019 14:57:31 +0800 Subject: [PATCH 414/470] rdb schema --- .../dtstack/flink/sql/side/SideSqlExec.java | 1 + .../dtstack/flink/sql/util/DtStringUtil.java | 9 +++++++ .../flink/sql/sink/mysql/MysqlSink.java | 8 +++--- .../sql/side/oracle/OracleAllSideInfo.java | 7 ++--- .../sql/side/oracle/OracleAsyncSideInfo.java | 5 +++- .../flink/sql/sink/oracle/OracleSink.java | 14 +++++----- .../sql/side/rdb/table/RdbSideParser.java | 3 +++ .../sql/side/rdb/table/RdbSideTableInfo.java | 12 +++++++++ .../dtstack/flink/sql/sink/rdb/RdbSink.java | 10 ++++--- .../sink/rdb/format/ExtendOutputFormat.java | 7 ++--- .../rdb/format/RetractJDBCOutputFormat.java | 18 ++++++++----- .../sql/sink/rdb/table/RdbSinkParser.java | 2 ++ .../sql/sink/rdb/table/RdbTableInfo.java | 12 +++++++++ .../sql/sink/sqlserver/SqlserverSink.java | 26 ++++++++++++++----- 14 files changed, 98 insertions(+), 36 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index e83eda37a..adfc4da6a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -361,6 +361,7 @@ private SqlNode replaceNodeInfo(SqlNode groupNode, HashBasedTable fields) { + public void buildSql(String scheam, String tableName, List fields) { buildInsertSql(tableName, fields); } @Override - public String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField) { + public String buildUpdateSql(String schema, String tableName, List fieldNames, Map> realIndexes, List fullField) { return null; } private void buildInsertSql(String tableName, List fields) { - String sqlTmp = "replace into " + tableName + " (${fields}) values (${placeholder})"; + String sqlTmp = "replace into " + tableName + " (${fields}) values (${placeholder})"; String fieldsStr = ""; String placeholder = ""; diff --git a/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java index 801e0a69b..914a57aef 100644 --- a/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java +++ b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllSideInfo.java @@ -22,6 +22,7 @@ import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; @@ -40,15 +41,11 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { sqlCondition = "select ${selectField} from ${tableName} "; - sqlCondition = sqlCondition.replace("${tableName}", dealFiled(rdbSideTableInfo.getTableName())).replace("${selectField}", dealLowerSelectFiled(sideSelectFields)); + sqlCondition = sqlCondition.replace("${tableName}", DtStringUtil.getTableFullPath(rdbSideTableInfo.getSchema(), rdbSideTableInfo.getTableName())).replace("${selectField}", dealLowerSelectFiled(sideSelectFields)); System.out.println("---------side_exe_sql-----\n" + sqlCondition); } - private String dealFiled(String field) { - return "\"" + field + "\""; - } - private String dealLowerSelectFiled(String fieldsStr) { StringBuilder sb = new StringBuilder(); String[] fields = fieldsStr.split(","); diff --git a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java index 2c87199f4..891aebe61 100644 --- a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java +++ b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; @@ -68,10 +69,12 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { } } - sqlCondition = sqlCondition.replace("${tableName}", dealLowerFiled(rdbSideTableInfo.getTableName())).replace("${selectField}", dealLowerSelectFiled(sideSelectFields)); + sqlCondition = sqlCondition.replace("${tableName}", DtStringUtil.getTableFullPath(rdbSideTableInfo.getSchema(), rdbSideTableInfo.getTableName())).replace("${selectField}", dealLowerSelectFiled(sideSelectFields)); System.out.println("---------side_exe_sql-----\n" + sqlCondition); } + + private String dealLowerFiled(String field) { return "\"" + field + "\""; } diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index 68c6efabf..129286e63 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -51,13 +51,14 @@ public RetractJDBCOutputFormat getOutputFormat() { } @Override - public void buildSql(String tableName, List fields) { - buildInsertSql(tableName, fields); + public void buildSql(String scheam, String tableName, List fields) { + buildInsertSql(scheam, tableName, fields); } - private void buildInsertSql(String tableName, List fields) { + private void buildInsertSql(String scheam, String tableName, List fields) { + + tableName = DtStringUtil.getTableFullPath(scheam,tableName); - tableName = DtStringUtil.addQuoteForStr(tableName); String sqlTmp = "insert into " + tableName + " (${fields}) values (${placeholder})"; List adaptFields = Lists.newArrayList(); @@ -83,8 +84,9 @@ private void buildInsertSql(String tableName, List fields) { * @return */ @Override - public String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField) { - tableName = DtStringUtil.addQuoteForStr(tableName); + public String buildUpdateSql(String scheam, String tableName, List fieldNames, Map> realIndexes, List fullField) { + tableName = DtStringUtil.getTableFullPath(scheam, tableName); + StringBuilder sb = new StringBuilder(); sb.append("MERGE INTO " + tableName + " T1 USING " diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java index e007cecb0..7c559856f 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java @@ -47,6 +47,9 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map, Serializab private int parallelism = -1; + private String schema; + public RichSinkFunction createJdbcSinkFunc() { if (driverName == null || dbURL == null || userName == null || password == null || sqlTypes == null || tableName == null) { @@ -98,6 +100,7 @@ public RichSinkFunction createJdbcSinkFunc() { outputFormat.setTypesArray(sqlTypes); outputFormat.setTableName(tableName); outputFormat.setDbType(dbType); + outputFormat.setSchema(schema); outputFormat.setDbSink(this); outputFormat.verifyField(); @@ -142,8 +145,9 @@ public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { this.registerTabName = tmpRegisterName; this.primaryKeys = rdbTableInfo.getPrimaryKeys(); this.dbType = rdbTableInfo.getType(); + this.schema = rdbTableInfo.getSchema(); - buildSql(tableName, fields); + buildSql(schema, tableName, fields); buildSqlTypes(fieldTypeArray); return this; } @@ -254,7 +258,7 @@ public void setDbType(String dbType) { * @param tableName * @param fields */ - public abstract void buildSql(String tableName, List fields); + public abstract void buildSql(String schema, String tableName, List fields); /** * sqlserver and oracle maybe implement @@ -264,7 +268,7 @@ public void setDbType(String dbType) { * @param realIndexes * @return */ - public abstract String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField); + public abstract String buildUpdateSql(String schema, String tableName, List fieldNames, Map> realIndexes, List fullField); public abstract String getDriverName(); diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index fd6c4ace0..80f7f2018 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -64,7 +64,7 @@ public boolean isReplaceInsertQuery() throws SQLException { public void fillRealIndexes() throws SQLException { Map> map = Maps.newHashMap(); - ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, getSchema().toUpperCase(), DtStringUtil.addQuoteForStr(getTableName()), true, false); + ResultSet rs = getDbConn().getMetaData().getIndexInfo(null, getSchema(), DtStringUtil.addQuoteForStr(getTableName()), true, false); while (rs.next()) { String indexName = rs.getString("INDEX_NAME"); @@ -95,7 +95,7 @@ public void fillRealIndexes() throws SQLException { */ public void fillFullColumns() throws SQLException { // table name not quote - ResultSet rs = getDbConn().getMetaData().getColumns(null, getSchema().toUpperCase(), getTableName(), null); + ResultSet rs = getDbConn().getMetaData().getColumns(null, getSchema(), getTableName(), null); while (rs.next()) { String columnName = rs.getString("COLUMN_NAME"); if (StringUtils.isNotBlank(columnName)) { @@ -113,7 +113,4 @@ public boolean containsIgnoreCase(List l, String s) { return false; } - public boolean existTabname() throws SQLException { - return getDbConn().getMetaData().getTables(null, getSchema().toUpperCase(), getTableName(), null).next(); - } } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 7969ce10e..41bb37792 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -64,6 +64,7 @@ public class RetractJDBCOutputFormat extends MetricOutputFormat { private String dbURL; private String tableName; private String dbType; + private String schema; private RdbSink dbSink; // trigger preparedStatement execute batch interval private long batchWaitInterval = 10000l; @@ -108,7 +109,7 @@ public void open(int taskNumber, int numTasks) throws IOException { if (existTabname()) { if (isReplaceInsertQuery()) { - insertQuery = dbSink.buildUpdateSql(tableName, Arrays.asList(dbSink.getFieldNames()), realIndexes, fullField); + insertQuery = dbSink.buildUpdateSql(schema , tableName, Arrays.asList(dbSink.getFieldNames()), realIndexes, fullField); } upload = dbConn.prepareStatement(insertQuery); } else { @@ -400,16 +401,19 @@ public void setUsername(String username) { this.username = username; } - /** - * username as default schema - * @return - */ public String getSchema() { - return username; + if (StringUtils.isNotEmpty(schema)) { + return schema; + } + return null; + } + + public void setSchema(String schema) { + this.schema = schema; } public boolean existTabname() throws SQLException { - return dbConn.getMetaData().getTables(null, null, tableName, null).next(); + return dbConn.getMetaData().getTables(null, getSchema(), tableName, null).next(); } public void setPassword(String password) { diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java index f135dc2f3..beb51ffaa 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbSinkParser.java @@ -46,6 +46,8 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map fields) { + public void buildSql(String scheam, String tableName, List fields) { buildInsertSql(tableName, fields); } @@ -64,13 +65,26 @@ private void buildInsertSql(String tableName, List fields) { } @Override - public String buildUpdateSql(String tableName, List fieldNames, Map> realIndexes, List fullField) { - return "MERGE INTO " + tableName + " T1 USING " + public String buildUpdateSql(String scheam, String tableName, List fieldNames, Map> realIndexes, List fullField) { + StringBuilder sb = new StringBuilder(); + + sb.append("MERGE INTO " + tableName + " T1 USING " + "(" + makeValues(fieldNames) + ") T2 ON (" - + updateKeySql(realIndexes) + ") WHEN MATCHED THEN UPDATE SET " - + getUpdateSql(fieldNames, fullField, "T1", "T2", keyColList(realIndexes)) + " WHEN NOT MATCHED THEN " + + updateKeySql(realIndexes) + ") "); + + + String updateSql = getUpdateSql(fieldNames, fullField, "T1", "T2", keyColList(realIndexes)); + + if (StringUtils.isNotEmpty(updateSql)) { + sb.append(" WHEN MATCHED THEN UPDATE SET "); + sb.append(updateSql); + } + + sb.append(" WHEN NOT MATCHED THEN " + "INSERT (" + quoteColumns(fieldNames) + ") VALUES (" - + quoteColumns(fieldNames, "T2") + ");"; + + quoteColumns(fieldNames, "T2") + ")"); + + return sb.toString(); } From 9acc8d8e78f7fabf37ee9656c2afcf37f0b152e6 Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Fri, 11 Oct 2019 10:58:31 +0800 Subject: [PATCH 415/470] =?UTF-8?q?=E6=B5=81=E7=BB=B4join,=E6=BA=90?= =?UTF-8?q?=E8=A1=A8=E7=9A=84=E5=85=B3=E8=81=94=E5=AD=97=E6=AE=B5=E4=B8=8D?= =?UTF-8?q?=E8=BE=93=E5=85=A5,=E6=97=A0=E6=B3=95=E6=89=93=E5=85=A5?= =?UTF-8?q?=E6=95=B0=E6=8D=AE=E5=88=B0=E7=BB=93=E6=9E=9C=E8=A1=A8.?= =?UTF-8?q?=E5=85=B3=E7=B3=BB=E5=9E=8B=E6=95=B0=E6=8D=AE=E5=B7=B2=E7=BB=8F?= =?UTF-8?q?=E4=BF=AE=E6=94=B9?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java | 4 ++++ .../com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java index 70fe17b84..6c3f205a1 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java @@ -119,6 +119,10 @@ public void flatMap(Row value, Collector out) throws Exception { for (Integer conValIndex : sideInfo.getEqualValIndex()) { Object equalObj = value.getField(conValIndex); if (equalObj == null) { + if (sideInfo.getJoinType() == JoinType.LEFT) { + Row row = fillData(value, null); + out.collect(row); + } return; } inputParams.add(equalObj); diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index 156f136ef..323bed3ea 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -78,7 +78,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except for (Integer conValIndex : sideInfo.getEqualValIndex()) { Object equalObj = input.getField(conValIndex); if (equalObj == null) { - resultFuture.complete(null); + dealMissKey(input, resultFuture); return; } inputParams.add(equalObj); From 5b5e769a9d45f9a5d885562487aac132af8d7837 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 11 Oct 2019 19:47:29 +0800 Subject: [PATCH 416/470] add timestamp --- .../kafka/CustomerJsonDeserialization.java | 45 ++++++++++++++++--- 1 file changed, 40 insertions(+), 5 deletions(-) diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 3b73f0d51..f4003f80e 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -23,11 +23,14 @@ import com.dtstack.flink.sql.source.AbsDeserialization; import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import com.dtstack.flink.sql.table.TableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; @@ -42,8 +45,13 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.lang.reflect.Array; import java.lang.reflect.Field; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Set; @@ -64,7 +72,7 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private static final long serialVersionUID = 2385115520960444192L; - private static int rowLenth = 1000; + private static int dirtyDataFrequency = 1000; private final ObjectMapper objectMapper = new ObjectMapper(); @@ -115,7 +123,7 @@ public Row deserialize(byte[] message) throws IOException { try { JsonNode root = objectMapper.readTree(message); - if (numInRecord.getCount()%rowLenth == 0){ + if (numInRecord.getCount() % dirtyDataFrequency == 0) { LOG.info(root.toString()); } @@ -137,17 +145,19 @@ public Row deserialize(byte[] message) throws IOException { } } else { // Read the value as specified type - Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + + Object value = convert(node, fieldTypes[i]); row.setField(i, value); } } numInResolveRecord.inc(); return row; - } catch (Throwable t) { + } catch (Exception e) { //add metric of dirty data - if (dirtyDataCounter.getCount()%rowLenth == 0){ + if (dirtyDataCounter.getCount() % dirtyDataFrequency == 0) { LOG.info("dirtyData: " + new String(message)); + LOG.error("" , e); } dirtyDataCounter.inc(); return null; @@ -245,4 +255,29 @@ protected void registerPtMetric(AbstractFetcher fetcher) throws Exceptio private static String partitionLagMetricName(TopicPartition tp) { return tp + ".records-lag"; } + + private Object convert(JsonNode node, TypeInformation info) { + if (info.getTypeClass().equals(Types.BOOLEAN.getTypeClass())) { + return node.asBoolean(); + } else if (info.getTypeClass().equals(Types.STRING.getTypeClass())) { + return node.asText(); + } else if (info.getTypeClass().equals(Types.SQL_DATE.getTypeClass())) { + return Date.valueOf(node.asText()); + } else if (info.getTypeClass().equals(Types.SQL_TIME.getTypeClass())) { + // local zone + return Time.valueOf(node.asText()); + } else if (info.getTypeClass().equals(Types.SQL_TIMESTAMP.getTypeClass())) { + // local zone + return Timestamp.valueOf(node.asText()); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return objectMapper.treeToValue(node, info.getTypeClass()); + } catch (JsonProcessingException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for node: " + node); + } + } + } + } From 54d7370179225e2651063c6462e27ec69adee7c7 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 11 Oct 2019 20:28:39 +0800 Subject: [PATCH 417/470] kafka parse timestamp --- .../kafka/CustomerJsonDeserialization.java | 45 ++++++++++++++++--- .../kafka/CustomerJsonDeserialization.java | 42 ++++++++++++++--- .../kafka/CustomerJsonDeserialization.java | 2 +- .../rdb/format/RetractJDBCOutputFormat.java | 2 +- 4 files changed, 79 insertions(+), 12 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 28f0e470b..2a526a4b5 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -23,11 +23,14 @@ import com.dtstack.flink.sql.source.AbsDeserialization; import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import com.dtstack.flink.sql.table.TableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; @@ -43,7 +46,11 @@ import java.io.IOException; import java.lang.reflect.Field; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Set; @@ -62,7 +69,7 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private static final long serialVersionUID = 2385115520960444192L; - private static int rowLenth = 1000; + private static int dirtyDataFrequency = 1000; private final ObjectMapper objectMapper = new ObjectMapper(); @@ -112,7 +119,7 @@ public Row deserialize(byte[] message) throws IOException { try { JsonNode root = objectMapper.readTree(message); - if (numInRecord.getCount()%rowLenth == 0){ + if (numInRecord.getCount() % dirtyDataFrequency == 0) { LOG.info(root.toString()); } @@ -134,17 +141,18 @@ public Row deserialize(byte[] message) throws IOException { } } else { // Read the value as specified type - Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + Object value = convert(node, fieldTypes[i]); row.setField(i, value); } } numInResolveRecord.inc(); return row; - } catch (Throwable t) { + } catch (Exception e) { //add metric of dirty data - if (dirtyDataCounter.getCount()%rowLenth == 0){ + if (dirtyDataCounter.getCount() % dirtyDataFrequency == 0 || LOG.isDebugEnabled()) { LOG.info("dirtyData: " + new String(message)); + LOG.info(" " ,e); } dirtyDataCounter.inc(); return null; @@ -243,4 +251,31 @@ protected void registerPtMetric(AbstractFetcher fetcher) throws Exceptio private static String partitionLagMetricName(TopicPartition tp) { return tp + ".records-lag"; } + + private Object convert(JsonNode node, TypeInformation info) { + if (info.getTypeClass().equals(Types.BOOLEAN.getTypeClass())) { + return node.asBoolean(); + } else if (info.getTypeClass().equals(Types.STRING.getTypeClass())) { + return node.asText(); + } else if (info.getTypeClass().equals(Types.SQL_DATE.getTypeClass())) { + return Date.valueOf(node.asText()); + } else if (info.getTypeClass().equals(Types.SQL_TIME.getTypeClass())) { + // local zone + return Time.valueOf(node.asText()); + } else if (info.getTypeClass().equals(Types.SQL_TIMESTAMP.getTypeClass())) { + // local zone + return Timestamp.valueOf(node.asText()); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return objectMapper.treeToValue(node, info.getTypeClass()); + } catch (JsonProcessingException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for node: " + node); + } + } + } + + + } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 66d41e0ba..d22ab1e49 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -23,11 +23,14 @@ import com.dtstack.flink.sql.source.AbsDeserialization; import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import com.dtstack.flink.sql.table.TableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; @@ -43,7 +46,11 @@ import java.io.IOException; import java.lang.reflect.Field; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Set; @@ -62,7 +69,7 @@ public class CustomerJsonDeserialization extends AbsDeserialization { private static final long serialVersionUID = 2385115520960444192L; - private static int rowLenth = 1000; + private static int dirtyDataFrequency = 1000; private final ObjectMapper objectMapper = new ObjectMapper(); @@ -112,7 +119,7 @@ public Row deserialize(byte[] message) throws IOException { try { JsonNode root = objectMapper.readTree(message); - if (numInRecord.getCount()%rowLenth == 0){ + if (numInRecord.getCount() % dirtyDataFrequency == 0) { LOG.info(root.toString()); } @@ -134,17 +141,18 @@ public Row deserialize(byte[] message) throws IOException { } } else { // Read the value as specified type - Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + Object value = convert(node, fieldTypes[i]); row.setField(i, value); } } numInResolveRecord.inc(); return row; - } catch (Throwable t) { + } catch (Exception e) { //add metric of dirty data - if (dirtyDataCounter.getCount()%rowLenth == 0){ + if (dirtyDataCounter.getCount() % dirtyDataFrequency == 0 || LOG.isDebugEnabled()) { LOG.info("dirtyData: " + new String(message)); + LOG.error(" ", e); } dirtyDataCounter.inc(); return null; @@ -244,4 +252,28 @@ protected void registerPtMetric(AbstractFetcher fetcher) throws Exceptio private static String partitionLagMetricName(TopicPartition tp) { return tp + ".records-lag"; } + + private Object convert(JsonNode node, TypeInformation info) { + if (info.getTypeClass().equals(Types.BOOLEAN.getTypeClass())) { + return node.asBoolean(); + } else if (info.getTypeClass().equals(Types.STRING.getTypeClass())) { + return node.asText(); + } else if (info.getTypeClass().equals(Types.SQL_DATE.getTypeClass())) { + return Date.valueOf(node.asText()); + } else if (info.getTypeClass().equals(Types.SQL_TIME.getTypeClass())) { + // local zone + return Time.valueOf(node.asText()); + } else if (info.getTypeClass().equals(Types.SQL_TIMESTAMP.getTypeClass())) { + // local zone + return Timestamp.valueOf(node.asText()); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return objectMapper.treeToValue(node, info.getTypeClass()); + } catch (JsonProcessingException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for node: " + node); + } + } + } } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index f4003f80e..a74173409 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -155,7 +155,7 @@ public Row deserialize(byte[] message) throws IOException { return row; } catch (Exception e) { //add metric of dirty data - if (dirtyDataCounter.getCount() % dirtyDataFrequency == 0) { + if (dirtyDataCounter.getCount() % dirtyDataFrequency == 0 || LOG.isDebugEnabled()) { LOG.info("dirtyData: " + new String(message)); LOG.error("" , e); } diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index a2a173a47..44024d886 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -204,7 +204,7 @@ private void writeSingleRecord(Row row) { dbConn.commit(); } catch (SQLException e) { outDirtyRecords.inc(); - if (outDirtyRecords.getCount() % dirtyDataPrintFrequency == 0) { + if (outDirtyRecords.getCount() % dirtyDataPrintFrequency == 0 || LOG.isDebugEnabled()) { LOG.error("record insert failed ..", row.toString()); LOG.error("", e); } From 3ebcf1cdd868ec82a65406a9cfe1ef0015fa0c32 Mon Sep 17 00:00:00 2001 From: toutian <953372946@qq.com> Date: Mon, 14 Oct 2019 20:32:44 +0800 Subject: [PATCH 418/470] flinksql 150 classloader --- .../ClassLoaderCallBackMethod.java | 40 +++++++++++++ .../sql/classloader/ClassLoaderManager.java | 59 +++++++++++++++++++ .../flink/sql/classloader/DtSupplier.java | 44 ++++++++++++++ .../flink/sql/side/StreamSideFactory.java | 21 +++---- .../sql/side/operator/SideAsyncOperator.java | 11 ++-- .../operator/SideWithAllCacheOperator.java | 14 ++--- .../flink/sql/sink/StreamSinkFactory.java | 53 ++++++----------- .../flink/sql/source/StreamSourceFactory.java | 42 +++++-------- .../dtstack/flink/sql/util/PluginUtil.java | 21 +++++++ 9 files changed, 215 insertions(+), 90 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderCallBackMethod.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderManager.java create mode 100644 core/src/main/java/com/dtstack/flink/sql/classloader/DtSupplier.java diff --git a/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderCallBackMethod.java b/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderCallBackMethod.java new file mode 100644 index 000000000..86dd93372 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderCallBackMethod.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 com.dtstack.flink.sql.classloader; + +/** + * company: www.dtstack.com + * author: toutian + * create: 2019/10/14 + */ +public class ClassLoaderCallBackMethod { + + public static R callbackAndReset(DtSupplier supplier, ClassLoader toSetClassLoader) throws Exception { + ClassLoader oldClassLoader = Thread.currentThread().getContextClassLoader(); + Thread.currentThread().setContextClassLoader(toSetClassLoader); + try { + return supplier.get(toSetClassLoader); + } finally { + Thread.currentThread().setContextClassLoader(oldClassLoader); + } + } + + +} diff --git a/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderManager.java b/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderManager.java new file mode 100644 index 000000000..87389feb4 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderManager.java @@ -0,0 +1,59 @@ +/* + * 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 com.dtstack.flink.sql.classloader; + +import com.dtstack.flink.sql.util.PluginUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.URL; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * company: www.dtstack.com + * author: toutian + * create: 2019/10/14 + */ +public class ClassLoaderManager { + + private static final Logger LOG = LoggerFactory.getLogger(ClassLoaderManager.class); + + private static Map pluginClassLoader = new ConcurrentHashMap<>(); + + public static R newInstance(String pluginJarPath, DtSupplier supplier) throws Exception { + ClassLoader classLoader = retrieveClassLoad(pluginJarPath); + return ClassLoaderCallBackMethod.callbackAndReset(supplier, classLoader); + } + + private static DtClassLoader retrieveClassLoad(String pluginJarPath) { + return pluginClassLoader.computeIfAbsent(pluginJarPath, k -> { + try { + URL[] urls = PluginUtil.getPluginJarUrls(pluginJarPath); + ClassLoader parentClassLoader = Thread.currentThread().getContextClassLoader(); + DtClassLoader classLoader = new DtClassLoader(urls, parentClassLoader); + LOG.info("pluginJarPath:{} create ClassLoad successful...", pluginJarPath); + return classLoader; + } catch (Throwable e) { + LOG.error("retrieve ClassLoad happens error:{}", e); + throw new RuntimeException("retrieve ClassLoad happens error"); + } + }); + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/classloader/DtSupplier.java b/core/src/main/java/com/dtstack/flink/sql/classloader/DtSupplier.java new file mode 100644 index 000000000..779426501 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/classloader/DtSupplier.java @@ -0,0 +1,44 @@ +/* + * 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 com.dtstack.flink.sql.classloader; + +/** + * Represents a supplier of results. + * + *

There is no requirement that a new or distinct result be returned each + * time the supplier is invoked. + * + *

This is a functional interface + * whose functional method is {@link #get()}. + * + * @param the type of results supplied by this supplier + * + * @since 1.8 + */ +@FunctionalInterface +public interface DtSupplier { + + /** + * Gets a result. + * + * @return a result + */ + T get(ClassLoader cl) throws Exception; +} diff --git a/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java b/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java index 4eb78b4c4..185911c7e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/StreamSideFactory.java @@ -16,11 +16,10 @@ * limitations under the License. */ - package com.dtstack.flink.sql.side; -import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.classloader.ClassLoaderManager; import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.table.AbsSideTableParser; import com.dtstack.flink.sql.table.AbsTableParser; @@ -30,6 +29,7 @@ * get specify side parser * Date: 2018/7/25 * Company: www.dtstack.com + * * @author xuchao */ @@ -40,18 +40,15 @@ public class StreamSideFactory { public static AbsTableParser getSqlParser(String pluginType, String sqlRootDir, String cacheType) throws Exception { String sideOperator = ECacheType.ALL.name().equals(cacheType) ? "all" : "async"; - ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); String pluginJarPath = PluginUtil.getSideJarFileDirPath(pluginType, sideOperator, "side", sqlRootDir); - - DtClassLoader dtClassLoader = (DtClassLoader) classLoader; - PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); String className = PluginUtil.getSqlParserClassName(pluginType, CURR_TYPE); - Class sideParser = dtClassLoader.loadClass(className); - if(!AbsSideTableParser.class.isAssignableFrom(sideParser)){ - throw new RuntimeException("class " + sideParser.getName() + " not subClass of AbsSideTableParser"); - } - - return sideParser.asSubclass(AbsTableParser.class).newInstance(); + return ClassLoaderManager.newInstance(pluginJarPath, (cl) -> { + Class sideParser = cl.loadClass(className); + if (!AbsSideTableParser.class.isAssignableFrom(sideParser)) { + throw new RuntimeException("class " + sideParser.getName() + " not subClass of AbsSideTableParser"); + } + return sideParser.asSubclass(AbsTableParser.class).newInstance(); + }); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java index df9a1c175..ea65e5bfd 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideAsyncOperator.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.side.operator; -import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.classloader.ClassLoaderManager; import com.dtstack.flink.sql.side.AsyncReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; @@ -49,14 +49,13 @@ public class SideAsyncOperator { private static AsyncReqRow loadAsyncReq(String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { - ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); String pathOfType = String.format(PATH_FORMAT, sideType); String pluginJarPath = PluginUtil.getJarFileDirPath(pathOfType, sqlRootDir); - DtClassLoader dtClassLoader = (DtClassLoader) classLoader; - PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); String className = PluginUtil.getSqlSideClassName(sideType, "side", "Async"); - return dtClassLoader.loadClass(className).asSubclass(AsyncReqRow.class) - .getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, SideTableInfo.class).newInstance(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + return ClassLoaderManager.newInstance(pluginJarPath, (cl) -> + cl.loadClass(className).asSubclass(AsyncReqRow.class) + .getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, SideTableInfo.class) + .newInstance(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } public static DataStream getSideJoinDataStream(DataStream inputStream, String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, diff --git a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java index 72a67d00b..2f382eefd 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/operator/SideWithAllCacheOperator.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.side.operator; -import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.classloader.ClassLoaderManager; import com.dtstack.flink.sql.side.AllReqRow; import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.JoinInfo; @@ -28,7 +28,6 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; -import java.net.MalformedURLException; import java.util.List; /** @@ -47,18 +46,13 @@ private static AllReqRow loadFlatMap(String sideType, String sqlRootDir, RowType JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) throws Exception { - ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); String pathOfType = String.format(PATH_FORMAT, sideType); String pluginJarPath = PluginUtil.getJarFileDirPath(pathOfType, sqlRootDir); - - DtClassLoader dtClassLoader = (DtClassLoader) classLoader; - PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); String className = PluginUtil.getSqlSideClassName(sideType, "side", "All"); - return dtClassLoader.loadClass(className).asSubclass(AllReqRow.class).getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, SideTableInfo.class) - .newInstance(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); - - + return ClassLoaderManager.newInstance(pluginJarPath, (cl) -> cl.loadClass(className).asSubclass(AllReqRow.class) + .getConstructor(RowTypeInfo.class, JoinInfo.class, List.class, SideTableInfo.class) + .newInstance(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); } public static DataStream getSideJoinDataStream(DataStream inputStream, String sideType, String sqlRootDir, RowTypeInfo rowTypeInfo, JoinInfo joinInfo, diff --git a/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java b/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java index ebda80c8a..53460081d 100644 --- a/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/sink/StreamSinkFactory.java @@ -20,12 +20,11 @@ package com.dtstack.flink.sql.sink; -import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.classloader.ClassLoaderManager; import com.dtstack.flink.sql.table.AbsTableParser; import com.dtstack.flink.sql.table.TargetTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.table.sinks.TableSink; /** @@ -42,51 +41,33 @@ public class StreamSinkFactory { private static final String DIR_NAME_FORMAT = "%ssink"; public static AbsTableParser getSqlParser(String pluginType, String sqlRootDir) throws Exception { - ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - - if(!(classLoader instanceof DtClassLoader)){ - throw new RuntimeException("it's not a correct classLoader instance, it's type must be DtClassLoader!"); - } - - DtClassLoader dtClassLoader = (DtClassLoader) classLoader; - String pluginJarPath = PluginUtil.getJarFileDirPath(String.format(DIR_NAME_FORMAT, pluginType), sqlRootDir); - PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); String typeNoVersion = DtStringUtil.getPluginTypeWithoutVersion(pluginType); String className = PluginUtil.getSqlParserClassName(typeNoVersion, CURR_TYPE); - Class targetParser = dtClassLoader.loadClass(className); - if(!AbsTableParser.class.isAssignableFrom(targetParser)){ - throw new RuntimeException("class " + targetParser.getName() + " not subClass of AbsTableParser"); - } - - return targetParser.asSubclass(AbsTableParser.class).newInstance(); + return ClassLoaderManager.newInstance(pluginJarPath, (cl) -> { + Class targetParser = cl.loadClass(className); + if(!AbsTableParser.class.isAssignableFrom(targetParser)){ + throw new RuntimeException("class " + targetParser.getName() + " not subClass of AbsTableParser"); + } + return targetParser.asSubclass(AbsTableParser.class).newInstance(); + }); } public static TableSink getTableSink(TargetTableInfo targetTableInfo, String localSqlRootDir) throws Exception { - - ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - if(!(classLoader instanceof DtClassLoader)){ - throw new RuntimeException("it's not a correct classLoader instance, it's type must be DtClassLoader!"); - } - - DtClassLoader dtClassLoader = (DtClassLoader) classLoader; - String pluginType = targetTableInfo.getType(); String pluginJarDirPath = PluginUtil.getJarFileDirPath(String.format(DIR_NAME_FORMAT, pluginType), localSqlRootDir); - - PluginUtil.addPluginJar(pluginJarDirPath, dtClassLoader); - String typeNoVersion = DtStringUtil.getPluginTypeWithoutVersion(pluginType); String className = PluginUtil.getGenerClassName(typeNoVersion, CURR_TYPE); - Class sinkClass = dtClassLoader.loadClass(className); - - if(!IStreamSinkGener.class.isAssignableFrom(sinkClass)){ - throw new RuntimeException("class " + sinkClass + " not subClass of IStreamSinkGener"); - } - IStreamSinkGener streamSinkGener = sinkClass.asSubclass(IStreamSinkGener.class).newInstance(); - Object result = streamSinkGener.genStreamSink(targetTableInfo); - return (TableSink) result; + return ClassLoaderManager.newInstance(pluginJarDirPath, (cl) -> { + Class sinkClass = cl.loadClass(className); + if(!IStreamSinkGener.class.isAssignableFrom(sinkClass)){ + throw new RuntimeException("class " + sinkClass + " not subClass of IStreamSinkGener"); + } + IStreamSinkGener streamSinkGener = sinkClass.asSubclass(IStreamSinkGener.class).newInstance(); + Object result = streamSinkGener.genStreamSink(targetTableInfo); + return (TableSink) result; + }); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java b/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java index b8dfe66e2..1057fb0ed 100644 --- a/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java +++ b/core/src/main/java/com/dtstack/flink/sql/source/StreamSourceFactory.java @@ -21,12 +21,11 @@ package com.dtstack.flink.sql.source; -import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.classloader.ClassLoaderManager; import com.dtstack.flink.sql.table.AbsSourceParser; import com.dtstack.flink.sql.table.SourceTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.PluginUtil; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; @@ -46,21 +45,16 @@ public class StreamSourceFactory { public static AbsSourceParser getSqlParser(String pluginType, String sqlRootDir) throws Exception { - ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - String pluginJarPath = PluginUtil.getJarFileDirPath(String.format(DIR_NAME_FORMAT, pluginType), sqlRootDir); - - DtClassLoader dtClassLoader = (DtClassLoader) classLoader; - PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); - String typeNoVersion = DtStringUtil.getPluginTypeWithoutVersion(pluginType); String className = PluginUtil.getSqlParserClassName(typeNoVersion, CURR_TYPE); - Class sourceParser = dtClassLoader.loadClass(className); - if(!AbsSourceParser.class.isAssignableFrom(sourceParser)){ - throw new RuntimeException("class " + sourceParser.getName() + " not subClass of AbsSourceParser"); - } - - return sourceParser.asSubclass(AbsSourceParser.class).newInstance(); + return ClassLoaderManager.newInstance(pluginJarPath, (cl) -> { + Class sourceParser = cl.loadClass(className); + if(!AbsSourceParser.class.isAssignableFrom(sourceParser)){ + throw new RuntimeException("class " + sourceParser.getName() + " not subClass of AbsSourceParser"); + } + return sourceParser.asSubclass(AbsSourceParser.class).newInstance(); + }); } /** @@ -73,21 +67,17 @@ public static Table getStreamSource(SourceTableInfo sourceTableInfo, StreamExecu String sourceTypeStr = sourceTableInfo.getType(); String typeNoVersion = DtStringUtil.getPluginTypeWithoutVersion(sourceTypeStr); - ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - String pluginJarPath = PluginUtil.getJarFileDirPath(String.format(DIR_NAME_FORMAT, sourceTypeStr), sqlRootDir); String className = PluginUtil.getGenerClassName(typeNoVersion, CURR_TYPE); - DtClassLoader dtClassLoader = (DtClassLoader) classLoader; - PluginUtil.addPluginJar(pluginJarPath, dtClassLoader); - Class sourceClass = dtClassLoader.loadClass(className); - - if(!IStreamSourceGener.class.isAssignableFrom(sourceClass)){ - throw new RuntimeException("class " + sourceClass.getName() + " not subClass of IStreamSourceGener"); - } + return ClassLoaderManager.newInstance(pluginJarPath, (cl) -> { + Class sourceClass = cl.loadClass(className); + if(!IStreamSourceGener.class.isAssignableFrom(sourceClass)){ + throw new RuntimeException("class " + sourceClass.getName() + " not subClass of IStreamSourceGener"); + } - IStreamSourceGener sourceGener = sourceClass.asSubclass(IStreamSourceGener.class).newInstance(); - Object object = sourceGener.genStreamSource(sourceTableInfo, env, tableEnv); - return (Table) object; + IStreamSourceGener sourceGener = sourceClass.asSubclass(IStreamSourceGener.class).newInstance(); + return (Table) sourceGener.genStreamSource(sourceTableInfo, env, tableEnv); + }); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java index 682df169e..8cf4c8b1b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java @@ -33,6 +33,8 @@ import java.io.IOException; import java.net.MalformedURLException; import java.net.URL; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.Properties; @@ -144,6 +146,25 @@ public static void addPluginJar(String pluginDir, DtClassLoader classLoader) thr } } + public static URL[] getPluginJarUrls(String pluginDir) throws MalformedURLException { + List urlList = new ArrayList<>(); + File dirFile = new File(pluginDir); + if(!dirFile.exists() || !dirFile.isDirectory()){ + throw new RuntimeException("plugin path:" + pluginDir + "is not exist."); + } + + File[] files = dirFile.listFiles(tmpFile -> tmpFile.isFile() && tmpFile.getName().endsWith(JAR_SUFFIX)); + if(files == null || files.length == 0){ + throw new RuntimeException("plugin path:" + pluginDir + " is null."); + } + + for(File file : files){ + URL pluginJarURL = file.toURI().toURL(); + urlList.add(pluginJarURL); + } + return urlList.toArray(new URL[urlList.size()]); + } + public static String getCoreJarFileName (String path, String prefix) throws Exception { String coreJarFileName = null; File pluginDir = new File(path); From db36b92ed00888dcdaff46914f9d0e4da69914d5 Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Tue, 15 Oct 2019 10:27:14 +0800 Subject: [PATCH 419/470] =?UTF-8?q?[flinksql][=E7=BB=B4=E8=A1=A8join=20npe?= =?UTF-8?q?=EF=BC=8C=E9=94=99=E8=AF=AF=E6=8F=90=E7=A4=BA=E4=B8=8D=E5=8F=8B?= =?UTF-8?q?=E5=A5=BD][19240]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../java/com/dtstack/flink/sql/side/SideSqlExec.java | 9 +++++++++ v1.8.0_dev_bugfix_joinnpe | 0 2 files changed, 9 insertions(+) create mode 100644 v1.8.0_dev_bugfix_joinnpe diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index ab3c47a8a..2f3e0b54b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -356,7 +356,16 @@ private SqlNode replaceOrderByTableName(SqlNode orderNode, String tableAlias) { private SqlNode replaceNodeInfo(SqlNode groupNode, HashBasedTable mappingTable, String tableAlias){ if(groupNode.getKind() == IDENTIFIER){ SqlIdentifier sqlIdentifier = (SqlIdentifier) groupNode; + // 如果没有表别名前缀,直接返回字段名称 + if (sqlIdentifier.names.size() == 1) { + return groupNode; + } String mappingFieldName = mappingTable.get(sqlIdentifier.getComponent(0).getSimple(), sqlIdentifier.getComponent(1).getSimple()); + // 如果有表别名前缀,但是在宽表中找不到映射,只需要设置别名,不需要替换映射 + if (null == mappingFieldName){ + // return sqlIdentifier.setName(0, tableAlias); + throw new RuntimeException("Column '" + sqlIdentifier.getComponent(1).getSimple() + "' not found in table '" + sqlIdentifier.getComponent(0).getSimple() + "'"); + } sqlIdentifier = sqlIdentifier.setName(0, tableAlias); return sqlIdentifier.setName(1, mappingFieldName); }else if(groupNode instanceof SqlBasicCall){ diff --git a/v1.8.0_dev_bugfix_joinnpe b/v1.8.0_dev_bugfix_joinnpe new file mode 100644 index 000000000..e69de29bb From 5a6f3edf5bff04e8529450de6d43909be1990145 Mon Sep 17 00:00:00 2001 From: toutian <953372946@qq.com> Date: Tue, 15 Oct 2019 16:54:38 +0800 Subject: [PATCH 420/470] udf classloader --- .../main/java/com/dtstack/flink/sql/Main.java | 26 +++----------- .../sql/classloader/ClassLoaderManager.java | 36 +++++++++++++++++++ .../com/dtstack/flink/sql/util/FlinkUtil.java | 21 +++++------ 3 files changed, 49 insertions(+), 34 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index d55f0ceea..088f7026f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql; -import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.classloader.ClassLoaderManager; import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.exec.FlinkSQLExec; import com.dtstack.flink.sql.parser.CreateFuncParser; @@ -77,10 +77,7 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.net.URL; -import java.net.URLClassLoader; import java.net.URLDecoder; -import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Properties; @@ -144,10 +141,6 @@ public static void main(String[] args) throws Exception { addJarFileList = objMapper.readValue(addJarListStr, List.class); } - ClassLoader threadClassLoader = Thread.currentThread().getContextClassLoader(); - DtClassLoader parentClassloader = new DtClassLoader(new URL[]{}, threadClassLoader); - Thread.currentThread().setContextClassLoader(parentClassloader); - confProp = URLDecoder.decode(confProp, Charsets.UTF_8.toString()); Properties confProperties = PluginUtil.jsonStrToObject(confProp, Properties.class); StreamExecutionEnvironment env = getStreamExeEnv(confProperties, deployMode); @@ -166,7 +159,7 @@ public static void main(String[] args) throws Exception { Map registerTableCache = Maps.newHashMap(); //register udf - registerUDF(sqlTree, jarURList, parentClassloader, tableEnv); + registerUDF(sqlTree, jarURList, tableEnv); //register table schema registerTable(sqlTree, env, tableEnv, localSqlPluginPath, remoteSqlPluginPath, sideTableMap, registerTableCache); @@ -219,9 +212,7 @@ public static void main(String[] args) throws Exception { } if(env instanceof MyLocalStreamEnvironment) { - List urlList = new ArrayList<>(); - urlList.addAll(Arrays.asList(parentClassloader.getURLs())); - ((MyLocalStreamEnvironment) env).setClasspaths(urlList); + ((MyLocalStreamEnvironment) env).setClasspaths(ClassLoaderManager.getClassPath()); } env.execute(name); @@ -245,19 +236,12 @@ private static void addEnvClassPath(StreamExecutionEnvironment env, Set cla } } - private static void registerUDF(SqlTree sqlTree, List jarURList, URLClassLoader parentClassloader, - StreamTableEnvironment tableEnv) + private static void registerUDF(SqlTree sqlTree, List jarURList, StreamTableEnvironment tableEnv) throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException { //register urf - URLClassLoader classLoader = null; List funcList = sqlTree.getFunctionList(); for (CreateFuncParser.SqlParserResult funcInfo : funcList) { - //classloader - if (classLoader == null) { - classLoader = FlinkUtil.loadExtraJar(jarURList, parentClassloader); - } - FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName(), - tableEnv, classLoader); + FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName(), tableEnv, jarURList); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderManager.java b/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderManager.java index 87389feb4..45663ff07 100644 --- a/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderManager.java +++ b/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderManager.java @@ -19,10 +19,16 @@ package com.dtstack.flink.sql.classloader; import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.net.URL; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -42,6 +48,11 @@ public static R newInstance(String pluginJarPath, DtSupplier supplier) th return ClassLoaderCallBackMethod.callbackAndReset(supplier, classLoader); } + public static R newInstance(List jarUrls, DtSupplier supplier) throws Exception { + ClassLoader classLoader = retrieveClassLoad(jarUrls); + return ClassLoaderCallBackMethod.callbackAndReset(supplier, classLoader); + } + private static DtClassLoader retrieveClassLoad(String pluginJarPath) { return pluginClassLoader.computeIfAbsent(pluginJarPath, k -> { try { @@ -56,4 +67,29 @@ private static DtClassLoader retrieveClassLoad(String pluginJarPath) { } }); } + + private static DtClassLoader retrieveClassLoad(List jarUrls) { + jarUrls.sort(Comparator.comparing(URL::toString)); + String jarUrlkey = StringUtils.join(jarUrls, "_"); + return pluginClassLoader.computeIfAbsent(jarUrlkey, k -> { + try { + URL[] urls = jarUrls.toArray(new URL[jarUrls.size()]); + ClassLoader parentClassLoader = Thread.currentThread().getContextClassLoader(); + DtClassLoader classLoader = new DtClassLoader(urls, parentClassLoader); + LOG.info("jarUrl:{} create ClassLoad successful...", jarUrlkey); + return classLoader; + } catch (Throwable e) { + LOG.error("retrieve ClassLoad happens error:{}", e); + throw new RuntimeException("retrieve ClassLoad happens error"); + } + }); + } + + public static List getClassPath() { + List classPaths = new ArrayList<>(); + for (Map.Entry entry : pluginClassLoader.entrySet()) { + classPaths.addAll(Arrays.asList(entry.getValue().getURLs())); + } + return classPaths; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java index 89476380e..a878b0d72 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql.util; +import com.dtstack.flink.sql.classloader.ClassLoaderManager; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.runtime.state.filesystem.FsStateBackend; @@ -151,12 +152,11 @@ public static void setStreamTimeCharacteristic(StreamExecutionEnvironment env, P * TABLE|SCALA * 注册UDF到table env */ - public static void registerUDF(String type, String classPath, String funcName, TableEnvironment tableEnv, - ClassLoader classLoader){ + public static void registerUDF(String type, String classPath, String funcName, TableEnvironment tableEnv, List jarURList){ if("SCALA".equalsIgnoreCase(type)){ - registerScalaUDF(classPath, funcName, tableEnv, classLoader); + registerScalaUDF(classPath, funcName, tableEnv, jarURList); }else if("TABLE".equalsIgnoreCase(type)){ - registerTableUDF(classPath, funcName, tableEnv, classLoader); + registerTableUDF(classPath, funcName, tableEnv, jarURList); }else{ throw new RuntimeException("not support of UDF which is not in (TABLE, SCALA)"); } @@ -169,11 +169,9 @@ public static void registerUDF(String type, String classPath, String funcName, T * @param funcName * @param tableEnv */ - public static void registerScalaUDF(String classPath, String funcName, TableEnvironment tableEnv, - ClassLoader classLoader){ + public static void registerScalaUDF(String classPath, String funcName, TableEnvironment tableEnv, List jarURList){ try{ - ScalarFunction udfFunc = Class.forName(classPath, false, classLoader) - .asSubclass(ScalarFunction.class).newInstance(); + ScalarFunction udfFunc = ClassLoaderManager.newInstance(jarURList, (cl) -> cl.loadClass(classPath).asSubclass(ScalarFunction.class).newInstance()); tableEnv.registerFunction(funcName, udfFunc); logger.info("register scala function:{} success.", funcName); }catch (Exception e){ @@ -189,12 +187,9 @@ public static void registerScalaUDF(String classPath, String funcName, TableEnvi * @param funcName * @param tableEnv */ - public static void registerTableUDF(String classPath, String funcName, TableEnvironment tableEnv, - ClassLoader classLoader){ + public static void registerTableUDF(String classPath, String funcName, TableEnvironment tableEnv, List jarURList){ try { - TableFunction udfFunc = Class.forName(classPath, false, classLoader) - .asSubclass(TableFunction.class).newInstance(); - + TableFunction udfFunc = ClassLoaderManager.newInstance(jarURList, (cl) -> cl.loadClass(classPath).asSubclass(TableFunction.class).newInstance()); if(tableEnv instanceof StreamTableEnvironment){ ((StreamTableEnvironment)tableEnv).registerFunction(funcName, udfFunc); }else if(tableEnv instanceof BatchTableEnvironment){ From 3b042e18002e34fc108c027f2a21a7394df0ec7a Mon Sep 17 00:00:00 2001 From: toutian <953372946@qq.com> Date: Tue, 15 Oct 2019 17:58:47 +0800 Subject: [PATCH 421/470] rename --- .../flink/sql/classloader/ClassLoaderManager.java | 9 ++++----- .../{DtSupplier.java => ClassLoaderSupplier.java} | 2 +- ...lBackMethod.java => ClassLoaderSupplierCallBack.java} | 4 ++-- 3 files changed, 7 insertions(+), 8 deletions(-) rename core/src/main/java/com/dtstack/flink/sql/classloader/{DtSupplier.java => ClassLoaderSupplier.java} (96%) rename core/src/main/java/com/dtstack/flink/sql/classloader/{ClassLoaderCallBackMethod.java => ClassLoaderSupplierCallBack.java} (88%) diff --git a/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderManager.java b/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderManager.java index 45663ff07..6db1058e5 100644 --- a/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderManager.java +++ b/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderManager.java @@ -26,7 +26,6 @@ import java.net.URL; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -43,14 +42,14 @@ public class ClassLoaderManager { private static Map pluginClassLoader = new ConcurrentHashMap<>(); - public static R newInstance(String pluginJarPath, DtSupplier supplier) throws Exception { + public static R newInstance(String pluginJarPath, ClassLoaderSupplier supplier) throws Exception { ClassLoader classLoader = retrieveClassLoad(pluginJarPath); - return ClassLoaderCallBackMethod.callbackAndReset(supplier, classLoader); + return ClassLoaderSupplierCallBack.callbackAndReset(supplier, classLoader); } - public static R newInstance(List jarUrls, DtSupplier supplier) throws Exception { + public static R newInstance(List jarUrls, ClassLoaderSupplier supplier) throws Exception { ClassLoader classLoader = retrieveClassLoad(jarUrls); - return ClassLoaderCallBackMethod.callbackAndReset(supplier, classLoader); + return ClassLoaderSupplierCallBack.callbackAndReset(supplier, classLoader); } private static DtClassLoader retrieveClassLoad(String pluginJarPath) { diff --git a/core/src/main/java/com/dtstack/flink/sql/classloader/DtSupplier.java b/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderSupplier.java similarity index 96% rename from core/src/main/java/com/dtstack/flink/sql/classloader/DtSupplier.java rename to core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderSupplier.java index 779426501..859aa75f4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/classloader/DtSupplier.java +++ b/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderSupplier.java @@ -33,7 +33,7 @@ * @since 1.8 */ @FunctionalInterface -public interface DtSupplier { +public interface ClassLoaderSupplier { /** * Gets a result. diff --git a/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderCallBackMethod.java b/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderSupplierCallBack.java similarity index 88% rename from core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderCallBackMethod.java rename to core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderSupplierCallBack.java index 86dd93372..51d37ef5e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderCallBackMethod.java +++ b/core/src/main/java/com/dtstack/flink/sql/classloader/ClassLoaderSupplierCallBack.java @@ -24,9 +24,9 @@ * author: toutian * create: 2019/10/14 */ -public class ClassLoaderCallBackMethod { +public class ClassLoaderSupplierCallBack { - public static R callbackAndReset(DtSupplier supplier, ClassLoader toSetClassLoader) throws Exception { + public static R callbackAndReset(ClassLoaderSupplier supplier, ClassLoader toSetClassLoader) throws Exception { ClassLoader oldClassLoader = Thread.currentThread().getContextClassLoader(); Thread.currentThread().setContextClassLoader(toSetClassLoader); try { From 46b0fbf466a41cdf0ab61f4fb19dd941e740dfba Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 16 Oct 2019 10:40:29 +0800 Subject: [PATCH 422/470] multiple join --- .../main/java/com/dtstack/flink/sql/Main.java | 2 +- .../com/dtstack/flink/sql/side/JoinInfo.java | 21 +++ .../flink/sql/side/ParserJoinField.java | 6 +- .../dtstack/flink/sql/side/SideSQLParser.java | 176 ++++++++++++++++-- .../dtstack/flink/sql/side/SideSqlExec.java | 2 + .../dtstack/flink/sql/util/ParseUtils.java | 169 +++++++++++++++++ .../sql/side/rdb/async/RdbAsyncSideInfo.java | 3 + 7 files changed, 357 insertions(+), 22 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index bbe997752..e36d08f13 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -101,7 +101,7 @@ public class Main { private static final Logger LOG = LoggerFactory.getLogger(Main.class); - private static Config config = org.apache.calcite.sql.parser.SqlParser + public static Config config = org.apache.calcite.sql.parser.SqlParser .configBuilder() .setLex(Lex.MYSQL) .build(); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java index 03dbde5a6..6043fe23a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java @@ -25,6 +25,7 @@ import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import java.io.Serializable; +import java.util.Map; /** * Join信息 @@ -41,6 +42,8 @@ public class JoinInfo implements Serializable { //左表是否是维表 private boolean leftIsSideTable; + private boolean leftIsMidTable; + //右表是否是维表 private boolean rightIsSideTable; @@ -63,6 +66,8 @@ public class JoinInfo implements Serializable { private SqlNode selectNode; private JoinType joinType; + // 左边是中间转换表,做表映射关系,给替换属性名称使用 + private Map leftTabMapping; public String getSideTableName(){ if(leftIsSideTable){ @@ -87,6 +92,22 @@ public String getNewTableName(){ return leftStr + "_" + rightTableName; } + public boolean isLeftIsMidTable() { + return leftIsMidTable; + } + + public void setLeftIsMidTable(boolean leftIsMidTable) { + this.leftIsMidTable = leftIsMidTable; + } + + public Map getLeftTabMapping() { + return leftTabMapping; + } + + public void setLeftTabMapping(Map leftTabMapping) { + this.leftTabMapping = leftTabMapping; + } + public String getNewTableAlias(){ return leftTableAlias + "_" + rightTableAlias; } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java b/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java index df242a390..27d82e779 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java @@ -41,8 +41,12 @@ public class ParserJoinField { + /** - * Need to parse the fields of information and where selectlist + * build row by field + * @param sqlNode select node + * @param scope join left and right table all info + * @param getAll true,get all fields from two tables; false, extract useful field from select node * @return */ public static List getRowTypeInfo(SqlNode sqlNode, JoinScope scope, boolean getAll){ diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index b0ccc5feb..934812477 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -20,6 +20,8 @@ package com.dtstack.flink.sql.side; +import com.dtstack.flink.sql.Main; +import com.dtstack.flink.sql.util.ParseUtils; import org.apache.calcite.config.Lex; import org.apache.calcite.sql.JoinType; import org.apache.calcite.sql.SqlAsOperator; @@ -38,9 +40,16 @@ import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.commons.collections.CollectionUtils; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; import org.apache.flink.calcite.shaded.com.google.common.collect.Queues; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.util.List; +import java.util.Map; import java.util.Queue; import java.util.Set; @@ -54,6 +63,11 @@ */ public class SideSQLParser { + private static final Logger LOG = LoggerFactory.getLogger(SideSQLParser.class); + + private final char SPLIT = '_'; + + private String tempSQL = "SELECT * FROM TMP"; public Queue getExeQueue(String exeSql, Set sideTableSet) throws SqlParseException { System.out.println("---exeSql---"); @@ -63,7 +77,7 @@ public Queue getExeQueue(String exeSql, Set sideTableSet) throws .configBuilder() .setLex(Lex.MYSQL) .build(); - SqlParser sqlParser = SqlParser.create(exeSql,config); + SqlParser sqlParser = SqlParser.create(exeSql,Main.config); SqlNode sqlNode = sqlParser.parseStmt(); parseSql(sqlNode, sideTableSet, queueInfo); queueInfo.offer(sqlNode); @@ -143,18 +157,37 @@ private Object parseSql(SqlNode sqlNode, Set sideTableSet, Queue return ""; } - private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue queueInfo){ + private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue queueInfo) { SqlNode leftNode = joinNode.getLeft(); SqlNode rightNode = joinNode.getRight(); JoinType joinType = joinNode.getJoinType(); String leftTbName = ""; String leftTbAlias = ""; + Map midTableMapping = null ; + boolean leftIsMidTable = false; + if(leftNode.getKind() == IDENTIFIER){ leftTbName = leftNode.toString(); }else if(leftNode.getKind() == JOIN){ - Object leftNodeJoinInfo = parseSql(leftNode, sideTableSet, queueInfo); - System.out.println(leftNodeJoinInfo); + JoinInfo leftNodeJoinInfo = (JoinInfo)parseSql(leftNode, sideTableSet, queueInfo);//解析多JOIN + // select * from xxx + SqlNode sqlNode = buildSelectByLeftNode(leftNode); + // ( select * from xxx) as xxx_0 + SqlBasicCall newAsNode = buildAsNodeByJoinInfo(leftNodeJoinInfo, sqlNode); + + leftNode = newAsNode; + + joinNode.setLeft(leftNode); + + leftIsMidTable = true; + + midTableMapping = saveTabMapping(leftNodeJoinInfo); + + AliasInfo aliasInfo = (AliasInfo) parseSql(newAsNode, sideTableSet, queueInfo); + leftTbName = aliasInfo.getName(); + leftTbAlias = aliasInfo.getAlias(); + }else if(leftNode.getKind() == AS){ AliasInfo aliasInfo = (AliasInfo) parseSql(leftNode, sideTableSet, queueInfo); leftTbName = aliasInfo.getName(); @@ -184,6 +217,12 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< throw new RuntimeException("side join not support join type of right[current support inner join and left join]"); } + if (leftIsMidTable) { + // 替换右边 on语句 中的字段别名 + SqlNode afterReplaceNameCondition = ParseUtils.replaceJoinConditionTabName(joinNode.getCondition(), midTableMapping); + joinNode.setOperand(5, afterReplaceNameCondition); + } + JoinInfo tableInfo = new JoinInfo(); tableInfo.setLeftTableName(leftTbName); tableInfo.setRightTableName(rightTableName); @@ -204,11 +243,86 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< tableInfo.setJoinType(joinType); tableInfo.setCondition(joinNode.getCondition()); + tableInfo.setLeftIsMidTable(leftIsMidTable); + tableInfo.setLeftTabMapping(midTableMapping); + return tableInfo; } + private Map saveTabMapping(JoinInfo leftNodeJoinInfo) { + Map midTableMapping; + + String midTab = buidTableName(leftNodeJoinInfo.getLeftTableAlias(), SPLIT, leftNodeJoinInfo.getRightTableAlias()); + midTab += "_0"; + + midTableMapping = Maps.newHashMap(); + + if(leftNodeJoinInfo.isLeftIsMidTable()) { + midTableMapping.putAll(leftNodeJoinInfo.getLeftTabMapping()); + } + + midTableMapping.put(leftNodeJoinInfo.getLeftTableAlias(), midTab); + midTableMapping.put(leftNodeJoinInfo.getRightTableAlias(), midTab); + return midTableMapping; + } + + private SqlNode buildSelectByLeftNode(SqlNode leftNode) { + SqlParser sqlParser = SqlParser.create(tempSQL, Main.config); + SqlNode sqlNode = null; + try { + sqlNode = sqlParser.parseStmt(); + }catch (Exception e) { + LOG.error("tmp sql parse error..", e); + } + + ((SqlSelect) sqlNode).setFrom(leftNode); + return sqlNode; + } + private void dealSelectResultWithJoinInfo(JoinInfo joinInfo, SqlSelect sqlNode, Queue queueInfo){ + // 中间虚拟表进行表名称替换 + if (joinInfo.isLeftIsMidTable()){ + SqlNode whereNode = sqlNode.getWhere(); + SqlNodeList sqlGroup = sqlNode.getGroup(); + SqlNodeList sqlSelectList = sqlNode.getSelectList(); + List newSelectNodeList = Lists.newArrayList(); + + for( int i=0; i sideTableList){ if(sideTableList.contains(tableName)){ return true; } - return false; } } diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index adfc4da6a..4b03f044b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -112,6 +112,8 @@ public void exec(String sql, Map sideTableMap, StreamTabl } if(pollSqlNode.getKind() == INSERT){ + System.out.println("----------real exec sql-----------" ); + System.out.println(pollSqlNode.toString()); FlinkSQLExec.sqlUpdate(tableEnv, pollSqlNode.toString()); if(LOG.isInfoEnabled()){ LOG.info("exec sql: " + pollSqlNode.toString()); diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java index accbcdd1a..d4d2b7e48 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java @@ -36,12 +36,23 @@ package com.dtstack.flink.sql.util; +import com.dtstack.flink.sql.side.JoinInfo; import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlDataTypeSpec; +import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlLiteral; import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.fun.SqlCase; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.calcite.shaded.com.google.common.collect.HashBasedTable; import java.util.List; +import java.util.Map; + +import static org.apache.calcite.sql.SqlKind.*; +import static org.apache.calcite.sql.SqlKind.OTHER; /** * @Auther: jiangjunjie @@ -78,4 +89,162 @@ public static String parseOperator(SqlKind sqlKind) { return sqlKind.sql; } + public static SqlNode replaceJoinConditionTabName(SqlNode conditionNode, Map mappingTable) { + SqlNode[] operands = ((SqlBasicCall) conditionNode).getOperands(); + + for (int i = 0; i < operands.length; i++) { + SqlNode sqlNode = operands[i]; + SqlNode replaceNode = replaceNodeInfo(sqlNode, mappingTable); + operands[i] = replaceNode; + } + return conditionNode; + } + + /** + * m.id covert m_x_0.id + * @param selectNode + * @param mapTab + * @return + */ + public static SqlNode replaceSelectFieldTabName(SqlNode selectNode, Map mapTab) { + if (selectNode.getKind() == AS) { + SqlNode leftNode = ((SqlBasicCall) selectNode).getOperands()[0]; + SqlNode replaceNode = replaceSelectFieldTabName(leftNode, mapTab); + if (replaceNode != null) { + ((SqlBasicCall) selectNode).getOperands()[0] = replaceNode; + } + + return selectNode; + }else if(selectNode.getKind() == IDENTIFIER){ + SqlIdentifier sqlIdentifier = (SqlIdentifier) selectNode; + + if(sqlIdentifier.names.size() == 1){ + return selectNode; + } + + String newTableName = ParseUtils.getRootName(mapTab, sqlIdentifier.getComponent(0).getSimple()); + + if(newTableName == null){ + return selectNode; + } + sqlIdentifier = sqlIdentifier.setName(0, newTableName); + return sqlIdentifier; + + }else if(selectNode.getKind() == LITERAL || selectNode.getKind() == LITERAL_CHAIN){//字面含义 + return selectNode; + }else if( AGGREGATE.contains(selectNode.getKind()) + || AVG_AGG_FUNCTIONS.contains(selectNode.getKind()) + || COMPARISON.contains(selectNode.getKind()) + || selectNode.getKind() == OTHER_FUNCTION + || selectNode.getKind() == DIVIDE + || selectNode.getKind() == CAST + || selectNode.getKind() == TRIM + || selectNode.getKind() == TIMES + || selectNode.getKind() == PLUS + || selectNode.getKind() == NOT_IN + || selectNode.getKind() == OR + || selectNode.getKind() == AND + || selectNode.getKind() == MINUS + || selectNode.getKind() == TUMBLE + || selectNode.getKind() == TUMBLE_START + || selectNode.getKind() == TUMBLE_END + || selectNode.getKind() == SESSION + || selectNode.getKind() == SESSION_START + || selectNode.getKind() == SESSION_END + || selectNode.getKind() == HOP + || selectNode.getKind() == HOP_START + || selectNode.getKind() == HOP_END + || selectNode.getKind() == BETWEEN + || selectNode.getKind() == IS_NULL + || selectNode.getKind() == IS_NOT_NULL + || selectNode.getKind() == CONTAINS + + ){ + SqlBasicCall sqlBasicCall = (SqlBasicCall) selectNode; + for(int i=0; i mapTab) { + if (parseNode.getKind() == IDENTIFIER) { + SqlIdentifier sqlIdentifier = (SqlIdentifier) parseNode; + + String newTableName = ParseUtils.getRootName(mapTab, sqlIdentifier.getComponent(0).getSimple());; + + if (newTableName == null || sqlIdentifier.names.size() == 1) { + return sqlIdentifier; + } + sqlIdentifier = sqlIdentifier.setName(0, newTableName); + return sqlIdentifier; + } else if (parseNode instanceof SqlBasicCall) { + SqlBasicCall sqlBasicCall = (SqlBasicCall) parseNode; + for (int i = 0; i < sqlBasicCall.getOperandList().size(); i++) { + SqlNode sqlNode = sqlBasicCall.getOperandList().get(i); + SqlNode replaceNode = replaceSelectFieldTabName(sqlNode, mapTab); + sqlBasicCall.getOperands()[i] = replaceNode; + } + + return sqlBasicCall; + } else { + return parseNode; + } + } + + + public static String getRootName(Map maps, String key) { + String res = null; + while (maps.get(key) !=null) { + res = maps.get(key); + key = res; + } + return res; + } } diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java index aba1d6214..2a8859ca1 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java @@ -79,6 +79,9 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { } sqlCondition = sqlCondition.replace("${tableName}", rdbSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); + + System.out.println("--------side sql query:-------------------"); + System.out.println(sqlCondition); } From 5bc93239b3cc7fc42e39e9b6b6587bf969ee4bbd Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Thu, 17 Oct 2019 09:55:54 +0800 Subject: [PATCH 423/470] =?UTF-8?q?=E7=BB=B4=E8=A1=A8join=20npe=E9=97=AE?= =?UTF-8?q?=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index adfc4da6a..6291d0cc6 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -360,8 +360,14 @@ private SqlNode replaceOrderByTableName(SqlNode orderNode, String tableAlias) { private SqlNode replaceNodeInfo(SqlNode groupNode, HashBasedTable mappingTable, String tableAlias){ if(groupNode.getKind() == IDENTIFIER){ SqlIdentifier sqlIdentifier = (SqlIdentifier) groupNode; + if(sqlIdentifier.names.size() == 1){ + return sqlIdentifier; + } String mappingFieldName = mappingTable.get(sqlIdentifier.getComponent(0).getSimple(), sqlIdentifier.getComponent(1).getSimple()); + if(mappingFieldName == null){ + throw new RuntimeException("can't find mapping fieldName:" + sqlIdentifier.toString() ); + } sqlIdentifier = sqlIdentifier.setName(0, tableAlias); return sqlIdentifier.setName(1, mappingFieldName); }else if(groupNode instanceof SqlBasicCall){ From e4d4f48b7c783c114b1017d359962c2781700da9 Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Thu, 17 Oct 2019 17:35:48 +0800 Subject: [PATCH 424/470] =?UTF-8?q?=E5=AE=9E=E6=97=B6=E8=AE=A1=E7=AE=97?= =?UTF-8?q?=E8=B0=93=E8=AF=8D=E4=B8=8B=E6=8E=A8,=E5=85=B3=E7=B3=BB?= =?UTF-8?q?=E5=9E=8B=E6=95=B0=E6=8D=AE=E5=BA=93=E5=BC=82=E6=AD=A5=E6=96=B9?= =?UTF-8?q?=E5=BC=8F?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flink/sql/util/ParseUtils.java | 56 ++++++++++++++++++- .../sql/side/rdb/async/RdbAsyncSideInfo.java | 22 +++++++- 2 files changed, 76 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java index accbcdd1a..e934e756f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java @@ -39,9 +39,13 @@ import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.*; +import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import java.util.List; +import java.util.Map; /** * @Auther: jiangjunjie @@ -49,6 +53,56 @@ * @Description: */ public class ParseUtils { + public static void parseSideWhere(SqlNode whereNode, Map physicalFields, List whereConditionList) { + SqlKind sqlKind = whereNode.getKind(); + if ((sqlKind == SqlKind.OR || sqlKind == SqlKind.AND) && ((SqlBasicCall) whereNode).getOperandList().size() == 2) { + SqlNode[] sqlOperandsList = ((SqlBasicCall) whereNode).getOperands(); + // whereNode是一颗先解析or再解析and的二叉树。二叉树中序遍历,先左子树,其次中间节点,最后右子树 + parseSideWhere(sqlOperandsList[0], physicalFields, whereConditionList); + whereConditionList.add(sqlKind.name()); + parseSideWhere(sqlOperandsList[1], physicalFields, whereConditionList); + } else { + SqlIdentifier sqlIdentifier = (SqlIdentifier) ((SqlBasicCall) whereNode).getOperands()[0]; + String fieldName = null; + if (sqlIdentifier.names.size() == 1) { + fieldName = sqlIdentifier.getComponent(0).getSimple(); + } else { + fieldName = sqlIdentifier.getComponent(1).getSimple(); + } + if (physicalFields.containsKey(fieldName)) { + String sideFieldName = physicalFields.get(fieldName); + // clone SqlIdentifier node + SqlParserPos sqlParserPos = new SqlParserPos(0, 0); + SqlIdentifier sqlIdentifierClone = new SqlIdentifier("", null, sqlParserPos); + List namesClone = Lists.newArrayList(); + for(String name :sqlIdentifier.names){ + namesClone.add(name); + } + sqlIdentifierClone.setNames(namesClone,null); + // clone SqlBasicCall node + SqlBasicCall sqlBasicCall = (SqlBasicCall)whereNode; + SqlNode[] sqlNodes = sqlBasicCall.getOperands(); + SqlNode[] sqlNodesClone = new SqlNode[sqlNodes.length]; + for (int i = 0; i < sqlNodes.length; i++) { + sqlNodesClone[i] = sqlNodes[i]; + } + SqlBasicCall sqlBasicCallClone = new SqlBasicCall(sqlBasicCall.getOperator(), sqlNodesClone, sqlParserPos); + // 替换维表中真实字段名 + List names = Lists.newArrayList(); + names.add(sideFieldName); + sqlIdentifierClone.setNames(names, null); + + sqlBasicCallClone.setOperand(0, sqlIdentifierClone); + whereConditionList.add(sqlBasicCallClone.toString()); + } else { + // 如果字段不是维表中字段,删除字段前的链接符 + if (whereConditionList.size() >= 1) { + whereConditionList.remove(whereConditionList.size() - 1); + } + } + } + } + public static void parseAnd(SqlNode conditionNode, List sqlNodeList){ if(conditionNode.getKind() == SqlKind.AND && ((SqlBasicCall)conditionNode).getOperandList().size()==2){ parseAnd(((SqlBasicCall)conditionNode).getOperands()[0], sqlNodeList); @@ -78,4 +132,4 @@ public static String parseOperator(SqlKind sqlKind) { return sqlKind.sql; } -} +} \ No newline at end of file diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java index aba1d6214..15d8f2c2b 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java @@ -28,10 +28,14 @@ import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.*; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import java.util.Arrays; import java.util.List; +import java.util.Map; + /** * Reason: @@ -68,6 +72,14 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { dealOneEqualCon(sqlNode, sideTableName); } + List whereConditionList = Lists.newArrayList();; + Map physicalFields = rdbSideTableInfo.getPhysicalFields(); + SqlNode whereNode = ((SqlSelect) joinInfo.getSelectNode()).getWhere(); + if (whereNode != null) { + // 解析维表中的过滤条件 + ParseUtils.parseSideWhere(whereNode, physicalFields, whereConditionList); + } + sqlCondition = "select ${selectField} from ${tableName} where "; for (int i = 0; i < equalFieldList.size(); i++) { String equalField = sideTableInfo.getPhysicalFields().getOrDefault(equalFieldList.get(i), equalFieldList.get(i)); @@ -77,6 +89,14 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { sqlCondition += " and "; } } + if (0 != whereConditionList.size()) { + // 如果where条件中第一个符合条件的是维表中的条件 + String firstCondition = whereConditionList.get(0); + if (!"and".equalsIgnoreCase(firstCondition) && !"or".equalsIgnoreCase(firstCondition)) { + sqlCondition += " and "; + } + sqlCondition += String.join(" ", whereConditionList); + } sqlCondition = sqlCondition.replace("${tableName}", rdbSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); } @@ -134,4 +154,4 @@ public void dealOneEqualCon(SqlNode sqlNode, String sideTableName) { } -} +} \ No newline at end of file From 2e86c0b91e15910b20f261f4e76dbaa34a9aaf2b Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Thu, 17 Oct 2019 20:48:41 +0800 Subject: [PATCH 425/470] stream join convert --- .../main/java/com/dtstack/flink/sql/Main.java | 7 +- .../com/dtstack/flink/sql/side/JoinInfo.java | 2 +- .../dtstack/flink/sql/side/SideSQLParser.java | 102 +++++++++++------- .../dtstack/flink/sql/util/ParseUtils.java | 14 +++ 4 files changed, 80 insertions(+), 45 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index e36d08f13..bb557a8b8 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -21,6 +21,7 @@ package com.dtstack.flink.sql; import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.config.CalciteConfig; import com.dtstack.flink.sql.constrant.ConfigConstrant; import com.dtstack.flink.sql.enums.ClusterMode; import com.dtstack.flink.sql.enums.ECacheType; @@ -101,10 +102,6 @@ public class Main { private static final Logger LOG = LoggerFactory.getLogger(Main.class); - public static Config config = org.apache.calcite.sql.parser.SqlParser - .configBuilder() - .setLex(Lex.MYSQL) - .build(); public static void main(String[] args) throws Exception { @@ -180,7 +177,7 @@ private static void sqlTranslation(Options options,StreamTableEnvironment tableE CreateTmpTableParser.SqlParserResult tmp = sqlTree.getTmpTableMap().get(tableName); String realSql = DtStringUtil.replaceIgnoreQuota(result.getExecSql(), "`", ""); - SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql,config).parseStmt(); + SqlNode sqlNode = org.apache.calcite.sql.parser.SqlParser.create(realSql, CalciteConfig.MYSQL_LEX_CONFIG).parseStmt(); String tmpSql = ((SqlInsert) sqlNode).getSource().toString(); tmp.setExecSql(tmpSql); sideSqlExec.registerTmpTable(tmp, sideTableMap, tableEnv, registerTableCache); diff --git a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java index 6043fe23a..f2c08d1b2 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java @@ -41,7 +41,7 @@ public class JoinInfo implements Serializable { //左表是否是维表 private boolean leftIsSideTable; - + //左表是 转换后的中间表 private boolean leftIsMidTable; //右表是否是维表 diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 934812477..5ec79ec3b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -20,9 +20,8 @@ package com.dtstack.flink.sql.side; -import com.dtstack.flink.sql.Main; +import com.dtstack.flink.sql.config.CalciteConfig; import com.dtstack.flink.sql.util.ParseUtils; -import org.apache.calcite.config.Lex; import org.apache.calcite.sql.JoinType; import org.apache.calcite.sql.SqlAsOperator; import org.apache.calcite.sql.SqlBasicCall; @@ -41,6 +40,7 @@ import org.apache.calcite.sql.parser.SqlParser; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.calcite.shaded.com.google.common.base.Strings; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; @@ -48,6 +48,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Queue; @@ -72,12 +73,11 @@ public class SideSQLParser { public Queue getExeQueue(String exeSql, Set sideTableSet) throws SqlParseException { System.out.println("---exeSql---"); System.out.println(exeSql); + LOG.info("---exeSql---"); + LOG.info(exeSql); + Queue queueInfo = Queues.newLinkedBlockingQueue(); - SqlParser.Config config = SqlParser - .configBuilder() - .setLex(Lex.MYSQL) - .build(); - SqlParser sqlParser = SqlParser.create(exeSql,Main.config); + SqlParser sqlParser = SqlParser.create(exeSql, CalciteConfig.MYSQL_LEX_CONFIG); SqlNode sqlNode = sqlParser.parseStmt(); parseSql(sqlNode, sideTableSet, queueInfo); queueInfo.offer(sqlNode); @@ -163,30 +163,40 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< JoinType joinType = joinNode.getJoinType(); String leftTbName = ""; String leftTbAlias = ""; + String rightTableName = ""; + String rightTableAlias = ""; Map midTableMapping = null ; boolean leftIsMidTable = false; + // 右节点已经被解析 + boolean rightIsParse = false; + Tuple2 rightTableNameAndAlias = null; if(leftNode.getKind() == IDENTIFIER){ leftTbName = leftNode.toString(); }else if(leftNode.getKind() == JOIN){ JoinInfo leftNodeJoinInfo = (JoinInfo)parseSql(leftNode, sideTableSet, queueInfo);//解析多JOIN - // select * from xxx - SqlNode sqlNode = buildSelectByLeftNode(leftNode); - // ( select * from xxx) as xxx_0 - SqlBasicCall newAsNode = buildAsNodeByJoinInfo(leftNodeJoinInfo, sqlNode); - - leftNode = newAsNode; - - joinNode.setLeft(leftNode); - - leftIsMidTable = true; - midTableMapping = saveTabMapping(leftNodeJoinInfo); - - AliasInfo aliasInfo = (AliasInfo) parseSql(newAsNode, sideTableSet, queueInfo); - leftTbName = aliasInfo.getName(); - leftTbAlias = aliasInfo.getAlias(); + rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo); + rightIsParse = true; + if (checkIsSideTable(rightTableNameAndAlias.f0, sideTableSet)) { + // select * from xxx + SqlNode sqlNode = buildSelectByLeftNode(leftNode); + // ( select * from xxx) as xxx_0 + SqlBasicCall newAsNode = buildAsNodeByJoinInfo(leftNodeJoinInfo, sqlNode); + leftNode = newAsNode; + joinNode.setLeft(leftNode); + + leftIsMidTable = true; + midTableMapping = saveTabMapping(leftNodeJoinInfo); + + AliasInfo aliasInfo = (AliasInfo) parseSql(newAsNode, sideTableSet, queueInfo); + leftTbName = aliasInfo.getName(); + leftTbAlias = aliasInfo.getAlias(); + } else { + leftTbName = leftNodeJoinInfo.getRightTableName(); + leftTbAlias = leftNodeJoinInfo.getRightTableAlias(); + } }else if(leftNode.getKind() == AS){ AliasInfo aliasInfo = (AliasInfo) parseSql(leftNode, sideTableSet, queueInfo); @@ -201,16 +211,11 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< throw new RuntimeException("side-table must be at the right of join operator"); } - String rightTableName = ""; - String rightTableAlias = ""; - - if(rightNode.getKind() == IDENTIFIER){ - rightTableName = rightNode.toString(); - }else{ - AliasInfo aliasInfo = (AliasInfo)parseSql(rightNode, sideTableSet, queueInfo); - rightTableName = aliasInfo.getName(); - rightTableAlias = aliasInfo.getAlias(); + if (!rightIsParse) { + rightTableNameAndAlias = parseRightNode(rightNode, sideTableSet, queueInfo); } + rightTableName = rightTableNameAndAlias.f0; + rightTableAlias = rightTableNameAndAlias.f1; boolean rightIsSide = checkIsSideTable(rightTableName, sideTableSet); if(joinType == JoinType.RIGHT){ @@ -249,25 +254,44 @@ private JoinInfo dealJoinNode(SqlJoin joinNode, Set sideTableSet, Queue< return tableInfo; } + private Tuple2 parseRightNode(SqlNode sqlNode, Set sideTableSet, Queue queueInfo) { + Tuple2 tabName = new Tuple2<>("", ""); + if(sqlNode.getKind() == IDENTIFIER){ + tabName.f0 = sqlNode.toString(); + }else{ + AliasInfo aliasInfo = (AliasInfo)parseSql(sqlNode, sideTableSet, queueInfo); + tabName.f0 = aliasInfo.getName(); + tabName.f1 = aliasInfo.getAlias(); + } + return tabName; + } + private Map saveTabMapping(JoinInfo leftNodeJoinInfo) { - Map midTableMapping; + Map midTableMapping = Maps.newHashMap();; String midTab = buidTableName(leftNodeJoinInfo.getLeftTableAlias(), SPLIT, leftNodeJoinInfo.getRightTableAlias()); - midTab += "_0"; - - midTableMapping = Maps.newHashMap(); + String finalMidTab = midTab + "_0"; if(leftNodeJoinInfo.isLeftIsMidTable()) { midTableMapping.putAll(leftNodeJoinInfo.getLeftTabMapping()); } - - midTableMapping.put(leftNodeJoinInfo.getLeftTableAlias(), midTab); - midTableMapping.put(leftNodeJoinInfo.getRightTableAlias(), midTab); + fillLeftAllTable(leftNodeJoinInfo, midTableMapping, finalMidTab); return midTableMapping; } + private void fillLeftAllTable(JoinInfo leftNodeJoinInfo, Map midTableMapping, String finalMidTab) { + List tablesName = Lists.newArrayList(); + ParseUtils.parseLeftNodeTableName(leftNodeJoinInfo.getLeftNode(), tablesName); + + tablesName.forEach(tab ->{ + midTableMapping.put(tab, finalMidTab); + }); + midTableMapping.put(leftNodeJoinInfo.getRightTableAlias(), finalMidTab); + } + + private SqlNode buildSelectByLeftNode(SqlNode leftNode) { - SqlParser sqlParser = SqlParser.create(tempSQL, Main.config); + SqlParser sqlParser = SqlParser.create(tempSQL, CalciteConfig.MYSQL_LEX_CONFIG); SqlNode sqlNode = null; try { sqlNode = sqlParser.parseStmt(); diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java index d4d2b7e48..4a2861254 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java @@ -40,6 +40,7 @@ import org.apache.calcite.sql.SqlBasicCall; import org.apache.calcite.sql.SqlDataTypeSpec; import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlJoin; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlLiteral; import org.apache.calcite.sql.SqlNode; @@ -247,4 +248,17 @@ public static String getRootName(Map maps, String key) { } return res; } + + public static void parseLeftNodeTableName(SqlNode leftJoin, List tablesName) { + if (leftJoin.getKind() == IDENTIFIER) { + SqlIdentifier sqlIdentifier = (SqlIdentifier) leftJoin; + tablesName.add(sqlIdentifier.names.get(0)); + } else if (leftJoin.getKind() == AS) { + SqlNode sqlNode = ((SqlBasicCall) leftJoin).getOperands()[1]; + tablesName.add(sqlNode.toString()); + } else if (leftJoin.getKind() == JOIN) { + parseLeftNodeTableName(((SqlJoin) leftJoin).getLeft(), tablesName); + parseLeftNodeTableName(((SqlJoin) leftJoin).getRight(), tablesName); + } + } } From 5b21e5aa304cca238ed0c23486dad4a35d5f255d Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Thu, 17 Oct 2019 20:51:20 +0800 Subject: [PATCH 426/470] add calcite config --- .../flink/sql/config/CalciteConfig.java | 34 +++++++++++++++++++ 1 file changed, 34 insertions(+) create mode 100644 core/src/main/java/com/dtstack/flink/sql/config/CalciteConfig.java diff --git a/core/src/main/java/com/dtstack/flink/sql/config/CalciteConfig.java b/core/src/main/java/com/dtstack/flink/sql/config/CalciteConfig.java new file mode 100644 index 000000000..a7a711d12 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/config/CalciteConfig.java @@ -0,0 +1,34 @@ +/* + * 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 com.dtstack.flink.sql.config; + +import org.apache.calcite.config.Lex; +import org.apache.calcite.sql.parser.SqlParser; + +public class CalciteConfig { + + public static SqlParser.Config MYSQL_LEX_CONFIG = org.apache.calcite.sql.parser.SqlParser + .configBuilder() + .setLex(Lex.MYSQL) + .build(); + + + +} From c7fe9d548e9d46cb0a69574ed5d3f2a7f1a7b761 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 18 Oct 2019 17:41:27 +0800 Subject: [PATCH 427/470] =?UTF-8?q?=E5=88=A0=E9=99=A4=E6=97=A0=E7=94=A8?= =?UTF-8?q?=E6=96=87=E4=BB=B6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- v1.8.0_dev_bugfix_joinnpe | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 v1.8.0_dev_bugfix_joinnpe diff --git a/v1.8.0_dev_bugfix_joinnpe b/v1.8.0_dev_bugfix_joinnpe deleted file mode 100644 index e69de29bb..000000000 From 47999244e694ff9e87f24ccec442c1c480fa8578 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Fri, 18 Oct 2019 20:24:16 +0800 Subject: [PATCH 428/470] modify class pre --- .../main/java/com/dtstack/flink/sql/config/CalciteConfig.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/config/CalciteConfig.java b/core/src/main/java/com/dtstack/flink/sql/config/CalciteConfig.java index a7a711d12..54ae66bbc 100644 --- a/core/src/main/java/com/dtstack/flink/sql/config/CalciteConfig.java +++ b/core/src/main/java/com/dtstack/flink/sql/config/CalciteConfig.java @@ -21,10 +21,11 @@ import org.apache.calcite.config.Lex; import org.apache.calcite.sql.parser.SqlParser; +import org.apache.calcite.sql.parser.SqlParser.Config; public class CalciteConfig { - public static SqlParser.Config MYSQL_LEX_CONFIG = org.apache.calcite.sql.parser.SqlParser + public static Config MYSQL_LEX_CONFIG = SqlParser .configBuilder() .setLex(Lex.MYSQL) .build(); From 900b45e57bbc3e02e6e31c0c1d10b02634272424 Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Sun, 20 Oct 2019 20:58:07 +0800 Subject: [PATCH 429/470] =?UTF-8?q?cep=E5=8A=9F?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flink/sql/parser/InsertSqlParser.java | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index ff2bb9e4b..e946c2b5a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -21,13 +21,7 @@ package com.dtstack.flink.sql.parser; import org.apache.calcite.config.Lex; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlInsert; -import org.apache.calcite.sql.SqlJoin; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlOrderBy; -import org.apache.calcite.sql.SqlSelect; +import org.apache.calcite.sql.*; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; import org.apache.commons.lang3.StringUtils; @@ -119,6 +113,10 @@ private static void parseNode(SqlNode sqlNode, SqlParseResult sqlParseResult){ sqlParseResult.addSourceTable(identifierNode.toString()); } break; + case MATCH_RECOGNIZE: + SqlMatchRecognize node = (SqlMatchRecognize) sqlNode; + sqlParseResult.addSourceTable(node.getTableRef().toString()); + break; case UNION: SqlNode unionLeft = ((SqlBasicCall)sqlNode).getOperands()[0]; SqlNode unionRight = ((SqlBasicCall)sqlNode).getOperands()[1]; From 3f860e64cafea6212750e319e2da6fff8d3f3d9c Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 21 Oct 2019 15:58:11 +0800 Subject: [PATCH 430/470] extract replaceWhenOrThenSelectFieldTabName method --- .../dtstack/flink/sql/util/ParseUtils.java | 31 +++++++++---------- 1 file changed, 14 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java index 4a2861254..bc4a0e577 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java @@ -78,12 +78,12 @@ public static void parseJoinCompareOperate(SqlNode condition, List sqlJo parseJoinCompareOperate(sqlNode, sqlJoinCompareOperate); } } else { - String operator = parseOperator(joinCondition.getKind()); + String operator = transformNotEqualsOperator(joinCondition.getKind()); sqlJoinCompareOperate.add(operator); } } - public static String parseOperator(SqlKind sqlKind) { + public static String transformNotEqualsOperator(SqlKind sqlKind) { if (StringUtils.equalsIgnoreCase(sqlKind.toString(), "NOT_EQUALS")){ return "!="; } @@ -188,21 +188,8 @@ public static SqlNode replaceSelectFieldTabName(SqlNode selectNode, Map mapTab, SqlNodeList thenOperands) { + for(int i=0; i mapTab) { if (parseNode.getKind() == IDENTIFIER) { SqlIdentifier sqlIdentifier = (SqlIdentifier) parseNode; From b2e22867cc536eee49ab52acd0fd6af9bc8ca7b8 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 21 Oct 2019 22:10:25 +0800 Subject: [PATCH 431/470] midify DtStringUtil splitIgnoreQuota --- .../com/dtstack/flink/sql/util/DtStringUtil.java | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index 9d23afad6..ed31618d1 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -59,7 +59,13 @@ public static List splitIgnoreQuota(String str, char delimiter){ boolean inSingleQuotes = false; int bracketLeftNum = 0; StringBuilder b = new StringBuilder(); - for (char c : str.toCharArray()) { + char[] chars = str.toCharArray(); + int idx = 0; + for (char c : chars) { + char flag = 0; + if (idx > 0) { + flag = chars[idx - 1]; + } if(c == delimiter){ if (inQuotes) { b.append(c); @@ -71,10 +77,10 @@ public static List splitIgnoreQuota(String str, char delimiter){ tokensList.add(b.toString()); b = new StringBuilder(); } - }else if(c == '\"'){ + }else if(c == '\"' && '\\'!=flag && !inSingleQuotes){ inQuotes = !inQuotes; b.append(c); - }else if(c == '\''){ + }else if(c == '\'' && '\\'!=flag && !inQuotes){ inSingleQuotes = !inSingleQuotes; b.append(c); }else if(c == '('){ @@ -86,6 +92,7 @@ public static List splitIgnoreQuota(String str, char delimiter){ }else{ b.append(c); } + idx++; } tokensList.add(b.toString()); From 1f9234267130b2ba201c6eff62606e61e40ae7f1 Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Tue, 22 Oct 2019 19:13:37 +0800 Subject: [PATCH 432/470] =?UTF-8?q?[flinksql][udf=E7=B1=BB=E4=B8=8D?= =?UTF-8?q?=E8=83=BD=E5=8A=A0=E8=BD=BD][19234]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/src/main/java/com/dtstack/flink/sql/Main.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index d55f0ceea..2541c0312 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -249,12 +249,14 @@ private static void registerUDF(SqlTree sqlTree, List jarURList, URLClassLo StreamTableEnvironment tableEnv) throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException { //register urf + // udf函数只能由appclassloader加载 + ClassLoader superClassLoader = Thread.currentThread().getContextClassLoader().getParent(); URLClassLoader classLoader = null; List funcList = sqlTree.getFunctionList(); for (CreateFuncParser.SqlParserResult funcInfo : funcList) { //classloader if (classLoader == null) { - classLoader = FlinkUtil.loadExtraJar(jarURList, parentClassloader); + classLoader = FlinkUtil.loadExtraJar(jarURList, (URLClassLoader)superClassLoader); } FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName(), tableEnv, classLoader); From 743d2949854de6ef45e9dc6873f5ce19e4cbfdd4 Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Thu, 24 Oct 2019 16:06:03 +0800 Subject: [PATCH 433/470] =?UTF-8?q?[flinksql][udf=E7=B1=BB=E5=8A=A0?= =?UTF-8?q?=E8=BD=BD=E5=99=A8=E5=92=8CtableEnv=E4=B8=8D=E6=98=AF=E5=90=8C?= =?UTF-8?q?=E4=B8=80=E4=B8=AA][19234]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- core/src/main/java/com/dtstack/flink/sql/Main.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 2541c0312..6ea1fea98 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -249,14 +249,14 @@ private static void registerUDF(SqlTree sqlTree, List jarURList, URLClassLo StreamTableEnvironment tableEnv) throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException { //register urf - // udf函数只能由appclassloader加载 - ClassLoader superClassLoader = Thread.currentThread().getContextClassLoader().getParent(); + // udf和tableEnv须由同一个类加载器加载 + ClassLoader levelClassLoader = tableEnv.getClass().getClassLoader(); URLClassLoader classLoader = null; List funcList = sqlTree.getFunctionList(); for (CreateFuncParser.SqlParserResult funcInfo : funcList) { //classloader if (classLoader == null) { - classLoader = FlinkUtil.loadExtraJar(jarURList, (URLClassLoader)superClassLoader); + classLoader = FlinkUtil.loadExtraJar(jarURList, (URLClassLoader)levelClassLoader); } FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName(), tableEnv, classLoader); From ed0ecd7684814d19bb81b6b5c8e88888ba214733 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 28 Oct 2019 10:49:27 +0800 Subject: [PATCH 434/470] fix conflict --- core/src/main/java/com/dtstack/flink/sql/Main.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index b1a140ba9..088f7026f 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql; -import com.dtstack.flink.sql.classloader.DtClassLoader; +import com.dtstack.flink.sql.classloader.ClassLoaderManager; import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.exec.FlinkSQLExec; import com.dtstack.flink.sql.parser.CreateFuncParser; From 87f432e2e5d40fefb7c4c8cf4f5b024dda8126c2 Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Mon, 28 Oct 2019 15:48:09 +0800 Subject: [PATCH 435/470] =?UTF-8?q?udf=E7=B1=BB=E5=8A=A0=E8=BD=BD=E5=99=A8?= =?UTF-8?q?=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/Main.java | 12 ++++++++++-- .../com/dtstack/flink/sql/util/FlinkUtil.java | 16 +++++++++------- 2 files changed, 19 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 088f7026f..603210a5d 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -77,6 +77,7 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.net.URL; +import java.net.URLClassLoader; import java.net.URLDecoder; import java.util.List; import java.util.Map; @@ -237,11 +238,18 @@ private static void addEnvClassPath(StreamExecutionEnvironment env, Set cla } private static void registerUDF(SqlTree sqlTree, List jarURList, StreamTableEnvironment tableEnv) - throws ClassNotFoundException, NoSuchMethodException, IllegalAccessException, InvocationTargetException { + throws NoSuchMethodException, IllegalAccessException, InvocationTargetException { //register urf + // udf和tableEnv须由同一个类加载器加载 + ClassLoader levelClassLoader = tableEnv.getClass().getClassLoader(); + URLClassLoader classLoader = null; List funcList = sqlTree.getFunctionList(); for (CreateFuncParser.SqlParserResult funcInfo : funcList) { - FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName(), tableEnv, jarURList); + //classloader + if (classLoader == null) { + classLoader = FlinkUtil.loadExtraJar(jarURList, (URLClassLoader)levelClassLoader); + } + FlinkUtil.registerUDF(funcInfo.getType(), funcInfo.getClassName(), funcInfo.getName(), tableEnv, classLoader); } } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java index a878b0d72..58be0f74c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java @@ -152,11 +152,11 @@ public static void setStreamTimeCharacteristic(StreamExecutionEnvironment env, P * TABLE|SCALA * 注册UDF到table env */ - public static void registerUDF(String type, String classPath, String funcName, TableEnvironment tableEnv, List jarURList){ + public static void registerUDF(String type, String classPath, String funcName, TableEnvironment tableEnv, ClassLoader classLoader){ if("SCALA".equalsIgnoreCase(type)){ - registerScalaUDF(classPath, funcName, tableEnv, jarURList); + registerScalaUDF(classPath, funcName, tableEnv, classLoader); }else if("TABLE".equalsIgnoreCase(type)){ - registerTableUDF(classPath, funcName, tableEnv, jarURList); + registerTableUDF(classPath, funcName, tableEnv, classLoader); }else{ throw new RuntimeException("not support of UDF which is not in (TABLE, SCALA)"); } @@ -169,9 +169,10 @@ public static void registerUDF(String type, String classPath, String funcName, T * @param funcName * @param tableEnv */ - public static void registerScalaUDF(String classPath, String funcName, TableEnvironment tableEnv, List jarURList){ + public static void registerScalaUDF(String classPath, String funcName, TableEnvironment tableEnv, ClassLoader classLoader){ try{ - ScalarFunction udfFunc = ClassLoaderManager.newInstance(jarURList, (cl) -> cl.loadClass(classPath).asSubclass(ScalarFunction.class).newInstance()); + ScalarFunction udfFunc = Class.forName(classPath, false, classLoader) + .asSubclass(ScalarFunction.class).newInstance(); tableEnv.registerFunction(funcName, udfFunc); logger.info("register scala function:{} success.", funcName); }catch (Exception e){ @@ -187,9 +188,10 @@ public static void registerScalaUDF(String classPath, String funcName, TableEnvi * @param funcName * @param tableEnv */ - public static void registerTableUDF(String classPath, String funcName, TableEnvironment tableEnv, List jarURList){ + public static void registerTableUDF(String classPath, String funcName, TableEnvironment tableEnv, ClassLoader classLoader){ try { - TableFunction udfFunc = ClassLoaderManager.newInstance(jarURList, (cl) -> cl.loadClass(classPath).asSubclass(TableFunction.class).newInstance()); + ScalarFunction udfFunc = Class.forName(classPath, false, classLoader) + .asSubclass(ScalarFunction.class).newInstance(); if(tableEnv instanceof StreamTableEnvironment){ ((StreamTableEnvironment)tableEnv).registerFunction(funcName, udfFunc); }else if(tableEnv instanceof BatchTableEnvironment){ From bcbbf63ea5aac193417295d2e6684168e3174910 Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Mon, 28 Oct 2019 19:28:30 +0800 Subject: [PATCH 436/470] =?UTF-8?q?udf=E7=B1=BB=E5=8A=A0=E8=BD=BD=E9=97=AE?= =?UTF-8?q?=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flink/sql/util/FlinkUtil.java | 52 +++++++++++++++---- 1 file changed, 41 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java index a878b0d72..bafb643fe 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java @@ -22,6 +22,7 @@ import com.dtstack.flink.sql.classloader.ClassLoaderManager; +import com.dtstack.flink.sql.constrant.ConfigConstrant; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.runtime.state.filesystem.FsStateBackend; @@ -32,6 +33,7 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.java.BatchTableEnvironment; import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.TableFunction; import org.slf4j.Logger; @@ -148,17 +150,18 @@ public static void setStreamTimeCharacteristic(StreamExecutionEnvironment env, P /** - * FIXME 暂时不支持 UDF 实现类--有参构造方法 - * TABLE|SCALA + * TABLE|SCALA|AGGREGATE * 注册UDF到table env */ - public static void registerUDF(String type, String classPath, String funcName, TableEnvironment tableEnv, List jarURList){ + public static void registerUDF(String type, String classPath, String funcName, TableEnvironment tableEnv, ClassLoader classLoader){ if("SCALA".equalsIgnoreCase(type)){ - registerScalaUDF(classPath, funcName, tableEnv, jarURList); + registerScalaUDF(classPath, funcName, tableEnv, classLoader); }else if("TABLE".equalsIgnoreCase(type)){ - registerTableUDF(classPath, funcName, tableEnv, jarURList); + registerTableUDF(classPath, funcName, tableEnv, classLoader); + }else if("AGGREGATE".equalsIgnoreCase(type)){ + registerAggregateUDF(classPath, funcName, tableEnv, classLoader); }else{ - throw new RuntimeException("not support of UDF which is not in (TABLE, SCALA)"); + throw new RuntimeException("not support of UDF which is not in (TABLE, SCALA, AGGREGATE)"); } } @@ -169,9 +172,10 @@ public static void registerUDF(String type, String classPath, String funcName, T * @param funcName * @param tableEnv */ - public static void registerScalaUDF(String classPath, String funcName, TableEnvironment tableEnv, List jarURList){ + public static void registerScalaUDF(String classPath, String funcName, TableEnvironment tableEnv, ClassLoader classLoader){ try{ - ScalarFunction udfFunc = ClassLoaderManager.newInstance(jarURList, (cl) -> cl.loadClass(classPath).asSubclass(ScalarFunction.class).newInstance()); + ScalarFunction udfFunc = Class.forName(classPath, false, classLoader) + .asSubclass(ScalarFunction.class).newInstance(); tableEnv.registerFunction(funcName, udfFunc); logger.info("register scala function:{} success.", funcName); }catch (Exception e){ @@ -182,14 +186,15 @@ public static void registerScalaUDF(String classPath, String funcName, TableEnvi /** * 注册自定义TABLEFFUNC方法到env上 - * TODO 对User-Defined Aggregate Functions的支持 + * * @param classPath * @param funcName * @param tableEnv */ - public static void registerTableUDF(String classPath, String funcName, TableEnvironment tableEnv, List jarURList){ + public static void registerTableUDF(String classPath, String funcName, TableEnvironment tableEnv, ClassLoader classLoader){ try { - TableFunction udfFunc = ClassLoaderManager.newInstance(jarURList, (cl) -> cl.loadClass(classPath).asSubclass(TableFunction.class).newInstance()); + TableFunction udfFunc = Class.forName(classPath, false, classLoader) + .asSubclass(TableFunction.class).newInstance(); if(tableEnv instanceof StreamTableEnvironment){ ((StreamTableEnvironment)tableEnv).registerFunction(funcName, udfFunc); }else if(tableEnv instanceof BatchTableEnvironment){ @@ -205,6 +210,31 @@ public static void registerTableUDF(String classPath, String funcName, TableEnvi } } + /** + * 注册自定义Aggregate FUNC方法到env上 + * + * @param classPath + * @param funcName + * @param tableEnv + */ + public static void registerAggregateUDF(String classPath, String funcName, TableEnvironment tableEnv, ClassLoader classLoader) { + try { + AggregateFunction udfFunc = Class.forName(classPath, false, classLoader) + .asSubclass(AggregateFunction.class).newInstance(); + if (tableEnv instanceof StreamTableEnvironment) { + ((StreamTableEnvironment) tableEnv).registerFunction(funcName, udfFunc); + } else if (tableEnv instanceof BatchTableEnvironment) { + ((BatchTableEnvironment) tableEnv).registerFunction(funcName, udfFunc); + } else { + throw new RuntimeException("no support tableEnvironment class for " + tableEnv.getClass().getName()); + } + + logger.info("register Aggregate function:{} success.", funcName); + } catch (Exception e) { + logger.error("", e); + throw new RuntimeException("register Aggregate UDF exception:", e); + } + } /** * From a503015654a26b3cbc4e526a66307400e44b9a52 Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Mon, 28 Oct 2019 20:56:20 +0800 Subject: [PATCH 437/470] fix conflict --- .../com/dtstack/flink/sql/util/FlinkUtil.java | 38 +++++++++---------- 1 file changed, 18 insertions(+), 20 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java index bafb643fe..5811105e4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/FlinkUtil.java @@ -16,7 +16,7 @@ * limitations under the License. */ - + package com.dtstack.flink.sql.util; @@ -33,9 +33,10 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.java.BatchTableEnvironment; import org.apache.flink.table.api.java.StreamTableEnvironment; -import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.functions.ScalarFunction; import org.apache.flink.table.functions.TableFunction; +import org.apache.flink.table.functions.AggregateFunction; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -70,12 +71,14 @@ public static void openCheckpoint(StreamExecutionEnvironment env, Properties pro } //设置了时间间隔才表明开启了checkpoint - if(properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_INTERVAL_KEY) == null){ + if(properties.getProperty(ConfigConstrant.SQL_CHECKPOINT_INTERVAL_KEY) == null && properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_INTERVAL_KEY) == null){ return; }else{ - Long interval = Long.valueOf(properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_INTERVAL_KEY)); + Long sql_interval = Long.valueOf(properties.getProperty(ConfigConstrant.SQL_CHECKPOINT_INTERVAL_KEY,"0")); + Long flink_interval = Long.valueOf(properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_INTERVAL_KEY, "0")); + long checkpointInterval = Math.max(sql_interval, flink_interval); //start checkpoint every ${interval} - env.enableCheckpointing(interval); + env.enableCheckpointing(checkpointInterval); } String checkMode = properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_MODE_KEY); @@ -103,7 +106,14 @@ public static void openCheckpoint(StreamExecutionEnvironment env, Properties pro env.getCheckpointConfig().setMaxConcurrentCheckpoints(maxConcurrCheckpoints); } - String cleanupModeStr = properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_CLEANUPMODE_KEY); + Boolean sqlCleanMode = MathUtil.getBoolean(properties.getProperty(ConfigConstrant.SQL_CHECKPOINT_CLEANUPMODE_KEY), false); + Boolean flinkCleanMode = MathUtil.getBoolean(properties.getProperty(ConfigConstrant.FLINK_CHECKPOINT_CLEANUPMODE_KEY), false); + + String cleanupModeStr = "false"; + if (sqlCleanMode || flinkCleanMode ){ + cleanupModeStr = "true"; + } + if ("true".equalsIgnoreCase(cleanupModeStr)){ env.getCheckpointConfig().enableExternalizedCheckpoints( CheckpointConfig.ExternalizedCheckpointCleanup.DELETE_ON_CANCELLATION); @@ -139,6 +149,7 @@ public static void setStreamTimeCharacteristic(StreamExecutionEnvironment env, P if(characteristicStr.equalsIgnoreCase(tmp.toString())){ env.setStreamTimeCharacteristic(tmp); flag = true; + break; } } @@ -148,7 +159,6 @@ public static void setStreamTimeCharacteristic(StreamExecutionEnvironment env, P } - /** * TABLE|SCALA|AGGREGATE * 注册UDF到table env @@ -269,21 +279,9 @@ public static long getBufferTimeoutMillis(Properties properties){ } public static URLClassLoader loadExtraJar(List jarURLList, URLClassLoader classLoader) throws NoSuchMethodException, IllegalAccessException, InvocationTargetException { - - int size = 0; - for(URL url : jarURLList){ - if(url.toString().endsWith(".jar")){ - size++; - } - } - - URL[] urlArray = new URL[size]; - int i=0; for(URL url : jarURLList){ if(url.toString().endsWith(".jar")){ - urlArray[i] = url; urlClassLoaderAddUrl(classLoader, url); - i++; } } @@ -311,4 +309,4 @@ public static TypeInformation[] transformTypes(Class[] fieldTypes){ return types; } -} +} \ No newline at end of file From da8db69e807b0199865098a84345240ea40884a4 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Thu, 31 Oct 2019 10:34:35 +0800 Subject: [PATCH 438/470] add plugin load mode classpath or shipfile --- .../main/java/com/dtstack/flink/sql/Main.java | 37 +++++++++---- .../flink/sql/enums/PluginLoadMode.java | 15 ++++++ .../com/dtstack/flink/sql/option/Options.java | 12 +++++ .../dtstack/flink/sql/util/PluginUtil.java | 16 ++++++ .../perjob/PerJobClusterClientBuilder.java | 53 +++++++++++++++---- .../sql/launcher/perjob/PerJobSubmitter.java | 14 +---- 6 files changed, 112 insertions(+), 35 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/enums/PluginLoadMode.java diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 1462858a6..cd4cd1194 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -25,6 +25,7 @@ import com.dtstack.flink.sql.constrant.ConfigConstrant; import com.dtstack.flink.sql.enums.ClusterMode; import com.dtstack.flink.sql.enums.ECacheType; +import com.dtstack.flink.sql.enums.PluginLoadMode; import com.dtstack.flink.sql.environment.MyLocalStreamEnvironment; import com.dtstack.flink.sql.exec.FlinkSQLExec; import com.dtstack.flink.sql.option.OptionParser; @@ -45,10 +46,10 @@ import com.dtstack.flink.sql.watermarker.WaterMarkerAssigner; import com.dtstack.flink.sql.util.FlinkUtil; import com.dtstack.flink.sql.util.PluginUtil; -import org.apache.calcite.config.Lex; import org.apache.calcite.sql.SqlInsert; import org.apache.calcite.sql.SqlNode; import org.apache.commons.io.Charsets; +import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Time; @@ -84,7 +85,6 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import com.dtstack.flink.sql.option.Options; -import org.apache.calcite.sql.parser.SqlParser.Config; /** * Date: 2018/6/26 @@ -109,9 +109,9 @@ public static void main(String[] args) throws Exception { String name = options.getName(); String addJarListStr = options.getAddjar(); String localSqlPluginPath = options.getLocalSqlPluginPath(); - String remoteSqlPluginPath = options.getRemoteSqlPluginPath(); String deployMode = options.getMode(); String confProp = options.getConfProp(); + sql = URLDecoder.decode(sql, Charsets.UTF_8.name()); SqlParser.setLocalSqlPluginRoot(localSqlPluginPath); @@ -141,7 +141,7 @@ public static void main(String[] args) throws Exception { //register udf registerUDF(sqlTree, jarURList, tableEnv); //register table schema - registerTable(sqlTree, env, tableEnv, localSqlPluginPath, remoteSqlPluginPath, sideTableMap, registerTableCache); + registerTable(sqlTree, env, tableEnv, options, sideTableMap, registerTableCache); sqlTranslation(options,tableEnv,sqlTree,sideTableMap,registerTableCache); @@ -230,8 +230,7 @@ private static void registerUDF(SqlTree sqlTree, List jarURList, StreamTabl } - private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv, - String localSqlPluginPath, String remoteSqlPluginPath, + private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv, Options options, Map sideTableMap, Map registerTableCache) throws Exception { Set classPathSet = Sets.newHashSet(); WaterMarkerAssigner waterMarkerAssigner = new WaterMarkerAssigner(); @@ -240,7 +239,7 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en if (tableInfo instanceof SourceTableInfo) { SourceTableInfo sourceTableInfo = (SourceTableInfo) tableInfo; - Table table = StreamSourceFactory.getStreamSource(sourceTableInfo, env, tableEnv, localSqlPluginPath); + Table table = StreamSourceFactory.getStreamSource(sourceTableInfo, env, tableEnv, options.getLocalSqlPluginPath()); tableEnv.registerTable(sourceTableInfo.getAdaptName(), table); //Note --- parameter conversion function can not be used inside a function of the type of polymerization //Create table in which the function is arranged only need adaptation sql @@ -267,18 +266,18 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en LOG.info("registe table {} success.", tableInfo.getName()); } registerTableCache.put(tableInfo.getName(), regTable); - classPathSet.add(PluginUtil.getRemoteJarFilePath(tableInfo.getType(), SourceTableInfo.SOURCE_SUFFIX, remoteSqlPluginPath, localSqlPluginPath)); + classPathSet.add(buildSourceAndSinkPathByLoadMode(tableInfo.getType(), SourceTableInfo.SOURCE_SUFFIX, options)); } else if (tableInfo instanceof TargetTableInfo) { - TableSink tableSink = StreamSinkFactory.getTableSink((TargetTableInfo) tableInfo, localSqlPluginPath); + TableSink tableSink = StreamSinkFactory.getTableSink((TargetTableInfo) tableInfo, options.getLocalSqlPluginPath()); TypeInformation[] flinkTypes = FlinkUtil.transformTypes(tableInfo.getFieldClasses()); tableEnv.registerTableSink(tableInfo.getName(), tableInfo.getFields(), flinkTypes, tableSink); - classPathSet.add( PluginUtil.getRemoteJarFilePath(tableInfo.getType(), TargetTableInfo.TARGET_SUFFIX, remoteSqlPluginPath, localSqlPluginPath)); + classPathSet.add(buildSourceAndSinkPathByLoadMode(tableInfo.getType(), TargetTableInfo.TARGET_SUFFIX, options)); } else if(tableInfo instanceof SideTableInfo){ String sideOperator = ECacheType.ALL.name().equals(((SideTableInfo) tableInfo).getCacheType()) ? "all" : "async"; sideTableMap.put(tableInfo.getName(), (SideTableInfo) tableInfo); - classPathSet.add(PluginUtil.getRemoteSideJarFilePath(tableInfo.getType(), sideOperator, SideTableInfo.TARGET_SUFFIX, remoteSqlPluginPath, localSqlPluginPath)); + classPathSet.add(buildSidePathByLoadMode(tableInfo.getType(), sideOperator, SideTableInfo.TARGET_SUFFIX, options)); }else { throw new RuntimeException("not support table type:" + tableInfo.getType()); } @@ -294,6 +293,22 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en } } + private static URL buildSourceAndSinkPathByLoadMode(String type, String suffix, Options options) throws Exception { + String pluginLoadMode = options.getPluginLoadMode(); + if (StringUtils.equalsIgnoreCase(pluginLoadMode, PluginLoadMode.classpath.name())) { + return PluginUtil.getRemoteJarFilePath(type, suffix, options.getRemoteSqlPluginPath(), options.getLocalSqlPluginPath()); + } + return PluginUtil.getLocalJarFilePath(type, suffix, options.getLocalSqlPluginPath()); + } + + private static URL buildSidePathByLoadMode(String type, String operator, String suffix, Options options) throws Exception { + String pluginLoadMode = options.getPluginLoadMode(); + if (StringUtils.equalsIgnoreCase(pluginLoadMode, PluginLoadMode.classpath.name())) { + return PluginUtil.getRemoteSideJarFilePath(type, operator, suffix, options.getRemoteSqlPluginPath(), options.getLocalSqlPluginPath()); + } + return PluginUtil.getLocalSideJarFilePath(type, operator, suffix, options.getLocalSqlPluginPath()); + } + private static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws Exception { confProperties = PropertiesUtils.propertiesTrim(confProperties); diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/PluginLoadMode.java b/core/src/main/java/com/dtstack/flink/sql/enums/PluginLoadMode.java new file mode 100644 index 000000000..5dd6f2f42 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/enums/PluginLoadMode.java @@ -0,0 +1,15 @@ +package com.dtstack.flink.sql.enums; + +public enum PluginLoadMode { + classpath(0),shipfile(1); + + private int type; + + PluginLoadMode(int type){ + this.type = type; + } + + public int getType(){ + return this.type; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/option/Options.java b/core/src/main/java/com/dtstack/flink/sql/option/Options.java index eef54a617..7100a17f2 100644 --- a/core/src/main/java/com/dtstack/flink/sql/option/Options.java +++ b/core/src/main/java/com/dtstack/flink/sql/option/Options.java @@ -19,6 +19,7 @@ package com.dtstack.flink.sql.option; import com.dtstack.flink.sql.enums.ClusterMode; +import com.dtstack.flink.sql.enums.PluginLoadMode; /** @@ -71,6 +72,9 @@ public class Options { @OptionRequired(description = "yarn session configuration,such as yid") private String yarnSessionConf = "{}"; + @OptionRequired(description = "plugin load mode, by classpath or shipfile") + private String pluginLoadMode = PluginLoadMode.classpath.name(); + public String getMode() { return mode; } @@ -182,4 +186,12 @@ public String getYarnSessionConf() { public void setYarnSessionConf(String yarnSessionConf) { this.yarnSessionConf = yarnSessionConf; } + + public String getPluginLoadMode() { + return pluginLoadMode; + } + + public void setPluginLoadMode(String pluginLoadMode) { + this.pluginLoadMode = pluginLoadMode; + } } diff --git a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java index 8cf4c8b1b..96a7f5768 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java @@ -117,6 +117,14 @@ public static URL getRemoteJarFilePath(String pluginType, String tableType, Stri return new URL("file:" + remoteSqlRootDir + SP + dirName + SP + jarName); } + public static URL getLocalJarFilePath(String pluginType, String tableType, String localSqlPluginPath) throws Exception { + String dirName = pluginType + tableType.toLowerCase(); + String prefix = String.format("%s-%s", pluginType, tableType.toLowerCase()); + String jarPath = localSqlPluginPath + SP + dirName; + String jarName = getCoreJarFileName(jarPath, prefix); + return new URL("file:" + jarPath + SP + jarName); + } + public static URL getRemoteSideJarFilePath(String pluginType, String sideOperator, String tableType, String remoteSqlRootDir, String localSqlPluginPath) throws Exception { String dirName = pluginType + sideOperator + tableType.toLowerCase(); String prefix = String.format("%s-%s-%s", pluginType, sideOperator, tableType.toLowerCase()); @@ -125,6 +133,14 @@ public static URL getRemoteSideJarFilePath(String pluginType, String sideOperato return new URL("file:" + remoteSqlRootDir + SP + dirName + SP + jarName); } + public static URL getLocalSideJarFilePath(String pluginType, String sideOperator, String tableType, String localSqlPluginPath) throws Exception { + String dirName = pluginType + sideOperator + tableType.toLowerCase(); + String prefix = String.format("%s-%s-%s", pluginType, sideOperator, tableType.toLowerCase()); + String jarPath = localSqlPluginPath + SP + dirName; + String jarName = getCoreJarFileName(jarPath, prefix); + return new URL("file:" + jarPath + SP + jarName); + } + public static String upperCaseFirstChar(String str){ return str.substring(0, 1).toUpperCase() + str.substring(1); } diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java index 9157d219e..82d56d2dc 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java @@ -19,9 +19,12 @@ package com.dtstack.flink.sql.launcher.perjob; import com.dtstack.flink.sql.launcher.YarnConfLoader; +import com.dtstack.flink.sql.option.Options; import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.configuration.Configuration; import org.apache.flink.hadoop.shaded.com.google.common.base.Strings; +import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; import org.apache.hadoop.fs.Path; @@ -30,8 +33,10 @@ import java.io.File; import java.net.MalformedURLException; +import java.net.URL; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Properties; /** @@ -42,6 +47,7 @@ */ public class PerJobClusterClientBuilder { + private static final String FLINK_PLUGIN_CLASSPATH_LOAD = "classpath"; private YarnClient yarnClient; @@ -60,43 +66,68 @@ public void init(String yarnConfDir){ System.out.println("----init yarn success ----"); } - public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, String flinkJarPath, String queue) throws MalformedURLException { + public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties confProp, String flinkJarPath, Options launcherOptions, JobGraph jobGraph) throws MalformedURLException { Configuration newConf = new Configuration(); - confProp.forEach((key, val) -> newConf.setString(key.toString(), val.toString()) ); + confProp.forEach((key, val) -> newConf.setString(key.toString(), val.toString())); AbstractYarnClusterDescriptor clusterDescriptor = getClusterDescriptor(newConf, yarnConf, "."); if (StringUtils.isNotBlank(flinkJarPath)) { - if (!new File(flinkJarPath).exists()) { throw new RuntimeException("The Flink jar path is not exist"); } - } - List shipFiles = new ArrayList<>(); + List shipFiles = new ArrayList<>(); if (flinkJarPath != null) { File[] jars = new File(flinkJarPath).listFiles(); - - for (File file : jars){ - if (file.toURI().toURL().toString().contains("flink-dist")){ + for (File file : jars) { + if (file.toURI().toURL().toString().contains("flink-dist")) { clusterDescriptor.setLocalJarPath(new Path(file.toURI().toURL().toString())); } else { shipFiles.add(file); } } - } else { throw new RuntimeException("The Flink jar path is null"); } - clusterDescriptor.addShipFiles(shipFiles); + // classpath , all node need contain plugin jar + String pluginLoadMode = launcherOptions.getPluginLoadMode(); + if (StringUtils.equalsIgnoreCase(pluginLoadMode, FLINK_PLUGIN_CLASSPATH_LOAD)){ + fillJobGraphClassPath(jobGraph); + } else { + List pluginPaths = getPluginPathToShipFiles(jobGraph); + shipFiles.addAll(pluginPaths); + } - if(!Strings.isNullOrEmpty(queue)){ + clusterDescriptor.addShipFiles(shipFiles); + String queue = launcherOptions.getQueue(); + if (!Strings.isNullOrEmpty(queue)) { clusterDescriptor.setQueue(queue); } return clusterDescriptor; } + private static void fillJobGraphClassPath(JobGraph jobGraph) throws MalformedURLException { + Map jobCacheFileConfig = jobGraph.getUserArtifacts(); + for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ + if(tmp.getKey().startsWith("class_path")){ + jobGraph.getClasspaths().add(new URL("file:" + tmp.getValue().filePath)); + } + } + } + + private List getPluginPathToShipFiles(JobGraph jobGraph) { + List shipFiles = new ArrayList<>(); + Map jobCacheFileConfig = jobGraph.getUserArtifacts(); + for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ + if(tmp.getKey().startsWith("class_path")){ + shipFiles.add(new File(tmp.getValue().filePath)); + } + } + return shipFiles; + } + private AbstractYarnClusterDescriptor getClusterDescriptor( Configuration configuration, YarnConfiguration yarnConfiguration, diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java index ccc696a54..e4735621a 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java @@ -52,8 +52,6 @@ public class PerJobSubmitter { private static final Logger LOG = LoggerFactory.getLogger(PerJobSubmitter.class); public static String submit(Options launcherOptions, JobGraph jobGraph) throws Exception { - - fillJobGraphClassPath(jobGraph); if (!StringUtils.isBlank(launcherOptions.getAddjar())) { String addjarPath = URLDecoder.decode(launcherOptions.getAddjar(), Charsets.UTF_8.toString()); List paths = getJarPaths(addjarPath); @@ -62,8 +60,6 @@ public static String submit(Options launcherOptions, JobGraph jobGraph) throws E }); } - - String confProp = launcherOptions.getConfProp(); confProp = URLDecoder.decode(confProp, Charsets.UTF_8.toString()); Properties confProperties = PluginUtil.jsonStrToObject(confProp, Properties.class); @@ -74,7 +70,7 @@ public static String submit(Options launcherOptions, JobGraph jobGraph) throws E String flinkJarPath = launcherOptions.getFlinkJarPath(); - AbstractYarnClusterDescriptor yarnClusterDescriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(confProperties, flinkJarPath, launcherOptions.getQueue()); + AbstractYarnClusterDescriptor yarnClusterDescriptor = perJobClusterClientBuilder.createPerJobClusterDescriptor(confProperties, flinkJarPath, launcherOptions, jobGraph); ClusterClient clusterClient = yarnClusterDescriptor.deployJobCluster(clusterSpecification, jobGraph,true); String applicationId = clusterClient.getClusterId().toString(); @@ -95,12 +91,4 @@ private static List getJarPaths(String addjarPath) { return paths; } - private static void fillJobGraphClassPath(JobGraph jobGraph) throws MalformedURLException { - Map jobCacheFileConfig = jobGraph.getUserArtifacts(); - for(Map.Entry tmp : jobCacheFileConfig.entrySet()){ - if(tmp.getKey().startsWith("class_path")){ - jobGraph.getClasspaths().add(new URL("file:" + tmp.getValue().filePath)); - } - } - } } From 8f04280eeb292489da481430e3111184e1c713d3 Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Fri, 1 Nov 2019 11:55:55 +0800 Subject: [PATCH 439/470] =?UTF-8?q?[flinksql][sql=E8=A7=A3=E6=9E=90?= =?UTF-8?q?=E5=A4=B1=E8=B4=A5][19739]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/com/dtstack/flink/sql/util/DtStringUtil.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index ed31618d1..3e7d828f3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -83,12 +83,6 @@ public static List splitIgnoreQuota(String str, char delimiter){ }else if(c == '\'' && '\\'!=flag && !inQuotes){ inSingleQuotes = !inSingleQuotes; b.append(c); - }else if(c == '('){ - bracketLeftNum++; - b.append(c); - }else if(c == ')'){ - bracketLeftNum--; - b.append(c); }else{ b.append(c); } From e54b6eb40fb7bbdec804eed43354337c58c1c983 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 2 Nov 2019 11:58:39 +0000 Subject: [PATCH 440/470] Bump fastjson from 1.2.7 to 1.2.25 in /launcher Bumps [fastjson](https://github.com/alibaba/fastjson) from 1.2.7 to 1.2.25. - [Release notes](https://github.com/alibaba/fastjson/releases) - [Commits](https://github.com/alibaba/fastjson/compare/1.2.7...1.2.25) Signed-off-by: dependabot[bot] --- launcher/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/launcher/pom.xml b/launcher/pom.xml index 2f8140fd5..4c6334411 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -33,7 +33,7 @@ com.alibaba fastjson - 1.2.7 + 1.2.25 From 753e682022819a210b899ec0d8c9724139ad4a68 Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Mon, 4 Nov 2019 11:14:55 +0800 Subject: [PATCH 441/470] =?UTF-8?q?=E5=85=B3=E7=B3=BB=E5=9E=8B=E6=95=B0?= =?UTF-8?q?=E6=8D=AE=E5=BA=93=E8=B0=93=E8=AF=8D=E4=B8=8B=E6=8E=A8?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java index 15d8f2c2b..ed8adb9df 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java @@ -93,8 +93,11 @@ public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { // 如果where条件中第一个符合条件的是维表中的条件 String firstCondition = whereConditionList.get(0); if (!"and".equalsIgnoreCase(firstCondition) && !"or".equalsIgnoreCase(firstCondition)) { - sqlCondition += " and "; + whereConditionList.add(0, "and ("); + } else { + whereConditionList.add(1, "("); } + whereConditionList.add(whereConditionList.size(), ")"); sqlCondition += String.join(" ", whereConditionList); } From beef7723b04b6dda0394d8fc34df308c0b16beb0 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 4 Nov 2019 11:25:18 +0800 Subject: [PATCH 442/470] modify method params --- .../main/java/com/dtstack/flink/sql/Main.java | 38 +++++++++---------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index cd4cd1194..c3ad1a97e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -109,6 +109,8 @@ public static void main(String[] args) throws Exception { String name = options.getName(); String addJarListStr = options.getAddjar(); String localSqlPluginPath = options.getLocalSqlPluginPath(); + String remoteSqlPluginPath = options.getRemoteSqlPluginPath(); + String pluginLoadMode = options.getPluginLoadMode(); String deployMode = options.getMode(); String confProp = options.getConfProp(); @@ -141,9 +143,9 @@ public static void main(String[] args) throws Exception { //register udf registerUDF(sqlTree, jarURList, tableEnv); //register table schema - registerTable(sqlTree, env, tableEnv, options, sideTableMap, registerTableCache); + registerTable(sqlTree, env, tableEnv, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode, sideTableMap, registerTableCache); - sqlTranslation(options,tableEnv,sqlTree,sideTableMap,registerTableCache); + sqlTranslation(localSqlPluginPath, tableEnv,sqlTree,sideTableMap,registerTableCache); if(env instanceof MyLocalStreamEnvironment) { ((MyLocalStreamEnvironment) env).setClasspaths(ClassLoaderManager.getClassPath()); @@ -152,9 +154,9 @@ public static void main(String[] args) throws Exception { env.execute(name); } - private static void sqlTranslation(Options options,StreamTableEnvironment tableEnv,SqlTree sqlTree,Map sideTableMap,Map registerTableCache) throws Exception { + private static void sqlTranslation(String localSqlPluginPath, StreamTableEnvironment tableEnv,SqlTree sqlTree,Map sideTableMap,Map registerTableCache) throws Exception { SideSqlExec sideSqlExec = new SideSqlExec(); - sideSqlExec.setLocalSqlPluginPath(options.getLocalSqlPluginPath()); + sideSqlExec.setLocalSqlPluginPath(localSqlPluginPath); for (CreateTmpTableParser.SqlParserResult result : sqlTree.getTmpSqlList()) { sideSqlExec.registerTmpTable(result, sideTableMap, tableEnv, registerTableCache); } @@ -230,8 +232,8 @@ private static void registerUDF(SqlTree sqlTree, List jarURList, StreamTabl } - private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv, Options options, - Map sideTableMap, Map registerTableCache) throws Exception { + private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv, String localSqlPluginPath, + String remoteSqlPluginPath, String pluginLoadMode, Map sideTableMap, Map registerTableCache) throws Exception { Set classPathSet = Sets.newHashSet(); WaterMarkerAssigner waterMarkerAssigner = new WaterMarkerAssigner(); for (TableInfo tableInfo : sqlTree.getTableInfoMap().values()) { @@ -239,7 +241,7 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en if (tableInfo instanceof SourceTableInfo) { SourceTableInfo sourceTableInfo = (SourceTableInfo) tableInfo; - Table table = StreamSourceFactory.getStreamSource(sourceTableInfo, env, tableEnv, options.getLocalSqlPluginPath()); + Table table = StreamSourceFactory.getStreamSource(sourceTableInfo, env, tableEnv, localSqlPluginPath); tableEnv.registerTable(sourceTableInfo.getAdaptName(), table); //Note --- parameter conversion function can not be used inside a function of the type of polymerization //Create table in which the function is arranged only need adaptation sql @@ -266,18 +268,18 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en LOG.info("registe table {} success.", tableInfo.getName()); } registerTableCache.put(tableInfo.getName(), regTable); - classPathSet.add(buildSourceAndSinkPathByLoadMode(tableInfo.getType(), SourceTableInfo.SOURCE_SUFFIX, options)); + classPathSet.add(buildSourceAndSinkPathByLoadMode(tableInfo.getType(), SourceTableInfo.SOURCE_SUFFIX, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode)); } else if (tableInfo instanceof TargetTableInfo) { - TableSink tableSink = StreamSinkFactory.getTableSink((TargetTableInfo) tableInfo, options.getLocalSqlPluginPath()); + TableSink tableSink = StreamSinkFactory.getTableSink((TargetTableInfo) tableInfo, localSqlPluginPath); TypeInformation[] flinkTypes = FlinkUtil.transformTypes(tableInfo.getFieldClasses()); tableEnv.registerTableSink(tableInfo.getName(), tableInfo.getFields(), flinkTypes, tableSink); - classPathSet.add(buildSourceAndSinkPathByLoadMode(tableInfo.getType(), TargetTableInfo.TARGET_SUFFIX, options)); + classPathSet.add(buildSourceAndSinkPathByLoadMode(tableInfo.getType(), TargetTableInfo.TARGET_SUFFIX, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode)); } else if(tableInfo instanceof SideTableInfo){ String sideOperator = ECacheType.ALL.name().equals(((SideTableInfo) tableInfo).getCacheType()) ? "all" : "async"; sideTableMap.put(tableInfo.getName(), (SideTableInfo) tableInfo); - classPathSet.add(buildSidePathByLoadMode(tableInfo.getType(), sideOperator, SideTableInfo.TARGET_SUFFIX, options)); + classPathSet.add(buildSidePathByLoadMode(tableInfo.getType(), sideOperator, SideTableInfo.TARGET_SUFFIX, localSqlPluginPath, remoteSqlPluginPath, pluginLoadMode)); }else { throw new RuntimeException("not support table type:" + tableInfo.getType()); } @@ -293,20 +295,18 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en } } - private static URL buildSourceAndSinkPathByLoadMode(String type, String suffix, Options options) throws Exception { - String pluginLoadMode = options.getPluginLoadMode(); + private static URL buildSourceAndSinkPathByLoadMode(String type, String suffix, String localSqlPluginPath, String remoteSqlPluginPath, String pluginLoadMode) throws Exception { if (StringUtils.equalsIgnoreCase(pluginLoadMode, PluginLoadMode.classpath.name())) { - return PluginUtil.getRemoteJarFilePath(type, suffix, options.getRemoteSqlPluginPath(), options.getLocalSqlPluginPath()); + return PluginUtil.getRemoteJarFilePath(type, suffix, remoteSqlPluginPath, localSqlPluginPath); } - return PluginUtil.getLocalJarFilePath(type, suffix, options.getLocalSqlPluginPath()); + return PluginUtil.getLocalJarFilePath(type, suffix, localSqlPluginPath); } - private static URL buildSidePathByLoadMode(String type, String operator, String suffix, Options options) throws Exception { - String pluginLoadMode = options.getPluginLoadMode(); + private static URL buildSidePathByLoadMode(String type, String operator, String suffix, String localSqlPluginPath, String remoteSqlPluginPath, String pluginLoadMode) throws Exception { if (StringUtils.equalsIgnoreCase(pluginLoadMode, PluginLoadMode.classpath.name())) { - return PluginUtil.getRemoteSideJarFilePath(type, operator, suffix, options.getRemoteSqlPluginPath(), options.getLocalSqlPluginPath()); + return PluginUtil.getRemoteSideJarFilePath(type, operator, suffix, remoteSqlPluginPath, localSqlPluginPath); } - return PluginUtil.getLocalSideJarFilePath(type, operator, suffix, options.getLocalSqlPluginPath()); + return PluginUtil.getLocalSideJarFilePath(type, operator, suffix, localSqlPluginPath); } private static StreamExecutionEnvironment getStreamExeEnv(Properties confProperties, String deployMode) throws Exception { From f35b450a048259101304be2645ec527b62ac3579 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BF=AE=E7=AB=B9?= Date: Mon, 4 Nov 2019 18:08:43 +0800 Subject: [PATCH 443/470] =?UTF-8?q?kudu=E7=BB=B4=E8=A1=A8=E5=92=8C?= =?UTF-8?q?=E7=BB=93=E6=9E=9C=E8=A1=A8?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- kudu/kudu-side/kudu-all-side/pom.xml | 89 ++++ .../flink/sql/side/kudu/KuduAllReqRow.java | 396 ++++++++++++++++++ .../flink/sql/side/kudu/KuduAllSideInfo.java | 88 ++++ kudu/kudu-side/kudu-async-side/pom.xml | 107 +++++ .../flink/sql/side/kudu/KuduAsyncReqRow.java | 320 ++++++++++++++ .../sql/side/kudu/KuduAsyncSideInfo.java | 99 +++++ kudu/kudu-side/kudu-side-core/pom.xml | 24 ++ .../sql/side/kudu/table/KuduSideParser.java | 74 ++++ .../side/kudu/table/KuduSideTableInfo.java | 155 +++++++ kudu/kudu-side/pom.xml | 23 + kudu/kudu-sink/pom.xml | 81 ++++ .../flink/sql/sink/kudu/KuduOutputFormat.java | 308 ++++++++++++++ .../dtstack/flink/sql/sink/kudu/KuduSink.java | 104 +++++ .../sql/sink/kudu/table/KuduSinkParser.java | 54 +++ .../sql/sink/kudu/table/KuduTableInfo.java | 92 ++++ kudu/pom.xml | 45 ++ pom.xml | 1 + 17 files changed, 2060 insertions(+) create mode 100644 kudu/kudu-side/kudu-all-side/pom.xml create mode 100644 kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllReqRow.java create mode 100644 kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllSideInfo.java create mode 100644 kudu/kudu-side/kudu-async-side/pom.xml create mode 100644 kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncReqRow.java create mode 100644 kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncSideInfo.java create mode 100644 kudu/kudu-side/kudu-side-core/pom.xml create mode 100644 kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideParser.java create mode 100644 kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideTableInfo.java create mode 100644 kudu/kudu-side/pom.xml create mode 100644 kudu/kudu-sink/pom.xml create mode 100644 kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduOutputFormat.java create mode 100644 kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduSink.java create mode 100644 kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduSinkParser.java create mode 100644 kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduTableInfo.java create mode 100644 kudu/pom.xml diff --git a/kudu/kudu-side/kudu-all-side/pom.xml b/kudu/kudu-side/kudu-all-side/pom.xml new file mode 100644 index 000000000..5c35d4cd9 --- /dev/null +++ b/kudu/kudu-side/kudu-all-side/pom.xml @@ -0,0 +1,89 @@ + + + + sql.side.kudu + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.all.kudu + kudu-all-side + + jar + + + + com.dtstack.flink + sql.side.kudu.core + 1.0-SNAPSHOT + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + org.slf4j + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllReqRow.java b/kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllReqRow.java new file mode 100644 index 000000000..681f5faab --- /dev/null +++ b/kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllReqRow.java @@ -0,0 +1,396 @@ +package com.dtstack.flink.sql.side.kudu; + +import com.dtstack.flink.sql.side.AllReqRow; +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.kudu.table.KuduSideTableInfo; +import org.apache.calcite.sql.JoinType; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; +import org.apache.kudu.ColumnSchema; +import org.apache.kudu.Schema; +import org.apache.kudu.Type; +import org.apache.kudu.client.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.SQLException; +import java.sql.Timestamp; +import java.util.*; +import java.util.concurrent.atomic.AtomicReference; + +public class KuduAllReqRow extends AllReqRow { + + private static final long serialVersionUID = 6051774809356082219L; + + private static final Logger LOG = LoggerFactory.getLogger(KuduAllReqRow.class); + /** + * 获取连接的尝试次数 + */ + private static final int CONN_RETRY_NUM = 3; + /** + * 缓存条数 + */ + private static final Long FETCH_SIZE = 1000L; + + private KuduClient client; + + private KuduTable table; + + + private AtomicReference>>> cacheRef = new AtomicReference<>(); + + public KuduAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new KuduAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + + @Override + public Row fillData(Row input, Object sideInput) { + Map cacheInfo = (Map) sideInput; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. + if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { + obj = ((Timestamp) obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for (Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()) { + if (cacheInfo == null) { + row.setField(entry.getKey(), null); + } else { + row.setField(entry.getKey(), cacheInfo.get(entry.getValue())); + } + } + + return row; + } + + @Override + protected void initCache() throws SQLException { + Map>> newCache = Maps.newConcurrentMap(); + cacheRef.set(newCache); + loadData(newCache); + } + + + @Override + protected void reloadCache() { + //reload cacheRef and replace to old cacheRef + Map>> newCache = Maps.newConcurrentMap(); + loadData(newCache); + + cacheRef.set(newCache); + LOG.info("----- Mongo all cacheRef reload end:{}", Calendar.getInstance()); + } + + + @Override + public void flatMap(Row value, Collector out) throws Exception { + List inputParams = Lists.newArrayList(); + for (Integer conValIndex : sideInfo.getEqualValIndex()) { + Object equalObj = value.getField(conValIndex); + if (equalObj == null) { + out.collect(null); + } + inputParams.add(equalObj); + } + + String key = buildKey(inputParams); + List> cacheList = cacheRef.get().get(key); + if (CollectionUtils.isEmpty(cacheList)) { + if (sideInfo.getJoinType() == JoinType.LEFT) { + Row row = fillData(value, null); + out.collect(row); + } + return; + } + + for (Map one : cacheList) { + out.collect(fillData(value, one)); + } + } + + private void loadData(Map>> tmpCache) { + KuduSideTableInfo tableInfo = (KuduSideTableInfo) sideInfo.getSideTableInfo(); + KuduScanner scanner = null; + try { + for (int i = 0; i < CONN_RETRY_NUM; i++) { + try { + scanner = getConn(tableInfo); + break; + } catch (Exception e) { + if (i == CONN_RETRY_NUM - 1) { + throw new RuntimeException("", e); + } + try { + String connInfo = "kuduMasters:" + tableInfo.getKuduMasters() + ";tableName:" + tableInfo.getTableName(); + LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + connInfo); + Thread.sleep(5 * 1000); + } catch (InterruptedException e1) { + e1.printStackTrace(); + } + } + } + //load data from table + assert scanner != null; + String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); + + + while (scanner.hasMoreRows()) { + RowResultIterator results = scanner.nextRows(); + while (results.hasNext()) { + RowResult result = results.next(); + Map oneRow = Maps.newHashMap(); + for (String sideFieldName1 : sideFieldNames) { + String sideFieldName = sideFieldName1.trim(); + ColumnSchema columnSchema = table.getSchema().getColumn(sideFieldName); + if (null != columnSchema) { + setMapValue(columnSchema.getType(), oneRow, sideFieldName, result); + } + } + String cacheKey = buildKey(oneRow, sideInfo.getEqualFieldList()); + List> list = tmpCache.computeIfAbsent(cacheKey, key -> Lists.newArrayList()); + list.add(oneRow); + } + } + + } catch (Exception e) { + LOG.error("", e); + } finally { + if (null != scanner) { + try { + scanner.close(); + } catch (KuduException e) { + LOG.error("Error while closing scanner.", e); + } + } + //放置到close中关闭 每次刷新时间较长则可以选择在这里关闭 +// if (null != client) { +// try { +// client.close(); +// } catch (Exception e) { +// LOG.error("Error while closing client.", e); +// } +// } + + } + + + } + + private String buildKey(List equalValList) { + StringBuilder sb = new StringBuilder(""); + for (Object equalVal : equalValList) { + sb.append(equalVal).append("_"); + } + + return sb.toString(); + } + + private String buildKey(Map val, List equalFieldList) { + StringBuilder sb = new StringBuilder(""); + for (String equalField : equalFieldList) { + sb.append(val.get(equalField)).append("_"); + } + return sb.toString(); + } + + private KuduScanner getConn(KuduSideTableInfo tableInfo) { + try { + if (client == null) { + String kuduMasters = tableInfo.getKuduMasters(); + String tableName = tableInfo.getTableName(); + Integer workerCount = tableInfo.getWorkerCount(); + Integer defaultSocketReadTimeoutMs = tableInfo.getDefaultSocketReadTimeoutMs(); + Integer defaultOperationTimeoutMs = tableInfo.getDefaultOperationTimeoutMs(); + + Preconditions.checkNotNull(kuduMasters, "kuduMasters could not be null"); + + KuduClient.KuduClientBuilder kuduClientBuilder = new KuduClient.KuduClientBuilder(kuduMasters); + if (null != workerCount) { + kuduClientBuilder.workerCount(workerCount); + } + if (null != defaultSocketReadTimeoutMs) { + kuduClientBuilder.defaultSocketReadTimeoutMs(defaultSocketReadTimeoutMs); + } + + if (null != defaultOperationTimeoutMs) { + kuduClientBuilder.defaultOperationTimeoutMs(defaultOperationTimeoutMs); + } + client = kuduClientBuilder.build(); + + if (!client.tableExists(tableName)) { + throw new IllegalArgumentException("Table Open Failed , please check table exists"); + } + table = client.openTable(tableName); + } + Schema schema = table.getSchema(); + LOG.info("connect kudu is successed!"); + KuduScanner.KuduScannerBuilder tokenBuilder = client.newScannerBuilder(table); + return buildScanner(tokenBuilder, schema, tableInfo); + } catch ( + Exception e) { + LOG.error("connect kudu is error:" + e.getMessage()); + } + return null; + } + + + /** + * @param builder 创建AsyncKuduScanner对象 + * @param schema kudu中表约束 + * @param tableInfo AsyncKuduScanner的配置信息 + * @return + */ + private KuduScanner buildScanner(KuduScanner.KuduScannerBuilder builder, Schema schema, KuduSideTableInfo + tableInfo) { + Integer batchSizeBytes = tableInfo.getBatchSizeBytes(); + Long limitNum = tableInfo.getLimitNum(); + Boolean isFaultTolerant = tableInfo.getFaultTolerant(); + //查询需要的字段 + String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); + //主键过滤条件 主键最小值 + String lowerBoundPrimaryKey = tableInfo.getLowerBoundPrimaryKey(); + //主键过滤条件 主键最大值 + String upperBoundPrimaryKey = tableInfo.getUpperBoundPrimaryKey(); + //主键字段 + String primaryKeys = tableInfo.getPrimaryKey(); + if (null == limitNum || limitNum <= 0) { + builder.limit(FETCH_SIZE); + } else { + builder.limit(limitNum); + } + if (null != batchSizeBytes) { + builder.batchSizeBytes(batchSizeBytes); + } + if (null != isFaultTolerant) { + builder.setFaultTolerant(isFaultTolerant); + } + + if (null != lowerBoundPrimaryKey && null != upperBoundPrimaryKey && null != primaryKeys) { + List columnSchemas = schema.getPrimaryKeyColumns(); + Map columnName = new HashMap(columnSchemas.size()); + for (int i = 0; i < columnSchemas.size(); i++) { + columnName.put(columnSchemas.get(i).getName(), i); + } + String[] primaryKey = splitString(primaryKeys); + String[] lowerBounds = splitString(lowerBoundPrimaryKey); + String[] upperBounds = splitString(upperBoundPrimaryKey); + for (int i = 0; i < primaryKey.length; i++) { + Integer index = columnName.get(primaryKey[i]); + if (null != index) { + builder.lowerBound(primaryKeyRange(columnSchemas.get(index).getType(), primaryKey[i], lowerBounds[i], schema)); + builder.exclusiveUpperBound(primaryKeyRange(columnSchemas.get(index).getType(), primaryKey[i], upperBounds[i], schema)); + } + } + } + List projectColumns = Arrays.asList(sideFieldNames); + return builder.setProjectedColumnNames(projectColumns).build(); + } + + private String[] splitString(String data) { + return data.split(","); + } + + private PartialRow primaryKeyRange(Type type, String primaryKey, String value, Schema schema) { + PartialRow partialRow = schema.newPartialRow(); + switch (type) { + case STRING: + partialRow.addString(primaryKey, value); + break; + case FLOAT: + partialRow.addFloat(primaryKey, Float.valueOf(value)); + break; + case INT8: + partialRow.addByte(primaryKey, Byte.valueOf(value)); + break; + case INT16: + partialRow.addShort(primaryKey, Short.valueOf(value)); + break; + case INT32: + partialRow.addInt(primaryKey, Integer.valueOf(value)); + break; + case INT64: + partialRow.addLong(primaryKey, Long.valueOf(value)); + break; + case DOUBLE: + partialRow.addDouble(primaryKey, Double.valueOf(value)); + break; + case BOOL: + partialRow.addBoolean(primaryKey, Boolean.valueOf(value)); + break; + case UNIXTIME_MICROS: + partialRow.addTimestamp(primaryKey, Timestamp.valueOf(value)); + break; + case BINARY: + partialRow.addBinary(primaryKey, value.getBytes()); + break; + default: + throw new IllegalArgumentException("Illegal var type: " + type); + } + return partialRow; + } + + private void setMapValue(Type type, Map oneRow, String sideFieldName, RowResult result) { + switch (type) { + case STRING: + oneRow.put(sideFieldName, result.getString(sideFieldName)); + break; + case FLOAT: + oneRow.put(sideFieldName, result.getFloat(sideFieldName)); + break; + case INT8: + oneRow.put(sideFieldName, result.getFloat(sideFieldName)); + break; + case INT16: + oneRow.put(sideFieldName, result.getShort(sideFieldName)); + break; + case INT32: + oneRow.put(sideFieldName, result.getInt(sideFieldName)); + break; + case INT64: + oneRow.put(sideFieldName, result.getLong(sideFieldName)); + break; + case DOUBLE: + oneRow.put(sideFieldName, result.getDouble(sideFieldName)); + break; + case BOOL: + oneRow.put(sideFieldName, result.getBoolean(sideFieldName)); + break; + case UNIXTIME_MICROS: + oneRow.put(sideFieldName, result.getTimestamp(sideFieldName)); + break; + case BINARY: + oneRow.put(sideFieldName, result.getBinary(sideFieldName)); + break; + default: + throw new IllegalArgumentException("Illegal var type: " + type); + } + } + + @Override + public void close() throws Exception { + //公用一个client 如果每次刷新间隔时间较长可以每次获取一个 + super.close(); + if (null != client) { + try { + client.close(); + } catch (Exception e) { + LOG.error("Error while closing client.", e); + } + } + } +} diff --git a/kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllSideInfo.java b/kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllSideInfo.java new file mode 100644 index 000000000..fd933a38a --- /dev/null +++ b/kudu/kudu-side/kudu-all-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAllSideInfo.java @@ -0,0 +1,88 @@ +package com.dtstack.flink.sql.side.kudu; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.kudu.table.KuduSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.List; + +public class KuduAllSideInfo extends SideInfo { + + private static final long serialVersionUID = 9005389633060174746L; + + public KuduAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + KuduSideTableInfo kuduSideTableInfo = (KuduSideTableInfo) sideTableInfo; + + sqlCondition = "select ${selectField} from ${tableName} "; + sqlCondition = sqlCondition.replace("${tableName}", kuduSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); + System.out.println("---------side_exe_sql-----\n" + sqlCondition); + } + + @Override + public void parseSelectFields(JoinInfo joinInfo) { + String sideTableName = joinInfo.getSideTableName(); + String nonSideTableName = joinInfo.getNonSideTable(); + List fields = Lists.newArrayList(); + + int sideIndex = 0; + for (int i = 0; i < outFieldInfoList.size(); i++) { + FieldInfo fieldInfo = outFieldInfoList.get(i); + if (fieldInfo.getTable().equalsIgnoreCase(sideTableName)) { + fields.add(fieldInfo.getFieldName()); + sideFieldIndex.put(i, sideIndex); + sideFieldNameIndex.put(i, fieldInfo.getFieldName()); + sideIndex++; + } else if (fieldInfo.getTable().equalsIgnoreCase(nonSideTableName)) { + int nonSideIndex = rowTypeInfo.getFieldIndex(fieldInfo.getFieldName()); + inFieldIndex.put(i, nonSideIndex); + } else { + throw new RuntimeException("unknown table " + fieldInfo.getTable()); + } + } + + if (fields.size() == 0) { + throw new RuntimeException("select non field from table " + sideTableName); + } + + //add join on condition field to select fields + SqlNode conditionNode = joinInfo.getCondition(); + + List sqlNodeList = Lists.newArrayList(); + if (conditionNode.getKind() == SqlKind.AND) { + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall) conditionNode).getOperands())); + } else { + sqlNodeList.add(conditionNode); + } + + for (SqlNode sqlNode : sqlNodeList) { + dealOneEqualCon(sqlNode, sideTableName); + } + + if (CollectionUtils.isEmpty(equalFieldList)) { + throw new RuntimeException("no join condition found after table " + joinInfo.getLeftTableName()); + } + + for (String equalField : equalFieldList) { + if (fields.contains(equalField)) { + continue; + } + + fields.add(equalField); + } + + sideSelectFields = String.join(",", fields); + } +} diff --git a/kudu/kudu-side/kudu-async-side/pom.xml b/kudu/kudu-side/kudu-async-side/pom.xml new file mode 100644 index 000000000..6ab098ea9 --- /dev/null +++ b/kudu/kudu-side/kudu-async-side/pom.xml @@ -0,0 +1,107 @@ + + + + sql.side.kudu + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.async.kudu + + kudu-async-side + + jar + + + + + + + io.vertx + vertx-jdbc-client + 3.5.2 + + + + io.vertx + vertx-core + 3.5.2 + + + + + com.dtstack.flink + sql.side.kudu.core + 1.0-SNAPSHOT + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + org.slf4j + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncReqRow.java b/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncReqRow.java new file mode 100644 index 000000000..10fee59f4 --- /dev/null +++ b/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncReqRow.java @@ -0,0 +1,320 @@ +package com.dtstack.flink.sql.side.kudu; + +import com.dtstack.flink.sql.enums.ECacheContentType; +import com.dtstack.flink.sql.side.*; +import com.dtstack.flink.sql.side.cache.CacheObj; +import com.dtstack.flink.sql.side.kudu.table.KuduSideTableInfo; +import com.stumbleupon.async.Callback; +import com.stumbleupon.async.Deferred; +import io.vertx.core.json.JsonArray; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; +import org.apache.kudu.ColumnSchema; +import org.apache.kudu.Schema; +import org.apache.kudu.Type; +import org.apache.kudu.client.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class KuduAsyncReqRow extends AsyncReqRow { + + private static final Logger LOG = LoggerFactory.getLogger(KuduAsyncReqRow.class); + /** + * 获取连接的尝试次数 + */ + private static final int CONN_RETRY_NUM = 3; + /** + * 缓存条数 + */ + private static final Long FETCH_SIZE = 1000L; + + private static final long serialVersionUID = 5028583854989267753L; + + + private AsyncKuduClient asyncClient; + + private KuduTable table; + + private KuduSideTableInfo kuduSideTableInfo; + + private AsyncKuduScanner.AsyncKuduScannerBuilder scannerBuilder; + + public KuduAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new KuduAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + kuduSideTableInfo = (KuduSideTableInfo) sideInfo.getSideTableInfo(); + connKuDu(); + } + + /** + * 连接kudu中的表 + * + * @throws KuduException + */ + private void connKuDu() throws KuduException { + if (null == table) { + String kuduMasters = kuduSideTableInfo.getKuduMasters(); + String tableName = kuduSideTableInfo.getTableName(); + Integer workerCount = kuduSideTableInfo.getWorkerCount(); + Integer defaultSocketReadTimeoutMs = kuduSideTableInfo.getDefaultSocketReadTimeoutMs(); + Integer defaultOperationTimeoutMs = kuduSideTableInfo.getDefaultOperationTimeoutMs(); + + Preconditions.checkNotNull(kuduMasters, "kuduMasters could not be null"); + + AsyncKuduClient.AsyncKuduClientBuilder asyncKuduClientBuilder = new AsyncKuduClient.AsyncKuduClientBuilder(kuduMasters); + if (null != workerCount) { + asyncKuduClientBuilder.workerCount(workerCount); + } + if (null != defaultSocketReadTimeoutMs) { + asyncKuduClientBuilder.defaultSocketReadTimeoutMs(defaultSocketReadTimeoutMs); + } + + if (null != defaultOperationTimeoutMs) { + asyncKuduClientBuilder.defaultOperationTimeoutMs(defaultOperationTimeoutMs); + } + asyncClient = asyncKuduClientBuilder.build(); + if (!asyncClient.syncClient().tableExists(tableName)) { + throw new IllegalArgumentException("Table Open Failed , please check table exists"); + } + table = asyncClient.syncClient().openTable(tableName); + LOG.info("connect kudu is successed!"); + } + scannerBuilder = asyncClient.newScannerBuilder(table); + Integer batchSizeBytes = kuduSideTableInfo.getBatchSizeBytes(); + Long limitNum = kuduSideTableInfo.getLimitNum(); + Boolean isFaultTolerant = kuduSideTableInfo.getFaultTolerant(); + //查询需要的字段 + String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); + + if (null == limitNum || limitNum <= 0) { + scannerBuilder.limit(FETCH_SIZE); + } else { + scannerBuilder.limit(limitNum); + } + if (null != batchSizeBytes) { + scannerBuilder.batchSizeBytes(batchSizeBytes); + } + if (null != isFaultTolerant) { + scannerBuilder.setFaultTolerant(isFaultTolerant); + } + + List projectColumns = Arrays.asList(sideFieldNames); + scannerBuilder.setProjectedColumnNames(projectColumns); + } + + + @Override + public void asyncInvoke(Row input, ResultFuture resultFuture) throws Exception { + //scannerBuilder 设置为null重新加载过滤条件 + scannerBuilder = null; + connKuDu(); + JsonArray inputParams = new JsonArray(); + Schema schema = table.getSchema(); + + for (Integer conValIndex : sideInfo.getEqualValIndex()) { + Object equalObj = input.getField(conValIndex); + if (equalObj == null) { + resultFuture.complete(null); + return; + } + //增加过滤条件 + scannerBuilder.addPredicate(KuduPredicate.newInListPredicate(schema.getColumn(sideInfo.getEqualFieldList().get(conValIndex)), Collections.singletonList(equalObj))); + inputParams.add(equalObj); + } + + String key = buildCacheKey(inputParams); + + if (openCache()) { + //判断数据是否已经加载到缓存中 + CacheObj val = getFromCache(key); + if (val != null) { + + if (ECacheContentType.MissVal == val.getType()) { + dealMissKey(input, resultFuture); + return; + } else if (ECacheContentType.SingleLine == val.getType()) { + Row row = fillData(input, val); + resultFuture.complete(Collections.singleton(row)); + } else if (ECacheContentType.MultiLine == val.getType()) { + List rowList = Lists.newArrayList(); + for (Object jsonArray : (List) val.getContent()) { + Row row = fillData(input, jsonArray); + rowList.add(row); + } + resultFuture.complete(rowList); + } else { + throw new RuntimeException("not support cache obj type " + val.getType()); + } + return; + } + } + List> cacheContent = Lists.newArrayList(); + AsyncKuduScanner asyncKuduScanner = scannerBuilder.build(); + List rowList = Lists.newArrayList(); + Deferred data = asyncKuduScanner.nextRows(); + //从之前的同步修改为调用异步的Callback + data.addCallbackDeferring(new GetListRowCB(input, cacheContent, rowList, asyncKuduScanner, resultFuture, key)); + } + + + @Override + public Row fillData(Row input, Object sideInput) { + Map cacheInfo = (Map) sideInput; + Row row = new Row(sideInfo.getOutFieldInfoList().size()); + for (Map.Entry entry : sideInfo.getInFieldIndex().entrySet()) { + Object obj = input.getField(entry.getValue()); + boolean isTimeIndicatorTypeInfo = TimeIndicatorTypeInfo.class.isAssignableFrom(sideInfo.getRowTypeInfo().getTypeAt(entry.getValue()).getClass()); + + //Type information for indicating event or processing time. However, it behaves like a regular SQL timestamp but is serialized as Long. + if (obj instanceof Timestamp && isTimeIndicatorTypeInfo) { + obj = ((Timestamp) obj).getTime(); + } + row.setField(entry.getKey(), obj); + } + + for (Map.Entry entry : sideInfo.getSideFieldNameIndex().entrySet()) { + if (cacheInfo == null) { + row.setField(entry.getKey(), null); + } else { + row.setField(entry.getKey(), cacheInfo.get(entry.getValue())); + } + } + + return row; + } + + public String buildCacheKey(JsonArray jsonArray) { + StringBuilder sb = new StringBuilder(); + for (Object ele : jsonArray.getList()) { + sb.append(ele.toString()) + .append("_"); + } + + return sb.toString(); + } + + @Override + public void close() throws Exception { + super.close(); + if (null != asyncClient) { + try { + asyncClient.close(); + } catch (Exception e) { + LOG.error("Error while closing client.", e); + } + } + } + + private void setMapValue(Type type, Map oneRow, String sideFieldName, RowResult result) { + switch (type) { + case STRING: + oneRow.put(sideFieldName, result.getString(sideFieldName)); + break; + case FLOAT: + oneRow.put(sideFieldName, result.getFloat(sideFieldName)); + break; + case INT8: + oneRow.put(sideFieldName, result.getFloat(sideFieldName)); + break; + case INT16: + oneRow.put(sideFieldName, result.getShort(sideFieldName)); + break; + case INT32: + oneRow.put(sideFieldName, result.getInt(sideFieldName)); + break; + case INT64: + oneRow.put(sideFieldName, result.getLong(sideFieldName)); + break; + case DOUBLE: + oneRow.put(sideFieldName, result.getDouble(sideFieldName)); + break; + case BOOL: + oneRow.put(sideFieldName, result.getBoolean(sideFieldName)); + break; + case UNIXTIME_MICROS: + oneRow.put(sideFieldName, result.getTimestamp(sideFieldName)); + break; + case BINARY: + oneRow.put(sideFieldName, result.getBinary(sideFieldName)); + break; + default: + throw new IllegalArgumentException("Illegal var type: " + type); + } + } + + class GetListRowCB implements Callback>, RowResultIterator> { + private Row input; + private List> cacheContent; + private List rowList; + private AsyncKuduScanner asyncKuduScanner; + private ResultFuture resultFuture; + private String key; + + + public GetListRowCB() { + } + + GetListRowCB(Row input, List> cacheContent, List rowList, AsyncKuduScanner asyncKuduScanner, ResultFuture resultFuture, String key) { + this.input = input; + this.cacheContent = cacheContent; + this.rowList = rowList; + this.asyncKuduScanner = asyncKuduScanner; + this.resultFuture = resultFuture; + this.key = key; + } + + @Override + public Deferred> call(RowResultIterator results) throws Exception { + for (RowResult result : results) { + Map oneRow = Maps.newHashMap(); + for (String sideFieldName1 : sideInfo.getSideSelectFields().split(",")) { + String sideFieldName = sideFieldName1.trim(); + ColumnSchema columnSchema = table.getSchema().getColumn(sideFieldName); + if (null != columnSchema) { + setMapValue(columnSchema.getType(), oneRow, sideFieldName, result); + } + } + Row row = fillData(input, oneRow); + if (openCache()) { + cacheContent.add(oneRow); + } + rowList.add(row); + } + if (asyncKuduScanner.hasMoreRows()) { + return asyncKuduScanner.nextRows().addCallbackDeferring(this); + } + + if (rowList.size() > 0) { + if (openCache()) { + putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); + } + resultFuture.complete(rowList); + } else { + dealMissKey(input, resultFuture); + if (openCache()) { + //放置在putCache的Miss中 一段时间内同一个key都会直接返回 + putCache(key, CacheMissVal.getMissKeyObj()); + } + } + + return null; + } + } + +} diff --git a/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncSideInfo.java b/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncSideInfo.java new file mode 100644 index 000000000..f341f9a8a --- /dev/null +++ b/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncSideInfo.java @@ -0,0 +1,99 @@ +package com.dtstack.flink.sql.side.kudu; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.kudu.table.KuduSideTableInfo; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; + +import java.util.List; + +public class KuduAsyncSideInfo extends SideInfo { + + + public KuduAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } + + @Override + public void buildEqualInfo(JoinInfo joinInfo, SideTableInfo sideTableInfo) { + KuduSideTableInfo kuduSideTableInfo = (KuduSideTableInfo) sideTableInfo; + + String sideTableName = joinInfo.getSideTableName(); + + SqlNode conditionNode = joinInfo.getCondition(); + + List sqlNodeList = Lists.newArrayList(); + if (conditionNode.getKind() == SqlKind.AND) { + sqlNodeList.addAll(Lists.newArrayList(((SqlBasicCall) conditionNode).getOperands())); + } else { + sqlNodeList.add(conditionNode); + } + + for (SqlNode sqlNode : sqlNodeList) { + dealOneEqualCon(sqlNode, sideTableName); + } + + sqlCondition = "select ${selectField} from ${tableName} "; + sqlCondition = sqlCondition.replace("${tableName}", kuduSideTableInfo.getTableName()).replace("${selectField}", sideSelectFields); + System.out.println("---------side_exe_sql-----\n" + sqlCondition); + } + + @Override + public void dealOneEqualCon(SqlNode sqlNode, String sideTableName) { + if (sqlNode.getKind() != SqlKind.EQUALS) { + throw new RuntimeException("not equal operator."); + } + + SqlIdentifier left = (SqlIdentifier) ((SqlBasicCall) sqlNode).getOperands()[0]; + SqlIdentifier right = (SqlIdentifier) ((SqlBasicCall) sqlNode).getOperands()[1]; + + String leftTableName = left.getComponent(0).getSimple(); + String leftField = left.getComponent(1).getSimple(); + + String rightTableName = right.getComponent(0).getSimple(); + String rightField = right.getComponent(1).getSimple(); + + if (leftTableName.equalsIgnoreCase(sideTableName)) { + equalFieldList.add(leftField); + int equalFieldIndex = -1; + for (int i = 0; i < rowTypeInfo.getFieldNames().length; i++) { + String fieldName = rowTypeInfo.getFieldNames()[i]; + if (fieldName.equalsIgnoreCase(rightField)) { + equalFieldIndex = i; + } + } + if (equalFieldIndex == -1) { + throw new RuntimeException("can't deal equal field: " + sqlNode); + } + + equalValIndex.add(equalFieldIndex); + + } else if (rightTableName.equalsIgnoreCase(sideTableName)) { + + equalFieldList.add(rightField); + int equalFieldIndex = -1; + for (int i = 0; i < rowTypeInfo.getFieldNames().length; i++) { + String fieldName = rowTypeInfo.getFieldNames()[i]; + if (fieldName.equalsIgnoreCase(leftField)) { + equalFieldIndex = i; + } + } + if (equalFieldIndex == -1) { + throw new RuntimeException("can't deal equal field: " + sqlNode.toString()); + } + + equalValIndex.add(equalFieldIndex); + + } else { + throw new RuntimeException("resolve equalFieldList error:" + sqlNode.toString()); + } + + } +} diff --git a/kudu/kudu-side/kudu-side-core/pom.xml b/kudu/kudu-side/kudu-side-core/pom.xml new file mode 100644 index 000000000..3d0a28ac4 --- /dev/null +++ b/kudu/kudu-side/kudu-side-core/pom.xml @@ -0,0 +1,24 @@ + + + + sql.side.kudu + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.kudu.core + + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + jar + \ No newline at end of file diff --git a/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideParser.java b/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideParser.java new file mode 100644 index 000000000..ea451286c --- /dev/null +++ b/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideParser.java @@ -0,0 +1,74 @@ +package com.dtstack.flink.sql.side.kudu.table; + +import com.dtstack.flink.sql.table.AbsSideTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; + +public class KuduSideParser extends AbsSideTableParser { + + + public static final String KUDU_MASTERS = "kuduMasters"; + + public static final String TABLE_NAME = "tableName"; + + public static final String WORKER_COUNT = "workerCount"; + + public static final String OPERATION_TIMEOUT_MS = "defaultOperationTimeoutMs"; + + public static final String SOCKET_READ_TIMEOUT_MS = "defaultSocketReadTimeoutMs"; + + /** + * 查询返回的最大字节数 + */ + public static final String BATCH_SIZE_BYTES = "batchSizeBytes"; + /** + * 查询返回数据条数 + */ + public static final String LIMIT_NUM = "limitNum"; + + /** + * 查询是否容错 查询失败是否扫描第二个副本 默认false 容错 + */ + public static final String IS_FAULT_TO_LERANT = "isFaultTolerant"; + /** + * 需要过滤的主键 + */ + public static final String PRIMARY_KEY = "primaryKey"; + /** + * 过滤主键的最小值 + */ + public static final String LOWER_BOUND_PRIMARY_KEY = "lowerBoundPrimaryKey"; + /** + * 过滤主键的最大值 不包含 + */ + public static final String UPPER_BOUND_PRIMARY_KEY = "upperBoundPrimaryKey"; + + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + KuduSideTableInfo kuduSideTableInfo = new KuduSideTableInfo(); + kuduSideTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kuduSideTableInfo); + + parseCacheProp(kuduSideTableInfo, props); + + kuduSideTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase()))); + kuduSideTableInfo.setKuduMasters(MathUtil.getString(props.get(KUDU_MASTERS.toLowerCase()))); + kuduSideTableInfo.setTableName(MathUtil.getString(props.get(TABLE_NAME.toLowerCase()))); + kuduSideTableInfo.setWorkerCount(MathUtil.getIntegerVal(props.get(WORKER_COUNT.toLowerCase()))); + kuduSideTableInfo.setDefaultOperationTimeoutMs(MathUtil.getIntegerVal(props.get(OPERATION_TIMEOUT_MS.toLowerCase()))); + kuduSideTableInfo.setDefaultSocketReadTimeoutMs(MathUtil.getIntegerVal(props.get(SOCKET_READ_TIMEOUT_MS.toLowerCase()))); + kuduSideTableInfo.setBatchSizeBytes(MathUtil.getIntegerVal(props.get(BATCH_SIZE_BYTES.toLowerCase()))); + kuduSideTableInfo.setLimitNum(MathUtil.getLongVal(props.get(LIMIT_NUM.toLowerCase()))); + kuduSideTableInfo.setFaultTolerant(MathUtil.getBoolean(props.get(IS_FAULT_TO_LERANT.toLowerCase()))); + kuduSideTableInfo.setPrimaryKey(MathUtil.getString(props.get(PRIMARY_KEY.toLowerCase()))); + kuduSideTableInfo.setLowerBoundPrimaryKey(MathUtil.getString(props.get(LOWER_BOUND_PRIMARY_KEY.toLowerCase()))); + kuduSideTableInfo.setUpperBoundPrimaryKey(MathUtil.getString(props.get(UPPER_BOUND_PRIMARY_KEY.toLowerCase()))); + return kuduSideTableInfo; + + } +} diff --git a/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideTableInfo.java b/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideTableInfo.java new file mode 100644 index 000000000..478140433 --- /dev/null +++ b/kudu/kudu-side/kudu-side-core/src/main/java/com/dtstack/flink/sql/side/kudu/table/KuduSideTableInfo.java @@ -0,0 +1,155 @@ +package com.dtstack.flink.sql.side.kudu.table; + +import com.dtstack.flink.sql.side.SideTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +public class KuduSideTableInfo extends SideTableInfo { + + private static final String CURR_TYPE = "kudu"; + + private static final long serialVersionUID = 1085582743577521861L; + + private String kuduMasters; + + private String tableName; + + private Integer workerCount; + + private Integer defaultOperationTimeoutMs; + + private Integer defaultSocketReadTimeoutMs; + /** + * 查询返回的最大字节数 + */ + private Integer batchSizeBytes; + + /** + * 查询返回数据条数 + */ + private Long limitNum; + /** + * 查询是否容错 查询失败是否扫描第二个副本 默认false 容错 + */ + private Boolean isFaultTolerant; + + /** + * 需要过滤的主键 + */ + private String primaryKey; + /** + * 过滤主键的最小值 + */ + private String lowerBoundPrimaryKey; + /** + * 过滤主键的最大值 不包含 + */ + private String upperBoundPrimaryKey; + + + public KuduSideTableInfo() { + setType(CURR_TYPE); + } + + + public String getKuduMasters() { + return kuduMasters; + } + + public void setKuduMasters(String kuduMasters) { + this.kuduMasters = kuduMasters; + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public Integer getWorkerCount() { + return workerCount; + } + + public void setWorkerCount(Integer workerCount) { + this.workerCount = workerCount; + } + + public Integer getDefaultOperationTimeoutMs() { + return defaultOperationTimeoutMs; + } + + public void setDefaultOperationTimeoutMs(Integer defaultOperationTimeoutMs) { + this.defaultOperationTimeoutMs = defaultOperationTimeoutMs; + } + + public Integer getDefaultSocketReadTimeoutMs() { + return defaultSocketReadTimeoutMs; + } + + public void setDefaultSocketReadTimeoutMs(Integer defaultSocketReadTimeoutMs) { + this.defaultSocketReadTimeoutMs = defaultSocketReadTimeoutMs; + } + + public Integer getBatchSizeBytes() { + return batchSizeBytes; + } + + public void setBatchSizeBytes(Integer batchSizeBytes) { + this.batchSizeBytes = batchSizeBytes; + } + + public Long getLimitNum() { + return limitNum; + } + + public void setLimitNum(Long limitNum) { + this.limitNum = limitNum; + } + + public Boolean getFaultTolerant() { + return isFaultTolerant; + } + + public void setFaultTolerant(Boolean faultTolerant) { + isFaultTolerant = faultTolerant; + } + + public String getLowerBoundPrimaryKey() { + return lowerBoundPrimaryKey; + } + + public void setLowerBoundPrimaryKey(String lowerBoundPrimaryKey) { + this.lowerBoundPrimaryKey = lowerBoundPrimaryKey; + } + + public String getUpperBoundPrimaryKey() { + return upperBoundPrimaryKey; + } + + public void setUpperBoundPrimaryKey(String upperBoundPrimaryKey) { + this.upperBoundPrimaryKey = upperBoundPrimaryKey; + } + + public String getPrimaryKey() { + return primaryKey; + } + + public void setPrimaryKey(String primaryKey) { + this.primaryKey = primaryKey; + } + + + @Override + public boolean check() { + Preconditions.checkNotNull(kuduMasters, "Cassandra field of kuduMasters is required"); + Preconditions.checkNotNull(tableName, "Cassandra field of tableName is required"); + return true; + } + + @Override + public String getType() { + // return super.getType().toLowerCase() + TARGET_SUFFIX; + return super.getType().toLowerCase(); + } +} diff --git a/kudu/kudu-side/pom.xml b/kudu/kudu-side/pom.xml new file mode 100644 index 000000000..db3ea5965 --- /dev/null +++ b/kudu/kudu-side/pom.xml @@ -0,0 +1,23 @@ + + + + sql.kudu + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.kudu + kudu-side + + kudu-side-core + kudu-all-side + kudu-async-side + + + + pom + \ No newline at end of file diff --git a/kudu/kudu-sink/pom.xml b/kudu/kudu-sink/pom.xml new file mode 100644 index 000000000..9c1bb2a6d --- /dev/null +++ b/kudu/kudu-sink/pom.xml @@ -0,0 +1,81 @@ + + + + sql.kudu + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.kudu + jar + + + kudu-sink + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + org.slf4j + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduOutputFormat.java b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduOutputFormat.java new file mode 100644 index 000000000..05259ee9b --- /dev/null +++ b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduOutputFormat.java @@ -0,0 +1,308 @@ +/* + * 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 com.dtstack.flink.sql.sink.kudu; + +import com.dtstack.flink.sql.sink.MetricOutputFormat; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.types.Row; +import org.apache.kudu.client.*; + +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.util.Date; + +public class KuduOutputFormat extends MetricOutputFormat { + + public enum WriteMode {INSERT, UPDATE, UPSERT} + +// public enum Consistency {EVENTUAL, STRONG} + + private String kuduMasters; + + private String tableName; + + private WriteMode writeMode; + + protected String[] fieldNames; + + TypeInformation[] fieldTypes; + +// protected List primaryKeys; + +// private Consistency consistency = Consistency.STRONG; + + private AsyncKuduClient client; + + private KuduTable table; + + + private Integer workerCount; + + private Integer defaultOperationTimeoutMs; + + private Integer defaultSocketReadTimeoutMs; + + + private KuduOutputFormat() { + } + + @Override + public void configure(Configuration parameters) { + + } + + @Override + public void open(int taskNumber, int numTasks) throws IOException { + establishConnection(); + initMetric(); + } + + + private void establishConnection() throws KuduException { + AsyncKuduClient.AsyncKuduClientBuilder asyncKuduClientBuilder = new AsyncKuduClient.AsyncKuduClientBuilder(kuduMasters); + if (null != workerCount) { + asyncKuduClientBuilder.workerCount(workerCount); + } + if (null != defaultSocketReadTimeoutMs) { + asyncKuduClientBuilder.workerCount(defaultSocketReadTimeoutMs); + } + + if (null != defaultOperationTimeoutMs) { + asyncKuduClientBuilder.workerCount(defaultOperationTimeoutMs); + } + client = asyncKuduClientBuilder.build(); + KuduClient syncClient = client.syncClient(); + + if (syncClient.tableExists(tableName)) { + table = syncClient.openTable(tableName); + } + } + + @Override + public void writeRecord(Tuple2 record) throws IOException { + Tuple2 tupleTrans = record; + Boolean retract = tupleTrans.getField(0); + if (!retract) { + return; + } + Row row = tupleTrans.getField(1); + if (row.getArity() != fieldNames.length) { + return; + } + + Operation operation = toOperation(writeMode, row); + AsyncKuduSession session = client.newSession(); + session.apply(operation); + session.close(); + outRecords.inc(); + } + + @Override + public void close() { + if (null != client) { + try { + client.close(); + } catch (Exception e) { + throw new IllegalArgumentException("[closeKudu]:" + e.getMessage()); + } + } + } + + public static KuduOutputFormatBuilder buildKuduOutputFormat() { + return new KuduOutputFormatBuilder(); + } + + public static class KuduOutputFormatBuilder { + private final KuduOutputFormat kuduOutputFormat; + + protected KuduOutputFormatBuilder() { + this.kuduOutputFormat = new KuduOutputFormat(); + } + + public KuduOutputFormatBuilder setKuduMasters(String kuduMasters) { + kuduOutputFormat.kuduMasters = kuduMasters; + return this; + } + + public KuduOutputFormatBuilder setTableName(String tableName) { + kuduOutputFormat.tableName = tableName; + return this; + } + + + public KuduOutputFormatBuilder setFieldNames(String[] fieldNames) { + kuduOutputFormat.fieldNames = fieldNames; + return this; + } + + public KuduOutputFormatBuilder setFieldTypes(TypeInformation[] fieldTypes) { + kuduOutputFormat.fieldTypes = fieldTypes; + return this; + } +// +// public KuduOutputFormatBuilder setPrimaryKeys(List primaryKeys) { +// kuduOutputFormat.primaryKeys = primaryKeys; +// return this; +// } + + public KuduOutputFormatBuilder setWriteMode(WriteMode writeMode) { + if (null == writeMode) { + kuduOutputFormat.writeMode = WriteMode.UPSERT; + } + kuduOutputFormat.writeMode = writeMode; + return this; + } + + public KuduOutputFormatBuilder setWorkerCount(Integer workerCount) { + kuduOutputFormat.workerCount = workerCount; + return this; + } + +// public KuduOutputFormatBuilder setConsistency(String consistency) { +// switch (consistency) { +// case "EVENTUAL": +// kuduOutputFormat.consistency = Consistency.EVENTUAL; +// break; +// case "STRONG": +// kuduOutputFormat.consistency = Consistency.STRONG; +// break; +// default: +// kuduOutputFormat.consistency = Consistency.STRONG; +// } +// return this; +// } + + + public KuduOutputFormatBuilder setDefaultOperationTimeoutMs(Integer defaultOperationTimeoutMs) { + kuduOutputFormat.defaultOperationTimeoutMs = defaultOperationTimeoutMs; + return this; + } + + public KuduOutputFormatBuilder setDefaultSocketReadTimeoutMs(Integer defaultSocketReadTimeoutMs) { + kuduOutputFormat.defaultSocketReadTimeoutMs = defaultSocketReadTimeoutMs; + return this; + } + + + public KuduOutputFormat finish() { + if (kuduOutputFormat.kuduMasters == null) { + throw new IllegalArgumentException("No kuduMasters supplied."); + } + + if (kuduOutputFormat.tableName == null) { + throw new IllegalArgumentException("No tablename supplied."); + } + + return kuduOutputFormat; + } + } + + private Operation toOperation(WriteMode writeMode, Row row) { + if (null == table) { + throw new IllegalArgumentException("Table Open Failed , please check table exists"); + } + Operation operation = toOperation(writeMode); + PartialRow partialRow = operation.getRow(); + + for (int index = 0; index < row.getArity(); index++) { + //解决kudu中全小写字段找不到的bug + String fieldName = fieldNames[index].toLowerCase(); + if (row.getField(index) == null) { + partialRow.setNull(fieldName); + } else { + if (fieldTypes[index].getTypeClass() == String.class) { + partialRow.addString(fieldName, (String) row.getField(index)); + continue; + } + if (fieldTypes[index].getTypeClass() == Float.class) { + partialRow.addFloat(fieldName, (Float) row.getField(index)); + continue; + } + if (fieldTypes[index].getTypeClass() == Byte.class) { + partialRow.addByte(fieldName, (Byte) row.getField(index)); + continue; + } + + if (fieldTypes[index].getTypeClass() == Short.class) { + partialRow.addShort(fieldName, (Short) row.getField(index)); + continue; + } + + if (fieldTypes[index].getTypeClass() == Integer.class) { + partialRow.addInt(fieldName, (Integer) row.getField(index)); + continue; + } + + if (fieldTypes[index].getTypeClass() == Long.class) { + partialRow.addLong(fieldName, (Long) row.getField(index)); + continue; + } + + if (fieldTypes[index].getTypeClass() == Double.class) { + partialRow.addDouble(fieldName, (Double) row.getField(index)); + continue; + } + + if (fieldTypes[index].getTypeClass() == BigDecimal.class) { + partialRow.addDecimal(fieldName, (BigDecimal) row.getField(index)); + continue; + } + if (fieldTypes[index].getTypeClass() == Boolean.class) { + partialRow.addBoolean(fieldName, (Boolean) row.getField(index)); + continue; + } + + if (fieldTypes[index].getTypeClass() == Date.class) { + partialRow.addTimestamp(fieldName, new Timestamp(((Date) row.getField(index)).getTime())); + continue; + } + + if (fieldTypes[index].getTypeClass() == Timestamp.class) { + partialRow.addTimestamp(fieldName, (Timestamp) row.getField(index)); + continue; + } + + if (fieldTypes[index].getTypeClass() == byte[].class) { + partialRow.addBinary(fieldName, (byte[]) row.getField(index)); + continue; + } + throw new IllegalArgumentException("Illegal var type: " + fieldTypes[index]); + } + } + return operation; + + } + + private Operation toOperation(WriteMode writeMode) { + switch (writeMode) { + case INSERT: + return table.newInsert(); + case UPDATE: + return table.newUpdate(); + case UPSERT: + return table.newUpsert(); + default: + return table.newUpsert(); + } + } + +} \ No newline at end of file diff --git a/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduSink.java b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduSink.java new file mode 100644 index 000000000..24dff85ae --- /dev/null +++ b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/KuduSink.java @@ -0,0 +1,104 @@ +package com.dtstack.flink.sql.sink.kudu; + + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.kudu.table.KuduTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.OutputFormatSinkFunction; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.table.sinks.RetractStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +import java.io.Serializable; + +public class KuduSink implements RetractStreamTableSink, Serializable, IStreamSinkGener { + + private String kuduMasters; + + private String tableName; + + private KuduOutputFormat.WriteMode writeMode; + + protected String[] fieldNames; + + TypeInformation[] fieldTypes; + +// protected List primaryKeys; + +// private KuduOutputFormat.Consistency consistency = KuduOutputFormat.Consistency.STRONG; + + + private Integer workerCount; + + private Integer defaultOperationTimeoutMs; + + private Integer defaultSocketReadTimeoutMs; + + private int parallelism = -1; + + @Override + public KuduSink genStreamSink(TargetTableInfo targetTableInfo) { + KuduTableInfo kuduTableInfo = (KuduTableInfo) targetTableInfo; + this.kuduMasters = kuduTableInfo.getKuduMasters(); + this.tableName = kuduTableInfo.getTableName(); + this.defaultOperationTimeoutMs = kuduTableInfo.getDefaultOperationTimeoutMs(); + this.defaultSocketReadTimeoutMs = kuduTableInfo.getDefaultSocketReadTimeoutMs(); + this.workerCount = kuduTableInfo.getWorkerCount(); + this.writeMode = kuduTableInfo.getWriteMode(); + + return this; + } + + @Override + public void emitDataStream(DataStream> dataStream) { + KuduOutputFormat.KuduOutputFormatBuilder builder = KuduOutputFormat.buildKuduOutputFormat(); + builder.setKuduMasters(this.kuduMasters) + .setTableName(this.tableName) + .setWriteMode(writeMode) + .setWorkerCount(this.workerCount) + .setDefaultOperationTimeoutMs(this.defaultOperationTimeoutMs) + .setDefaultSocketReadTimeoutMs(this.defaultSocketReadTimeoutMs) + .setFieldNames(this.fieldNames) + .setFieldTypes(this.fieldTypes); + KuduOutputFormat kuduOutputFormat = builder.finish(); + RichSinkFunction richSinkFunction = new OutputFormatSinkFunction(kuduOutputFormat); + dataStream.addSink(richSinkFunction); + } + + @Override + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } + + + @Override + public TupleTypeInfo> getOutputType() { + return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), getRecordType()); + } + + @Override + public TypeInformation getRecordType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + +} diff --git a/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduSinkParser.java b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduSinkParser.java new file mode 100644 index 000000000..f304c14b5 --- /dev/null +++ b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduSinkParser.java @@ -0,0 +1,54 @@ +package com.dtstack.flink.sql.sink.kudu.table; + +import com.dtstack.flink.sql.sink.kudu.KuduOutputFormat; +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +import static com.dtstack.flink.sql.table.TableInfo.PARALLELISM_KEY; + +public class KuduSinkParser extends AbsTableParser { + + public static final String KUDU_MASTERS = "kuduMasters"; + + public static final String TABLE_NAME = "tableName"; + + public static final String WRITE_MODE = "writeMode"; + + public static final String WORKER_COUNT = "workerCount"; + + public static final String OPERATION_TIMEOUT_MS = "defaultOperationTimeoutMs"; + + public static final String SOCKET_READ_TIMEOUT_MS = "defaultSocketReadTimeoutMs"; + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + KuduTableInfo kuduTableInfo = new KuduTableInfo(); + kuduTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kuduTableInfo); + + kuduTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PARALLELISM_KEY.toLowerCase()))); + kuduTableInfo.setKuduMasters(MathUtil.getString(props.get(KUDU_MASTERS.toLowerCase()))); + kuduTableInfo.setTableName(MathUtil.getString(props.get(TABLE_NAME.toLowerCase()))); + kuduTableInfo.setWriteMode(transWriteMode(MathUtil.getString(props.get(WRITE_MODE.toLowerCase())))); + kuduTableInfo.setWorkerCount(MathUtil.getIntegerVal(props.get(WORKER_COUNT.toLowerCase()))); + kuduTableInfo.setDefaultOperationTimeoutMs(MathUtil.getIntegerVal(props.get(OPERATION_TIMEOUT_MS.toLowerCase()))); + kuduTableInfo.setDefaultSocketReadTimeoutMs(MathUtil.getIntegerVal(props.get(SOCKET_READ_TIMEOUT_MS.toLowerCase()))); + return kuduTableInfo; + } + + private KuduOutputFormat.WriteMode transWriteMode(String writeMode) { + switch (writeMode) { + case "insert": + return KuduOutputFormat.WriteMode.INSERT; + case "update": + return KuduOutputFormat.WriteMode.UPDATE; + case "upsert": + return KuduOutputFormat.WriteMode.UPSERT; + default: + return KuduOutputFormat.WriteMode.UPSERT; + } + } +} diff --git a/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduTableInfo.java b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduTableInfo.java new file mode 100644 index 000000000..ed0b956a7 --- /dev/null +++ b/kudu/kudu-sink/src/main/java/com/dtstack/flink/sql/sink/kudu/table/KuduTableInfo.java @@ -0,0 +1,92 @@ +package com.dtstack.flink.sql.sink.kudu.table; + +import com.dtstack.flink.sql.sink.kudu.KuduOutputFormat; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +public class KuduTableInfo extends TargetTableInfo { + + private static final String CURR_TYPE = "kudu"; + + private String kuduMasters; + + private String tableName; + + private KuduOutputFormat.WriteMode writeMode; + + +// private KuduOutputFormat.Consistency consistency = KuduOutputFormat.Consistency.STRONG; + + + private Integer workerCount; + + private Integer defaultOperationTimeoutMs; + + private Integer defaultSocketReadTimeoutMs; + + public KuduTableInfo() { + setType(CURR_TYPE); + } + + + public String getKuduMasters() { + return kuduMasters; + } + + public void setKuduMasters(String kuduMasters) { + this.kuduMasters = kuduMasters; + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public KuduOutputFormat.WriteMode getWriteMode() { + return writeMode; + } + + public void setWriteMode(KuduOutputFormat.WriteMode writeMode) { + this.writeMode = writeMode; + } + + public Integer getWorkerCount() { + return workerCount; + } + + public void setWorkerCount(Integer workerCount) { + this.workerCount = workerCount; + } + + public Integer getDefaultOperationTimeoutMs() { + return defaultOperationTimeoutMs; + } + + public void setDefaultOperationTimeoutMs(Integer defaultOperationTimeoutMs) { + this.defaultOperationTimeoutMs = defaultOperationTimeoutMs; + } + + public Integer getDefaultSocketReadTimeoutMs() { + return defaultSocketReadTimeoutMs; + } + + public void setDefaultSocketReadTimeoutMs(Integer defaultSocketReadTimeoutMs) { + this.defaultSocketReadTimeoutMs = defaultSocketReadTimeoutMs; + } + + @Override + public boolean check() { + Preconditions.checkNotNull(kuduMasters, "Cassandra field of kuduMasters is required"); + Preconditions.checkNotNull(tableName, "Cassandra field of tableName is required"); + return true; + } + + @Override + public String getType() { + // return super.getType().toLowerCase() + TARGET_SUFFIX; + return super.getType().toLowerCase(); + } +} diff --git a/kudu/pom.xml b/kudu/pom.xml new file mode 100644 index 000000000..e29c233c1 --- /dev/null +++ b/kudu/pom.xml @@ -0,0 +1,45 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.kudu + pom + + kudu-sink + kudu-side + + + + + + junit + junit + 3.8.1 + test + + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + + org.apache.kudu + kudu-client + 1.9.0 + + + + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index 0e2d9e709..0bb26671b 100644 --- a/pom.xml +++ b/pom.xml @@ -23,6 +23,7 @@ sqlserver oracle cassandra + kudu From c4885d0caf0cdfd4356b584f852df0cc5f03d7f2 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 4 Nov 2019 18:51:46 +0800 Subject: [PATCH 444/470] clickhouse side table --- .../clickhouse-all-side/pom.xml | 92 +++++++++++++++++++ .../side/clickhouse/ClickhouseAllReqRow.java | 66 +++++++++++++ .../clickhouse/ClickhouseAllSideInfo.java | 34 +++++++ .../clickhouse-async-side/pom.xml | 91 ++++++++++++++++++ .../clickhouse/ClickhouseAsyncReqRow.java | 70 ++++++++++++++ .../clickhouse/ClickhouseAsyncSideInfo.java | 35 +++++++ .../clickhouse-side-core/pom.xml | 17 ++++ .../table/ClickhouseSideParser.java | 45 +++++++++ clickhouse/clickhouse-side/pom.xml | 36 ++++++++ clickhouse/clickhouse-sink/pom.xml | 15 +++ clickhouse/pom.xml | 41 +++++++++ pom.xml | 25 ++--- 12 files changed, 555 insertions(+), 12 deletions(-) create mode 100644 clickhouse/clickhouse-side/clickhouse-all-side/pom.xml create mode 100644 clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllReqRow.java create mode 100644 clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllSideInfo.java create mode 100644 clickhouse/clickhouse-side/clickhouse-async-side/pom.xml create mode 100644 clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java create mode 100644 clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncSideInfo.java create mode 100644 clickhouse/clickhouse-side/clickhouse-side-core/pom.xml create mode 100644 clickhouse/clickhouse-side/clickhouse-side-core/src/main/java/com/dtstack/flink/sql/side/clickhouse/table/ClickhouseSideParser.java create mode 100644 clickhouse/clickhouse-side/pom.xml create mode 100644 clickhouse/clickhouse-sink/pom.xml create mode 100644 clickhouse/pom.xml diff --git a/clickhouse/clickhouse-side/clickhouse-all-side/pom.xml b/clickhouse/clickhouse-side/clickhouse-all-side/pom.xml new file mode 100644 index 000000000..8e6c6f51a --- /dev/null +++ b/clickhouse/clickhouse-side/clickhouse-all-side/pom.xml @@ -0,0 +1,92 @@ + + + + sql.side.clickhouse + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.all.clickhouse + clickhouse-all-side + + jar + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.side.clickhouse.core + ${sql.side.clickhouse.core.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllReqRow.java b/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllReqRow.java new file mode 100644 index 000000000..ea5f621f9 --- /dev/null +++ b/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllReqRow.java @@ -0,0 +1,66 @@ +/* + * 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 com.dtstack.flink.sql.side.clickhouse; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.util.List; +import java.util.Map; + + +public class ClickhouseAllReqRow extends RdbAllReqRow { + + private static final Logger LOG = LoggerFactory.getLogger(ClickhouseAllReqRow.class); + + private static final String CLICKHOUSE_DRIVER = "ru.yandex.clickhouse.ClickHouseDriver"; + + public ClickhouseAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new ClickhouseAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + @Override + public Connection getConn(String dbURL, String userName, String password) { + try { + Class.forName(CLICKHOUSE_DRIVER); + //add param useCursorFetch=true + Map addParams = Maps.newHashMap(); + addParams.put("useCursorFetch", "true"); + String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); + return DriverManager.getConnection(targetDbUrl, userName, password); + } catch (Exception e) { + LOG.error("", e); + throw new RuntimeException("", e); + } + } + + @Override + public int getFetchSize() { + return Integer.MIN_VALUE; + } +} diff --git a/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllSideInfo.java b/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllSideInfo.java new file mode 100644 index 000000000..973c069b9 --- /dev/null +++ b/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllSideInfo.java @@ -0,0 +1,34 @@ +/* + * 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 com.dtstack.flink.sql.side.clickhouse; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; + +import java.util.List; + + +public class ClickhouseAllSideInfo extends RdbAllSideInfo { + public ClickhouseAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } +} diff --git a/clickhouse/clickhouse-side/clickhouse-async-side/pom.xml b/clickhouse/clickhouse-side/clickhouse-async-side/pom.xml new file mode 100644 index 000000000..ab889c5a8 --- /dev/null +++ b/clickhouse/clickhouse-side/clickhouse-async-side/pom.xml @@ -0,0 +1,91 @@ + + + + sql.side.clickhouse + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.async.clickhouse + clickhouse-async-side + + jar + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.side.clickhouse.core + ${sql.side.clickhouse.core.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java b/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java new file mode 100644 index 000000000..27537e3e2 --- /dev/null +++ b/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java @@ -0,0 +1,70 @@ +/* + * 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 com.dtstack.flink.sql.side.clickhouse; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.async.RdbAsyncReqRow; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import io.vertx.core.Vertx; +import io.vertx.core.VertxOptions; +import io.vertx.core.json.JsonObject; +import io.vertx.ext.jdbc.JDBCClient; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; + +import java.util.List; + + +public class ClickhouseAsyncReqRow extends RdbAsyncReqRow { + private static final String CLICKHOUSE_DRIVER = "ru.yandex.clickhouse.ClickHouseDriver"; + + public ClickhouseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new ClickhouseAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + JsonObject clickhouseClientConfig = new JsonObject(); + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); + clickhouseClientConfig.put("url", rdbSideTableInfo.getUrl()) + .put("driver_class", CLICKHOUSE_DRIVER) + .put("max_pool_size", DEFAULT_MAX_DB_CONN_POOL_SIZE) + .put("user", rdbSideTableInfo.getUserName()) + .put("password", rdbSideTableInfo.getPassword()) + .put("provider_class", DT_PROVIDER_CLASS) + .put("preferred_test_query", PREFERRED_TEST_QUERY_SQL) + .put("idle_connection_test_period", DEFAULT_IDLE_CONNECTION_TEST_PEROID) + .put("test_connection_on_checkin", DEFAULT_TEST_CONNECTION_ON_CHECKIN); + + System.setProperty("vertx.disableFileCPResolving", "true"); + + VertxOptions vo = new VertxOptions(); + vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); + vo.setWorkerPoolSize(DEFAULT_VERTX_WORKER_POOL_SIZE); + vo.setFileResolverCachingEnabled(false); + Vertx vertx = Vertx.vertx(vo); + setRdbSQLClient(JDBCClient.createNonShared(vertx, clickhouseClientConfig)); + } + +} diff --git a/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncSideInfo.java b/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncSideInfo.java new file mode 100644 index 000000000..254561de0 --- /dev/null +++ b/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncSideInfo.java @@ -0,0 +1,35 @@ +/* + * 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 com.dtstack.flink.sql.side.clickhouse; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; + +import java.util.List; + + +public class ClickhouseAsyncSideInfo extends RdbAsyncSideInfo { + + public ClickhouseAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } +} diff --git a/clickhouse/clickhouse-side/clickhouse-side-core/pom.xml b/clickhouse/clickhouse-side/clickhouse-side-core/pom.xml new file mode 100644 index 000000000..29304d922 --- /dev/null +++ b/clickhouse/clickhouse-side/clickhouse-side-core/pom.xml @@ -0,0 +1,17 @@ + + + + sql.side.clickhouse + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.clickhouse.core + 1.0-SNAPSHOT + jar + clickhouse-side-core + + \ No newline at end of file diff --git a/clickhouse/clickhouse-side/clickhouse-side-core/src/main/java/com/dtstack/flink/sql/side/clickhouse/table/ClickhouseSideParser.java b/clickhouse/clickhouse-side/clickhouse-side-core/src/main/java/com/dtstack/flink/sql/side/clickhouse/table/ClickhouseSideParser.java new file mode 100644 index 000000000..692b588ea --- /dev/null +++ b/clickhouse/clickhouse-side/clickhouse-side-core/src/main/java/com/dtstack/flink/sql/side/clickhouse/table/ClickhouseSideParser.java @@ -0,0 +1,45 @@ +/* + * 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 com.dtstack.flink.sql.side.clickhouse.table; + +import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; +import com.dtstack.flink.sql.table.TableInfo; + +import java.util.Map; + +/** + * Reason: + * Date: 2019/11/04 + * Company: www.dtstack.com + * + * @author maqi + */ + +public class ClickhouseSideParser extends RdbSideParser { + + private static final String CURR_TYPE = "clickhouse"; + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + TableInfo clickhouseTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + clickhouseTableInfo.setType(CURR_TYPE); + return clickhouseTableInfo; + } +} diff --git a/clickhouse/clickhouse-side/pom.xml b/clickhouse/clickhouse-side/pom.xml new file mode 100644 index 000000000..2ba6a14dc --- /dev/null +++ b/clickhouse/clickhouse-side/pom.xml @@ -0,0 +1,36 @@ + + + + sql.clickhouse + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.side.clickhouse + 1.0-SNAPSHOT + clickhouse-side + pom + + + clickhouse-side-core + clickhouse-async-side + clickhouse-all-side + + + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.side.rdb + ${rdb.side.version} + + + + \ No newline at end of file diff --git a/clickhouse/clickhouse-sink/pom.xml b/clickhouse/clickhouse-sink/pom.xml new file mode 100644 index 000000000..7892fa230 --- /dev/null +++ b/clickhouse/clickhouse-sink/pom.xml @@ -0,0 +1,15 @@ + + + + sql.clickhouse + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.sink.clickhouse + + + \ No newline at end of file diff --git a/clickhouse/pom.xml b/clickhouse/pom.xml new file mode 100644 index 000000000..37589bb36 --- /dev/null +++ b/clickhouse/pom.xml @@ -0,0 +1,41 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.clickhouse + pom + + + clickhouse-side + clickhouse-sink + + + + + 1.0-SNAPSHOT + 0.1.55 + + + + + com.dtstack.flink + sql.core + ${sql.core.version} + provided + + + + ru.yandex.clickhouse + clickhouse-jdbc + ${clickhouse.jdbc.version} + + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index cf203002a..254985045 100644 --- a/pom.xml +++ b/pom.xml @@ -10,22 +10,23 @@ http://maven.apache.org core - kafka09 - kafka10 - kafka11 - mysql - hbase - elasticsearch5 - mongo - redis5 + + + + + + + + launcher rdb - sqlserver - oracle - cassandra + + + serversocket - console + + clickhouse From dd5fc89bb5e7781fce681f7ec252601ebe017838 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 4 Nov 2019 19:22:30 +0800 Subject: [PATCH 445/470] fix splitIgoreQuota method --- .../dtstack/flink/sql/util/DtStringUtil.java | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index 3e7d828f3..8cf59f757 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -66,24 +66,30 @@ public static List splitIgnoreQuota(String str, char delimiter){ if (idx > 0) { flag = chars[idx - 1]; } - if(c == delimiter){ + if (c == delimiter) { if (inQuotes) { b.append(c); - } else if(inSingleQuotes){ + } else if (inSingleQuotes) { b.append(c); - } else if(bracketLeftNum > 0){ + } else if (bracketLeftNum > 0) { b.append(c); - }else { + } else { tokensList.add(b.toString()); b = new StringBuilder(); } - }else if(c == '\"' && '\\'!=flag && !inSingleQuotes){ + } else if (c == '\"' && '\\' != flag && !inSingleQuotes) { inQuotes = !inQuotes; b.append(c); - }else if(c == '\'' && '\\'!=flag && !inQuotes){ + } else if (c == '\'' && '\\' != flag && !inQuotes) { inSingleQuotes = !inSingleQuotes; b.append(c); - }else{ + } else if (c == '(') { + bracketLeftNum++; + b.append(c); + } else if (c == ')') { + bracketLeftNum--; + b.append(c); + } else { b.append(c); } idx++; From c109d655b2802fed9a8f1a9de88796aa0e926704 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BF=AE=E7=AB=B9?= Date: Tue, 5 Nov 2019 13:24:16 +0800 Subject: [PATCH 446/470] =?UTF-8?q?=E6=94=AF=E6=8C=81=E5=B5=8C=E5=A5=97jso?= =?UTF-8?q?n=E5=AD=97=E6=AE=B5=E6=8F=90=E5=8F=96=E5=92=8C=E6=95=B0?= =?UTF-8?q?=E7=BB=84=E7=B1=BB=E5=9E=8B=E8=A7=A3=E6=9E=90?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../kafka/CustomerJsonDeserialization.java | 51 ++++++++++++------- .../source/kafka/table/KafkaSourceParser.java | 2 +- .../kafka/CustomerJsonDeserialization.java | 50 +++++++++++------- .../source/kafka/table/KafkaSourceParser.java | 2 +- .../kafka/CustomerJsonDeserialization.java | 44 ++++++++++------ .../source/kafka/table/KafkaSourceParser.java | 2 +- 6 files changed, 97 insertions(+), 54 deletions(-) diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index d659b2882..53521bfad 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -32,7 +32,6 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; @@ -42,7 +41,7 @@ import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; import java.io.IOException; import java.lang.reflect.Field; import java.sql.Date; @@ -159,29 +158,37 @@ public Row deserialize(byte[] message) throws IOException { } } - public void setFailOnMissingField(boolean failOnMissingField) { - this.failOnMissingField = failOnMissingField; - } - - private JsonNode getIgnoreCase(String key) { + public JsonNode getIgnoreCase(String key) { String nodeMappingKey = rowAndFieldMapping.getOrDefault(key, key); - JsonNode node = nodeAndJsonNodeMapping.get(nodeMappingKey); - if(node == null){ - return null; - } - - JsonNodeType nodeType = node.getNodeType(); - if (nodeType == JsonNodeType.ARRAY){ - throw new IllegalStateException("Unsupported type information array .") ; - } + return nodeAndJsonNodeMapping.get(nodeMappingKey); + } - return node; + public void setFailOnMissingField(boolean failOnMissingField) { + this.failOnMissingField = failOnMissingField; } private void parseTree(JsonNode jsonNode, String prefix){ + if (jsonNode.isArray()) { + ArrayNode array = (ArrayNode) jsonNode; + for (int i = 0; i < array.size(); i++) { + JsonNode child = array.get(i); + String nodeKey = getNodeKey(prefix, i); + + if (child.isValueNode()) { + nodeAndJsonNodeMapping.put(nodeKey, child); + } else { + if (rowAndFieldMapping.containsValue(nodeKey)) { + nodeAndJsonNodeMapping.put(nodeKey, child); + } + parseTree(child, nodeKey); + } + } + return; + } + Iterator iterator = jsonNode.fieldNames(); while (iterator.hasNext()){ String next = iterator.next(); @@ -191,7 +198,7 @@ private void parseTree(JsonNode jsonNode, String prefix){ if (child.isValueNode()){ nodeAndJsonNodeMapping.put(nodeKey, child); } else if(child.isArray()){ - nodeAndJsonNodeMapping.put(nodeKey, new TextNode(child.toString())); + parseTree(child, nodeKey); }else { parseTree(child, nodeKey); } @@ -206,6 +213,14 @@ private String getNodeKey(String prefix, String nodeName){ return prefix + "." + nodeName; } + private String getNodeKey(String prefix, int i) { + if (Strings.isNullOrEmpty(prefix)) { + return "[" + i + "]"; + } + + return prefix + "[" + i + "]"; + } + public void setFetcher(AbstractFetcher fetcher) { this.fetcher = fetcher; } diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index c2fe6cb21..6c4b64d60 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -45,7 +45,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\S+\\.)*\\S+)\\s+(\\w+)\\s+AS\\s+(\\w+)(\\s+NOT\\s+NULL)?$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 3e5f83f9d..44e870ed0 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -32,8 +32,7 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.types.Row; @@ -161,19 +160,8 @@ public Row deserialize(byte[] message) throws IOException { public JsonNode getIgnoreCase(String key) { String nodeMappingKey = rowAndFieldMapping.getOrDefault(key, key); - JsonNode node = nodeAndJsonNodeMapping.get(nodeMappingKey); - if(node == null){ - return null; - } - - JsonNodeType nodeType = node.getNodeType(); - - if (nodeType==JsonNodeType.ARRAY){ - throw new IllegalStateException("Unsupported type information array .") ; - } - - return node; + return nodeAndJsonNodeMapping.get(nodeMappingKey); } @@ -183,17 +171,35 @@ public void setFailOnMissingField(boolean failOnMissingField) { private void parseTree(JsonNode jsonNode, String prefix){ + if (jsonNode.isArray()) { + ArrayNode array = (ArrayNode) jsonNode; + for (int i = 0; i < array.size(); i++) { + JsonNode child = array.get(i); + String nodeKey = getNodeKey(prefix, i); + + if (child.isValueNode()) { + nodeAndJsonNodeMapping.put(nodeKey, child); + } else { + if (rowAndFieldMapping.containsValue(nodeKey)) { + nodeAndJsonNodeMapping.put(nodeKey, child); + } + parseTree(child, nodeKey); + } + } + return; + } + Iterator iterator = jsonNode.fieldNames(); while (iterator.hasNext()){ String next = iterator.next(); JsonNode child = jsonNode.get(next); String nodeKey = getNodeKey(prefix, next); - if (child.isValueNode()){ + if (child.isValueNode()) { nodeAndJsonNodeMapping.put(nodeKey, child); - }else if(child.isArray()){ - nodeAndJsonNodeMapping.put(nodeKey, new TextNode(child.toString())); - }else { + } else if(child.isArray()){ + parseTree(child, nodeKey); + } else { parseTree(child, nodeKey); } } @@ -207,6 +213,14 @@ private String getNodeKey(String prefix, String nodeName){ return prefix + "." + nodeName; } + private String getNodeKey(String prefix, int i) { + if (Strings.isNullOrEmpty(prefix)) { + return "[" + i + "]"; + } + + return prefix + "[" + i + "]"; + } + public void setFetcher(AbstractFetcher fetcher) { this.fetcher = fetcher; } diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 020d54781..005d4d4f0 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -44,7 +44,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\S+\\.)*\\S+)\\s+(\\w+)\\s+AS\\s+(\\w+)(\\s+NOT\\s+NULL)?$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 1002731e8..943e8b148 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -31,8 +31,8 @@ import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; @@ -44,7 +44,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.lang.reflect.Array; import java.lang.reflect.Field; import java.sql.Date; import java.sql.Time; @@ -166,18 +165,7 @@ public Row deserialize(byte[] message) throws IOException { public JsonNode getIgnoreCase(String key) { String nodeMappingKey = rowAndFieldMapping.getOrDefault(key, key); - JsonNode node = nodeAndJsonNodeMapping.get(nodeMappingKey); - if(node == null){ - return null; - } - - JsonNodeType nodeType = node.getNodeType(); - - if (nodeType==JsonNodeType.ARRAY){ - throw new IllegalStateException("Unsupported type information array .") ; - } - - return node; + return nodeAndJsonNodeMapping.get(nodeMappingKey); } public void setFailOnMissingField(boolean failOnMissingField) { @@ -186,6 +174,24 @@ public void setFailOnMissingField(boolean failOnMissingField) { private void parseTree(JsonNode jsonNode, String prefix){ + if (jsonNode.isArray()) { + ArrayNode array = (ArrayNode) jsonNode; + for (int i = 0; i < array.size(); i++) { + JsonNode child = array.get(i); + String nodeKey = getNodeKey(prefix, i); + + if (child.isValueNode()) { + nodeAndJsonNodeMapping.put(nodeKey, child); + } else { + if (rowAndFieldMapping.containsValue(nodeKey)) { + nodeAndJsonNodeMapping.put(nodeKey, child); + } + parseTree(child, nodeKey); + } + } + return; + } + Iterator iterator = jsonNode.fieldNames(); while (iterator.hasNext()){ String next = iterator.next(); @@ -195,7 +201,7 @@ private void parseTree(JsonNode jsonNode, String prefix){ if (child.isValueNode()){ nodeAndJsonNodeMapping.put(nodeKey, child); }else if(child.isArray()){ - nodeAndJsonNodeMapping.put(nodeKey, new TextNode(child.toString())); + parseTree(child, nodeKey); }else { parseTree(child, nodeKey); } @@ -210,6 +216,14 @@ private String getNodeKey(String prefix, String nodeName){ return prefix + "." + nodeName; } + private String getNodeKey(String prefix, int i) { + if (Strings.isNullOrEmpty(prefix)) { + return "[" + i + "]"; + } + + return prefix + "[" + i + "]"; + } + public void setFetcher(AbstractFetcher fetcher) { this.fetcher = fetcher; } diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index 85f6cd189..d926de71b 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -44,7 +44,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\S+\\.)*\\S+)\\s+(\\w+)\\s+AS\\s+(\\w+)(\\s+NOT\\s+NULL)?$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); From b5d64871c02bf7d41ee4ccf1dfa1eec865b518f9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BF=AE=E7=AB=B9?= Date: Tue, 5 Nov 2019 13:54:56 +0800 Subject: [PATCH 447/470] =?UTF-8?q?=E5=9C=A8kafkaSource.md=E5=A2=9E?= =?UTF-8?q?=E5=8A=A0=E5=AF=B9=E5=B5=8C=E5=A5=97json=E5=92=8C=E6=95=B0?= =?UTF-8?q?=E7=BB=84=E7=B1=BB=E5=9E=8B=E5=AD=97=E6=AE=B5=E8=A7=A3=E6=9E=90?= =?UTF-8?q?=E7=9A=84=E8=AF=B4=E6=98=8E?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/kafkaSource.md | 70 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 70 insertions(+) diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index 748948a7b..04302d234 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -63,3 +63,73 @@ CREATE TABLE MyTable( parallelism ='1' ); ``` + +## 6.支持嵌套json、数据类型字段解析 + +嵌套json解析示例 + +json: {"name":"tom", "obj":{"channel": "root"}, "pv": 4, "xctime":1572932485} +``` +CREATE TABLE MyTable( + name varchar, + obj.channel varchar as channel, + pv INT, + xctime bigint, + CHARACTER_LENGTH(channel) AS timeLeng + )WITH( + type ='kafka09', + bootstrapServers ='172.16.8.198:9092', + zookeeperQuorum ='172.16.8.198:2181/kafka', + offsetReset ='latest', + groupId='nbTest', + topic ='nbTest1,nbTest2,nbTest3', + --- topic ='mqTest.*', + ---topicIsPattern='true', + parallelism ='1' + ); +``` + +数组类型字段解析示例 + +json: {"name":"tom", "obj":{"channel": "root"}, "user": [{"pv": 4}, {"pv": 10}], "xctime":1572932485} +``` +CREATE TABLE MyTable( + name varchar, + obj.channel varchar as channel, + user[1].pv INT as pv, + xctime bigint, + CHARACTER_LENGTH(channel) AS timeLeng + )WITH( + type ='kafka09', + bootstrapServers ='172.16.8.198:9092', + zookeeperQuorum ='172.16.8.198:2181/kafka', + offsetReset ='latest', + groupId='nbTest', + topic ='nbTest1,nbTest2,nbTest3', + --- topic ='mqTest.*', + ---topicIsPattern='true', + parallelism ='1' + ); +``` +or + +json: {"name":"tom", "obj":{"channel": "root"}, "pv": [4, 7, 10], "xctime":1572932485} +``` +CREATE TABLE MyTable( + name varchar, + obj.channel varchar as channel, + pv[1] INT as pv, + xctime bigint, + CHARACTER_LENGTH(channel) AS timeLeng + )WITH( + type ='kafka09', + bootstrapServers ='172.16.8.198:9092', + zookeeperQuorum ='172.16.8.198:2181/kafka', + offsetReset ='latest', + groupId='nbTest', + topic ='nbTest1,nbTest2,nbTest3', + --- topic ='mqTest.*', + ---topicIsPattern='true', + parallelism ='1' + ); +``` \ No newline at end of file From 73d5f019f535071fcfa8f94838a86307091a258a Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 5 Nov 2019 15:32:03 +0800 Subject: [PATCH 448/470] clickhouse sink and field parse --- .../side/clickhouse/ClickhouseAllReqRow.java | 23 +++--- .../clickhouse/ClickhouseAsyncReqRow.java | 8 +- .../table/ClickhouseSideParser.java | 7 ++ clickhouse/clickhouse-sink/pom.xml | 75 ++++++++++++++++++ .../sql/sink/clickhouse/ClickhouseSink.java | 79 +++++++++++++++++++ .../table/ClickhouseSinkParser.java | 44 +++++++++++ .../flink/sql/table/AbsTableParser.java | 7 +- pom.xml | 24 +++--- .../flink/sql/side/rdb/all/RdbAllReqRow.java | 2 - .../sql/side/rdb/table/RdbSideParser.java | 1 - .../rdb/format/RetractJDBCOutputFormat.java | 2 +- 11 files changed, 236 insertions(+), 36 deletions(-) create mode 100644 clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/ClickhouseSink.java create mode 100644 clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/table/ClickhouseSinkParser.java diff --git a/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllReqRow.java b/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllReqRow.java index ea5f621f9..68c0c7984 100644 --- a/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllReqRow.java +++ b/clickhouse/clickhouse-side/clickhouse-all-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAllReqRow.java @@ -23,6 +23,7 @@ import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; import com.dtstack.flink.sql.util.DtStringUtil; +import com.dtstack.flink.sql.util.JDBCUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; import org.slf4j.Logger; @@ -33,7 +34,6 @@ import java.util.List; import java.util.Map; - public class ClickhouseAllReqRow extends RdbAllReqRow { private static final Logger LOG = LoggerFactory.getLogger(ClickhouseAllReqRow.class); @@ -45,22 +45,21 @@ public ClickhouseAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List addParams = Maps.newHashMap(); - addParams.put("useCursorFetch", "true"); - String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); - return DriverManager.getConnection(targetDbUrl, userName, password); + Connection connection ; + JDBCUtils.forName(CLICKHOUSE_DRIVER, getClass().getClassLoader()); + // ClickHouseProperties contains all properties + if (userName == null) { + connection = DriverManager.getConnection(dbURL); + } else { + connection = DriverManager.getConnection(dbURL, userName, passWord); + } + return connection; } catch (Exception e) { LOG.error("", e); throw new RuntimeException("", e); } } - @Override - public int getFetchSize() { - return Integer.MIN_VALUE; - } } diff --git a/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java b/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java index 27537e3e2..305d65118 100644 --- a/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java +++ b/clickhouse/clickhouse-side/clickhouse-async-side/src/main/java/com/dtstack/flink/sql/side/clickhouse/ClickhouseAsyncReqRow.java @@ -41,7 +41,6 @@ public ClickhouseAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List4.0.0 sql.sink.clickhouse + clickhouse-sink + http://maven.apache.org + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.sink.rdb + ${sql.sink.rdb.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/ClickhouseSink.java b/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/ClickhouseSink.java new file mode 100644 index 000000000..66c1f26d5 --- /dev/null +++ b/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/ClickhouseSink.java @@ -0,0 +1,79 @@ +/* + * 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 com.dtstack.flink.sql.sink.clickhouse; + + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; + +import java.util.List; +import java.util.Map; + + +public class ClickhouseSink extends RdbSink implements IStreamSinkGener { + + private static final String CLICKHOUSE_DRIVER = "ru.yandex.clickhouse.ClickHouseDriver"; + + public ClickhouseSink() { + } + + @Override + public RetractJDBCOutputFormat getOutputFormat() { + return new RetractJDBCOutputFormat(); + } + + @Override + public void buildSql(String scheam, String tableName, List fields) { + buildInsertSql(tableName, fields); + } + + @Override + public String buildUpdateSql(String schema, String tableName, List fieldNames, Map> realIndexes, List fullField) { + return null; + } + + private void buildInsertSql(String tableName, List fields) { + String sqlTmp = "insert into " + tableName + " (${fields}) values (${placeholder})"; + String fieldsStr = ""; + String placeholder = ""; + + for (String fieldName : fields) { + fieldsStr += ",`" + fieldName + "`"; + placeholder += ",?"; + } + + fieldsStr = fieldsStr.replaceFirst(",", ""); + placeholder = placeholder.replaceFirst(",", ""); + + sqlTmp = sqlTmp.replace("${fields}", fieldsStr).replace("${placeholder}", placeholder); + this.sql = sqlTmp; + System.out.println("---insert sql----"); + System.out.println(sql); + } + + + @Override + public String getDriverName() { + return CLICKHOUSE_DRIVER; + } + + +} diff --git a/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/table/ClickhouseSinkParser.java b/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/table/ClickhouseSinkParser.java new file mode 100644 index 000000000..8c3df93d7 --- /dev/null +++ b/clickhouse/clickhouse-sink/src/main/java/com/dtstack/flink/sql/sink/clickhouse/table/ClickhouseSinkParser.java @@ -0,0 +1,44 @@ +/* + * 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 com.dtstack.flink.sql.sink.clickhouse.table; + +import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; +import com.dtstack.flink.sql.table.TableInfo; +import ru.yandex.clickhouse.domain.ClickHouseDataType; + +import java.util.Map; + + +public class ClickhouseSinkParser extends RdbSinkParser { + private static final String CURR_TYPE = "clickhouse"; + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + TableInfo clickhouseTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + clickhouseTableInfo.setType(CURR_TYPE); + return clickhouseTableInfo; + } + + @Override + public Class dbTypeConvertToJavaType(String fieldType) { + return ClickHouseDataType.fromTypeString(fieldType).getJavaClass(); + } + +} \ No newline at end of file diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index fc29932bf..aef4984d2 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -99,7 +99,7 @@ public void parseFieldsInfo(String fieldsInfo, TableInfo tableInfo){ System.arraycopy(filedInfoArr, 0, filedNameArr, 0, filedInfoArr.length - 1); String fieldName = String.join(" ", filedNameArr); String fieldType = filedInfoArr[filedInfoArr.length - 1 ].trim(); - Class fieldClass = ClassUtil.stringConvertClass(fieldType); + Class fieldClass = dbTypeConvertToJavaType(fieldType); tableInfo.addPhysicalMappings(filedInfoArr[0],filedInfoArr[0]); tableInfo.addField(fieldName); @@ -117,4 +117,9 @@ public static void dealPrimaryKey(Matcher matcher, TableInfo tableInfo){ List primaryKes = Lists.newArrayList(splitArry); tableInfo.setPrimaryKeys(primaryKes); } + + public Class dbTypeConvertToJavaType(String fieldType) { + return ClassUtil.stringConvertClass(fieldType); + } + } diff --git a/pom.xml b/pom.xml index 254985045..a21e21e10 100644 --- a/pom.xml +++ b/pom.xml @@ -10,22 +10,22 @@ http://maven.apache.org core - - - - - - - - + kafka09 + kafka10 + kafka11 + mysql + hbase + elasticsearch5 + mongo + redis5 launcher rdb - - - + sqlserver + oracle + cassandra serversocket - + console clickhouse diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java index 6c3f205a1..94d1de1ff 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java @@ -174,7 +174,6 @@ private void loadData(Map>> tmpCache) throws SQ try { for (int i = 0; i < CONN_RETRY_NUM; i++) { - try { connection = getConn(tableInfo.getUrl(), tableInfo.getUserName(), tableInfo.getPassword()); break; @@ -182,7 +181,6 @@ private void loadData(Map>> tmpCache) throws SQ if (i == CONN_RETRY_NUM - 1) { throw new RuntimeException("", e); } - try { String connInfo = "url:" + tableInfo.getUrl() + ";userName:" + tableInfo.getUserName() + ",pwd:" + tableInfo.getPassword(); LOG.warn("get conn fail, wait for 5 sec and try again, connInfo:" + connInfo); diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java index 7c559856f..b9811b0ee 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideParser.java @@ -47,7 +47,6 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map Date: Tue, 5 Nov 2019 16:48:02 +0800 Subject: [PATCH 449/470] =?UTF-8?q?=E5=A2=9E=E5=8A=A0kuduSink.md=E5=92=8Ck?= =?UTF-8?q?uduSide.md=E8=AF=B4=E6=98=8E?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 6 ++- docs/kuduSide.md | 127 +++++++++++++++++++++++++++++++++++++++++++++++ docs/kuduSink.md | 62 +++++++++++++++++++++++ 3 files changed, 193 insertions(+), 2 deletions(-) create mode 100644 docs/kuduSide.md create mode 100644 docs/kuduSink.md diff --git a/README.md b/README.md index 93edde5c3..b564e6b35 100644 --- a/README.md +++ b/README.md @@ -10,8 +10,8 @@ # 已支持 * 源表:kafka 0.9,1.x版本 - * 维表:mysql,SQlServer,oracle,hbase,mongo,redis,cassandra - * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra + * 维表:mysql, SQlServer,oracle, hbase, mongo, redis, cassandra, kudu + * 结果表:mysql, SQlServer, oracle, hbase, elasticsearch5.x, mongo, redis, cassandra, kudu # 后续开发计划 * 增加SQL支持CEP @@ -154,6 +154,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * [mongo 结果表插件](docs/mongoSink.md) * [redis 结果表插件](docs/redisSink.md) * [cassandra 结果表插件](docs/cassandraSink.md) +* [kudu 结果表插件](docs/kuduSink.md) ### 2.3 维表插件 * [hbase 维表插件](docs/hbaseSide.md) @@ -161,6 +162,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * [mongo 维表插件](docs/mongoSide.md) * [redis 维表插件](docs/redisSide.md) * [cassandra 维表插件](docs/cassandraSide.md) +* [kudu 维表插件](docs/kuduSide.md) ## 3 性能指标(新增) diff --git a/docs/kuduSide.md b/docs/kuduSide.md new file mode 100644 index 000000000..fece4a480 --- /dev/null +++ b/docs/kuduSide.md @@ -0,0 +1,127 @@ + +## 1.格式: +All: +``` +create table sideTable( + id int, + tablename1 VARCHAR, + PRIMARY KEY(id), + PERIOD FOR SYSTEM_TIME + )WITH( + type='kudu', + kuduMasters ='ip1,ip2,ip3', + tableName ='impala::default.testSide', + cache ='ALL', + primaryKey='id,xx', + lowerBoundPrimaryKey='10,xx', + upperBoundPrimaryKey='15,xx', + workerCount='1', + defaultOperationTimeoutMs='600000', + defaultSocketReadTimeoutMs='6000000', + batchSizeBytes='100000000', + limitNum='1000', + isFaultTolerant='false', + partitionedJoin='false' + ); +``` +LRU: +``` +create table sideTable( + id int, + tablename1 VARCHAR, + PRIMARY KEY(id), + PERIOD FOR SYSTEM_TIME + )WITH( + type='kudu', + kuduMasters ='ip1,ip2,ip3', + tableName ='impala::default.testSide', + cache ='LRU', + workerCount='1', + defaultOperationTimeoutMs='600000', + defaultSocketReadTimeoutMs='6000000', + batchSizeBytes='100000000', + limitNum='1000', + isFaultTolerant='false', + partitionedJoin='false' + ); + ``` + +## 2.支持版本 +kudu 1.9.0+cdh6.2.0 + +## 3.表结构定义 + + |参数名称|含义| + |----|---| + | tableName | 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同)| + | colName | 列名称| + | colType | 列类型 [colType支持的类型](colType.md)| + | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| + | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| + +## 3.参数 + + +|参数名称|含义|是否必填|默认值| +|----|---|---|-----| +|type | 表明维表的类型[hbase\|mysql|\kudu]|是|| +| kuduMasters | kudu master节点的地址;格式ip[ip,ip2]|是|| +| tableName | kudu 的表名称|是|| +| workerCount | 工作线程数 |否| +| defaultOperationTimeoutMs | 写入操作超时时间 |否| +| defaultSocketReadTimeoutMs | socket读取超时时间 |否| +| primaryKey | 需要过滤的主键 ALL模式独有 |否| +| lowerBoundPrimaryKey | 需要过滤的主键的最小值 ALL模式独有 |否| +| upperBoundPrimaryKey | 需要过滤的主键的最大值(不包含) ALL模式独有 |否| +| workerCount | 工作线程数 |否| +| defaultOperationTimeoutMs | 写入操作超时时间 |否| +| defaultSocketReadTimeoutMs | socket读取超时时间 |否| +| batchSizeBytes |返回数据的大小 | 否| +| limitNum |返回数据的条数 | 否| +| isFaultTolerant |查询是否容错 查询失败是否扫描第二个副本 默认false 容错 | 否| +| cache | 维表缓存策略(NONE/LRU/ALL)|否|NONE| +| partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| + + +-------------- +> 缓存策略 + * NONE: 不做内存缓存 + * LRU: + * cacheSize: 缓存的条目数量 + * cacheTTLMs:缓存的过期时间(ms) + +## 4.样例 +All: +``` +create table sideTable( + id int, + tablename1 VARCHAR, + PRIMARY KEY(id), + PERIOD FOR SYSTEM_TIME + )WITH( + type='kudu', + kuduMasters ='ip1,ip2,ip3', + tableName ='impala::default.testSide', + cache ='ALL', + primaryKey='id,xx', + lowerBoundPrimaryKey='10,xx', + upperBoundPrimaryKey='15,xx', + partitionedJoin='false' + ); +``` +LRU: +``` +create table sideTable( + id int, + tablename1 VARCHAR, + PRIMARY KEY(id), + PERIOD FOR SYSTEM_TIME + )WITH( + type='kudu', + kuduMasters ='ip1,ip2,ip3', + tableName ='impala::default.testSide', + cache ='LRU', + partitionedJoin='false' + ); + ``` + diff --git a/docs/kuduSink.md b/docs/kuduSink.md new file mode 100644 index 000000000..ba607c7ec --- /dev/null +++ b/docs/kuduSink.md @@ -0,0 +1,62 @@ +## 1.格式: +``` +CREATE TABLE tableName( + colName colType, + ... + colNameX colType + )WITH( + type ='kudu', + kuduMasters ='ip1,ip2,ip3', + tableName ='impala::default.test', + writeMode='upsert', + workerCount='1', + defaultOperationTimeoutMs='600000', + defaultSocketReadTimeoutMs='6000000', + parallelism ='parllNum' + ); + + +``` + +## 2.支持版本 +kudu 1.9.0+cdh6.2.0 + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName | 在 sql 中使用的名称;即注册到flink-table-env上的名称 +| colName | 列名称,redis中存储为 表名:主键名:主键值:列名]| +| colType | 列类型 [colType支持的类型](colType.md)| + + +## 4.参数: + +|参数名称|含义|是否必填|默认值| +|----|---|---|-----| +|type | 表明 输出表类型[mysql\|hbase\|elasticsearch\redis\|kudu\]|是|| +| kuduMasters | kudu master节点的地址;格式ip[ip,ip2]|是|| +| tableName | kudu 的表名称|是|| +| writeMode | 写入kudu的模式 insert|update|upsert |否 |upsert +| workerCount | 工作线程数 |否| +| defaultOperationTimeoutMs | 写入操作超时时间 |否| +| defaultSocketReadTimeoutMs | socket读取超时时间 |否| +|parallelism | 并行度设置|否|1| + + +## 5.样例: +``` +CREATE TABLE MyResult( + id int, + title VARCHAR, + amount decimal, + tablename1 VARCHAR + )WITH( + type ='kudu', + kuduMasters ='localhost1,localhost2,localhost3', + tableName ='impala::default.test', + writeMode='upsert', + parallelism ='1' + ); + + ``` \ No newline at end of file From fd66d84823e1c72e3ef410a639ab8d2c0dce50f3 Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Tue, 5 Nov 2019 17:47:10 +0800 Subject: [PATCH 450/470] kafka1.0 connector --- kafka/kafka-sink/pom.xml | 89 ++++++ .../kafka/CustomerFlinkKafkaProducer.java | 60 ++++ .../CustomerJsonRowSerializationSchema.java | 222 +++++++++++++++ .../kafka/CustomerKafkaJsonTableSink.java | 65 +++++ .../flink/sql/sink/kafka/KafkaSink.java | 144 ++++++++++ .../sql/sink/kafka/table/KafkaSinkParser.java | 63 +++++ .../sink/kafka/table/KafkaSinkTableInfo.java | 93 ++++++ kafka/kafka-source/pom.xml | 81 ++++++ .../kafka/CustomerJsonDeserialization.java | 266 ++++++++++++++++++ .../source/kafka/CustomerKafkaConsumer.java | 72 +++++ .../flink/sql/source/kafka/KafkaSource.java | 122 ++++++++ .../metric/KafkaTopicPartitionLagMetric.java | 46 +++ .../source/kafka/table/KafkaSourceParser.java | 97 +++++++ .../kafka/table/KafkaSourceTableInfo.java | 128 +++++++++ kafka/pom.xml | 43 +++ pom.xml | 1 + 16 files changed, 1592 insertions(+) create mode 100644 kafka/kafka-sink/pom.xml create mode 100644 kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer.java create mode 100644 kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java create mode 100644 kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafkaJsonTableSink.java create mode 100644 kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java create mode 100644 kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java create mode 100644 kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java create mode 100644 kafka/kafka-source/pom.xml create mode 100644 kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java create mode 100644 kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafkaConsumer.java create mode 100644 kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java create mode 100644 kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/metric/KafkaTopicPartitionLagMetric.java create mode 100644 kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java create mode 100644 kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java create mode 100644 kafka/pom.xml diff --git a/kafka/kafka-sink/pom.xml b/kafka/kafka-sink/pom.xml new file mode 100644 index 000000000..41aa899c2 --- /dev/null +++ b/kafka/kafka-sink/pom.xml @@ -0,0 +1,89 @@ + + + + sql.kafka + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.kafka + 1.0-SNAPSHOT + kafka-sink + jar + + + + + org.apache.flink + flink-json + ${flink.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + org.slf4j + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer.java b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer.java new file mode 100644 index 000000000..b7976a30e --- /dev/null +++ b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer.java @@ -0,0 +1,60 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka; + +import com.dtstack.flink.sql.metric.MetricConstant; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer; + +import java.util.Properties; + +/** + * @author: chuixue + * @create: 2019-11-05 11:54 + * @description: + **/ +public class CustomerFlinkKafkaProducer extends FlinkKafkaProducer { + + CustomerJsonRowSerializationSchema schema; + + public CustomerFlinkKafkaProducer(String topicId, SerializationSchema serializationSchema, Properties producerConfig) { + super(topicId, serializationSchema, producerConfig); + this.schema = (CustomerJsonRowSerializationSchema) serializationSchema; + } + + @Override + public void open(Configuration configuration) { + RuntimeContext ctx = getRuntimeContext(); + Counter counter = ctx.getMetricGroup().counter(MetricConstant.DT_NUM_RECORDS_OUT); + MeterView meter = ctx.getMetricGroup().meter(MetricConstant.DT_NUM_RECORDS_OUT_RATE, new MeterView(counter, 20)); + + schema.setCounter(counter); + + try { + super.open(configuration); + } catch (Exception e) { + throw new RuntimeException("",e); + } + } + +} diff --git a/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java new file mode 100644 index 000000000..d698d3ee2 --- /dev/null +++ b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java @@ -0,0 +1,222 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.formats.json.JsonRowDeserializationSchema; +import org.apache.flink.formats.json.JsonRowSchemaConverter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ContainerNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.sql.Time; +import java.sql.Timestamp; +import java.text.SimpleDateFormat; + +/** + * Serialization schema that serializes an object of Flink types into a JSON bytes. + * + *

Serializes the input Flink object into a JSON string and + * converts it into byte[]. + * + *

Result byte[] messages can be deserialized using {@link JsonRowDeserializationSchema}. + */ +@PublicEvolving +public class CustomerJsonRowSerializationSchema implements SerializationSchema { + + private static final long serialVersionUID = -2885556750743978636L; + + /** Type information describing the input type. */ + private final TypeInformation typeInfo; + + /** Object mapper that is used to create output JSON objects. */ + private final ObjectMapper mapper = new ObjectMapper(); + + /** Formatter for RFC 3339-compliant string representation of a time value (with UTC timezone, without milliseconds). */ + private SimpleDateFormat timeFormat = new SimpleDateFormat("HH:mm:ss'Z'"); + + /** Formatter for RFC 3339-compliant string representation of a time value (with UTC timezone). */ + private SimpleDateFormat timeFormatWithMillis = new SimpleDateFormat("HH:mm:ss.SSS'Z'"); + + /** Formatter for RFC 3339-compliant string representation of a timestamp value (with UTC timezone). */ + private SimpleDateFormat timestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"); + + /** Reusable object node. */ + private transient ObjectNode node; + + private Counter counter; + + /** + * Creates a JSON serialization schema for the given type information. + * + * @param typeInfo The field names of {@link Row} are used to map to JSON properties. + */ + public CustomerJsonRowSerializationSchema(TypeInformation typeInfo) { + Preconditions.checkNotNull(typeInfo, "Type information"); + this.typeInfo = typeInfo; + } + + /** + * Creates a JSON serialization schema for the given JSON schema. + * + * @param jsonSchema JSON schema describing the result type + * + * @see http://json-schema.org/ + */ + public CustomerJsonRowSerializationSchema(String jsonSchema) { + this(JsonRowSchemaConverter.convert(jsonSchema)); + } + + @Override + public byte[] serialize(Row row) { + if (node == null) { + node = mapper.createObjectNode(); + } + + try { + convertRow(node, (RowTypeInfo) typeInfo, row); + counter.inc(); + return mapper.writeValueAsBytes(node); + } catch (Throwable t) { + throw new RuntimeException("Could not serialize row '" + row + "'. " + + "Make sure that the schema matches the input.", t); + } + } + + // -------------------------------------------------------------------------------------------- + + private ObjectNode convertRow(ObjectNode reuse, RowTypeInfo info, Row row) { + if (reuse == null) { + reuse = mapper.createObjectNode(); + } + final String[] fieldNames = info.getFieldNames(); + final TypeInformation[] fieldTypes = info.getFieldTypes(); + + // validate the row + if (row.getArity() != fieldNames.length) { + throw new IllegalStateException(String.format( + "Number of elements in the row '%s' is different from number of field names: %d", row, fieldNames.length)); + } + + for (int i = 0; i < fieldNames.length; i++) { + final String name = fieldNames[i]; + + final JsonNode fieldConverted = convert(reuse, reuse.get(name), fieldTypes[i], row.getField(i)); + reuse.set(name, fieldConverted); + } + + return reuse; + } + + private JsonNode convert(ContainerNode container, JsonNode reuse, TypeInformation info, Object object) { + if (info == Types.VOID || object == null) { + return container.nullNode(); + } else if (info == Types.BOOLEAN) { + return container.booleanNode((Boolean) object); + } else if (info == Types.STRING) { + return container.textNode((String) object); + } else if (info == Types.BIG_DEC) { + // convert decimal if necessary + if (object instanceof BigDecimal) { + return container.numberNode((BigDecimal) object); + } + return container.numberNode(BigDecimal.valueOf(((Number) object).doubleValue())); + } else if (info == Types.BIG_INT) { + // convert integer if necessary + if (object instanceof BigInteger) { + return container.numberNode((BigInteger) object); + } + return container.numberNode(BigInteger.valueOf(((Number) object).longValue())); + } else if (info == Types.SQL_DATE) { + return container.textNode(object.toString()); + } else if (info == Types.SQL_TIME) { + final Time time = (Time) object; + // strip milliseconds if possible + if (time.getTime() % 1000 > 0) { + return container.textNode(timeFormatWithMillis.format(time)); + } + return container.textNode(timeFormat.format(time)); + } else if (info == Types.SQL_TIMESTAMP) { + return container.textNode(timestampFormat.format((Timestamp) object)); + } else if (info instanceof RowTypeInfo) { + if (reuse != null && reuse instanceof ObjectNode) { + return convertRow((ObjectNode) reuse, (RowTypeInfo) info, (Row) object); + } else { + return convertRow(null, (RowTypeInfo) info, (Row) object); + } + } else if (info instanceof ObjectArrayTypeInfo) { + if (reuse != null && reuse instanceof ArrayNode) { + return convertObjectArray((ArrayNode) reuse, ((ObjectArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } else { + return convertObjectArray(null, ((ObjectArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } + } else if (info instanceof BasicArrayTypeInfo) { + if (reuse != null && reuse instanceof ArrayNode) { + return convertObjectArray((ArrayNode) reuse, ((BasicArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } else { + return convertObjectArray(null, ((BasicArrayTypeInfo) info).getComponentInfo(), (Object[]) object); + } + } else if (info instanceof PrimitiveArrayTypeInfo && ((PrimitiveArrayTypeInfo) info).getComponentType() == Types.BYTE) { + return container.binaryNode((byte[]) object); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return mapper.valueToTree(object); + } catch (IllegalArgumentException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for object: " + object, e); + } + } + } + + private ArrayNode convertObjectArray(ArrayNode reuse, TypeInformation info, Object[] array) { + if (reuse == null) { + reuse = mapper.createArrayNode(); + } else { + reuse.removeAll(); + } + + for (Object object : array) { + reuse.add(convert(reuse, null, info, object)); + } + return reuse; + } + + public Counter getCounter() { + return counter; + } + + public void setCounter(Counter counter) { + this.counter = counter; + } +} diff --git a/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafkaJsonTableSink.java b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafkaJsonTableSink.java new file mode 100644 index 000000000..af6e54854 --- /dev/null +++ b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafkaJsonTableSink.java @@ -0,0 +1,65 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSink; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.utils.TableConnectorUtils; +import org.apache.flink.types.Row; + +import java.util.Optional; +import java.util.Properties; + +/** + * @author: chuixue + * @create: 2019-11-05 11:54 + * @description: + **/ +public class CustomerKafkaJsonTableSink extends KafkaTableSink { + + protected SerializationSchema schema; + + + public CustomerKafkaJsonTableSink(TableSchema schema, + String topic, + Properties properties, + Optional> partitioner, + SerializationSchema serializationSchema) { + + super(schema, topic, properties, partitioner, serializationSchema); + this.schema = serializationSchema; + } + + @Override + protected SinkFunction createKafkaProducer(String topic, Properties properties, SerializationSchema serializationSchema, Optional> optional) { + return new CustomerFlinkKafkaProducer(topic, serializationSchema, properties); + } + + @Override + public void emitDataStream(DataStream dataStream) { + SinkFunction kafkaProducer = createKafkaProducer(topic, properties, schema, partitioner); + // always enable flush on checkpoint to achieve at-least-once if query runs with checkpointing enabled. + //kafkaProducer.setFlushOnCheckpoint(true); + dataStream.addSink(kafkaProducer).name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); + } +} diff --git a/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java new file mode 100644 index 000000000..44bf9f98b --- /dev/null +++ b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/KafkaSink.java @@ -0,0 +1,144 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka; + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.kafka.table.KafkaSinkTableInfo; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.connectors.kafka.KafkaTableSinkBase; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.sinks.RetractStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; + +import java.util.Optional; +import java.util.Properties; + +/** + * @author: chuixue + * @create: 2019-11-05 11:45 + * @description: + **/ +public class KafkaSink implements RetractStreamTableSink, IStreamSinkGener { + + protected String[] fieldNames; + + protected TypeInformation[] fieldTypes; + + protected String topic; + + protected int parallelism; + + protected Properties properties; + + /** Serialization schema for encoding records to Kafka. */ + protected SerializationSchema serializationSchema; + + /** The schema of the table. */ + private TableSchema schema; + + /** Partitioner to select Kafka partition for each item. */ + protected Optional> partitioner; + + @Override + public KafkaSink genStreamSink(TargetTableInfo targetTableInfo) { + KafkaSinkTableInfo kafkaSinkTableInfo = (KafkaSinkTableInfo) targetTableInfo; + this.topic = kafkaSinkTableInfo.getTopic(); + + properties = new Properties(); + properties.setProperty("bootstrap.servers", kafkaSinkTableInfo.getBootstrapServers()); + + for (String key : kafkaSinkTableInfo.getKafkaParamKeys()) { + properties.setProperty(key, kafkaSinkTableInfo.getKafkaParam(key)); + } + this.partitioner = Optional.of(new FlinkFixedPartitioner<>()); + this.fieldNames = kafkaSinkTableInfo.getFields(); + TypeInformation[] types = new TypeInformation[kafkaSinkTableInfo.getFields().length]; + for (int i = 0; i < kafkaSinkTableInfo.getFieldClasses().length; i++) { + types[i] = TypeInformation.of(kafkaSinkTableInfo.getFieldClasses()[i]); + } + this.fieldTypes = types; + + TableSchema.Builder schemaBuilder = TableSchema.builder(); + for (int i=0;i getRecordType() { + return new RowTypeInfo(fieldTypes, fieldNames); + } + + @Override + public void emitDataStream(DataStream> dataStream) { + KafkaTableSinkBase kafkaTableSink = new CustomerKafkaJsonTableSink( + schema, + topic, + properties, + partitioner, + serializationSchema + ); + + DataStream ds = dataStream.map((Tuple2 record) -> { + return record.f1; + }).returns(getOutputType().getTypeAt(1)).setParallelism(parallelism); + + kafkaTableSink.emitDataStream(ds); + } + + @Override + public TupleTypeInfo> getOutputType() { + return new TupleTypeInfo(org.apache.flink.table.api.Types.BOOLEAN(), new RowTypeInfo(fieldTypes, fieldNames)); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public TypeInformation[] getFieldTypes() { + return fieldTypes; + } + + @Override + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { + this.fieldNames = fieldNames; + this.fieldTypes = fieldTypes; + return this; + } +} diff --git a/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java new file mode 100644 index 000000000..f633c8112 --- /dev/null +++ b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkParser.java @@ -0,0 +1,63 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka.table; + +import com.dtstack.flink.sql.table.AbsTableParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; + +import java.util.Map; + +/** + * @author: chuixue + * @create: 2019-11-05 11:46 + * @description: + **/ +public class KafkaSinkParser extends AbsTableParser { + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) throws Exception { + KafkaSinkTableInfo kafkaSinkTableInfo = new KafkaSinkTableInfo(); + kafkaSinkTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafkaSinkTableInfo); + kafkaSinkTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase()))); + + if (props.get(KafkaSinkTableInfo.SINK_DATA_TYPE) != null) { + kafkaSinkTableInfo.setSinkDataType(props.get(KafkaSinkTableInfo.SINK_DATA_TYPE).toString()); + } + +// if (props.get(KafkaSinkTableInfo.FIELD_DELINITER) != null) { +// kafka11SinkTableInfo.setFieldDelimiter(props.get(KafkaSinkTableInfo.FIELD_DELINITER).toString()); +// } + + kafkaSinkTableInfo.setBootstrapServers(MathUtil.getString(props.get(KafkaSinkTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase()))); + kafkaSinkTableInfo.setTopic(MathUtil.getString(props.get(KafkaSinkTableInfo.TOPIC_KEY.toLowerCase()))); + + Integer parallelism = MathUtil.getIntegerVal(props.get(KafkaSinkTableInfo.PARALLELISM_KEY.toLowerCase())); + kafkaSinkTableInfo.setParallelism(parallelism); + + for (String key : props.keySet()) { + if (!key.isEmpty() && key.startsWith("kafka.")) { + kafkaSinkTableInfo.addKafkaParam(key.substring(6), props.get(key).toString()); + } + } + kafkaSinkTableInfo.check(); + + return kafkaSinkTableInfo; + } +} diff --git a/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java new file mode 100644 index 000000000..1d23932c1 --- /dev/null +++ b/kafka/kafka-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -0,0 +1,93 @@ +/* + * 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 com.dtstack.flink.sql.sink.kafka.table; + +import com.dtstack.flink.sql.table.TargetTableInfo; +import com.google.common.base.Preconditions; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +/** + * @author: chuixue + * @create: 2019-11-05 11:46 + * @description: + **/ +public class KafkaSinkTableInfo extends TargetTableInfo { + //version + private static final String CURR_TYPE = "kafka"; + + public KafkaSinkTableInfo(){ + super.setType(CURR_TYPE); + } + public static final String BOOTSTRAPSERVERS_KEY = "bootstrapServers"; + + public static final String TOPIC_KEY = "topic"; + + private String bootstrapServers; + + public Map kafkaParam = new HashMap(); + + private String topic; + + + public void addKafkaParam(String key,String value){ + kafkaParam.put(key,value); + } + + public String getKafkaParam(String key){ + return kafkaParam.get(key); + } + + public Set getKafkaParamKeys(){ + return kafkaParam.keySet(); + } + + + public String getBootstrapServers() { + return bootstrapServers; + } + + public void setBootstrapServers(String bootstrapServers) { + this.bootstrapServers = bootstrapServers; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + + @Override + public boolean check() { + Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(topic, "kafka of topic is required"); + //Preconditions.checkNotNull(kafkaParam.get("groupId"), "kafka of groupId is required"); + return false; + } + + @Override + public String getType() { + return super.getType(); + } +} diff --git a/kafka/kafka-source/pom.xml b/kafka/kafka-source/pom.xml new file mode 100644 index 000000000..55ca950fb --- /dev/null +++ b/kafka/kafka-source/pom.xml @@ -0,0 +1,81 @@ + + + + sql.kafka + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.source.kafka + jar + + kafka-source + http://maven.apache.org + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + org.slf4j + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java new file mode 100644 index 000000000..ef65f2882 --- /dev/null +++ b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -0,0 +1,266 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import com.dtstack.flink.sql.source.kafka.metric.KafkaTopicPartitionLagMetric; +import com.dtstack.flink.sql.table.TableInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Strings; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; +import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.types.Row; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static com.dtstack.flink.sql.metric.MetricConstant.*; + +/** + * @author: chuixue + * @create: 2019-11-05 10:57 + * @description: + **/ +public class CustomerJsonDeserialization extends AbsDeserialization { + + private static final Logger LOG = LoggerFactory.getLogger(CustomerJsonDeserialization.class); + + private static final long serialVersionUID = 2385115520960444192L; + + private static int dirtyDataFrequency = 1000; + + private final ObjectMapper objectMapper = new ObjectMapper(); + + /** Type information describing the result type. */ + private final TypeInformation typeInfo; + + /** Field names to parse. Indices match fieldTypes indices. */ + private final String[] fieldNames; + + /** Types to parse fields as. Indices match fieldNames indices. */ + private final TypeInformation[] fieldTypes; + + private AbstractFetcher fetcher; + + private boolean firstMsg = true; + + private Map nodeAndJsonNodeMapping = Maps.newHashMap(); + + private Map rowAndFieldMapping; + + private List fieldExtraInfos; + + public CustomerJsonDeserialization(TypeInformation typeInfo, Map rowAndFieldMapping, List fieldExtraInfos){ + this.typeInfo = typeInfo; + this.fieldNames = ((RowTypeInfo) typeInfo).getFieldNames(); + this.fieldTypes = ((RowTypeInfo) typeInfo).getFieldTypes(); + this.rowAndFieldMapping= rowAndFieldMapping; + this.fieldExtraInfos = fieldExtraInfos; + } + + @Override + public Row deserialize(byte[] message) throws IOException { + + if(firstMsg){ + try { + registerPtMetric(fetcher); + } catch (Exception e) { + LOG.error("register topic partition metric error.", e); + } + + firstMsg = false; + } + + try { + JsonNode root = objectMapper.readTree(message); + + if (numInRecord.getCount() % dirtyDataFrequency == 0) { + LOG.info(root.toString()); + } + + numInRecord.inc(); + numInBytes.inc(message.length); + + parseTree(root, null); + Row row = new Row(fieldNames.length); + + for (int i = 0; i < fieldNames.length; i++) { + JsonNode node = getIgnoreCase(fieldNames[i]); + TableInfo.FieldExtraInfo fieldExtraInfo = fieldExtraInfos.get(i); + + if (node == null) { + if (fieldExtraInfo != null && fieldExtraInfo.getNotNull()) { + throw new IllegalStateException("Failed to find field with name '" + + fieldNames[i] + "'."); + } else { + row.setField(i, null); + } + } else { + // Read the value as specified type + + Object value = convert(node, fieldTypes[i]); + row.setField(i, value); + } + } + + numInResolveRecord.inc(); + return row; + } catch (Exception e) { + //add metric of dirty data + if (dirtyDataCounter.getCount() % dirtyDataFrequency == 0) { + LOG.info("dirtyData: " + new String(message)); + LOG.error("" , e); + } + dirtyDataCounter.inc(); + return null; + }finally { + nodeAndJsonNodeMapping.clear(); + } + } + + public JsonNode getIgnoreCase(String key) { + String nodeMappingKey = rowAndFieldMapping.getOrDefault(key, key); + JsonNode node = nodeAndJsonNodeMapping.get(nodeMappingKey); + if(node == null){ + return null; + } + + JsonNodeType nodeType = node.getNodeType(); + + if (nodeType==JsonNodeType.ARRAY){ + throw new IllegalStateException("Unsupported type information array .") ; + } + + return node; + } + + private void parseTree(JsonNode jsonNode, String prefix){ + + Iterator iterator = jsonNode.fieldNames(); + while (iterator.hasNext()){ + String next = iterator.next(); + JsonNode child = jsonNode.get(next); + String nodeKey = getNodeKey(prefix, next); + + if (child.isValueNode()){ + nodeAndJsonNodeMapping.put(nodeKey, child); + }else if(child.isArray()){ + nodeAndJsonNodeMapping.put(nodeKey, new TextNode(child.toString())); + }else { + parseTree(child, nodeKey); + } + } + } + + private String getNodeKey(String prefix, String nodeName){ + if(Strings.isNullOrEmpty(prefix)){ + return nodeName; + } + + return prefix + "." + nodeName; + } + + public void setFetcher(AbstractFetcher fetcher) { + this.fetcher = fetcher; + } + + protected void registerPtMetric(AbstractFetcher fetcher) throws Exception { + + Field consumerThreadField = fetcher.getClass().getSuperclass().getDeclaredField("consumerThread"); + consumerThreadField.setAccessible(true); + KafkaConsumerThread consumerThread = (KafkaConsumerThread) consumerThreadField.get(fetcher); + + Field hasAssignedPartitionsField = consumerThread.getClass().getDeclaredField("hasAssignedPartitions"); + hasAssignedPartitionsField.setAccessible(true); + + //wait until assignedPartitions + + boolean hasAssignedPartitions = (boolean) hasAssignedPartitionsField.get(consumerThread); + + if(!hasAssignedPartitions){ + throw new RuntimeException("wait 50 secs, but not assignedPartitions"); + } + + Field consumerField = consumerThread.getClass().getDeclaredField("consumer"); + consumerField.setAccessible(true); + + KafkaConsumer kafkaConsumer = (KafkaConsumer) consumerField.get(consumerThread); + Field subscriptionStateField = kafkaConsumer.getClass().getDeclaredField("subscriptions"); + subscriptionStateField.setAccessible(true); + + //topic partitions lag + SubscriptionState subscriptionState = (SubscriptionState) subscriptionStateField.get(kafkaConsumer); + Set assignedPartitions = subscriptionState.assignedPartitions(); + for(TopicPartition topicPartition : assignedPartitions){ + MetricGroup metricGroup = getRuntimeContext().getMetricGroup().addGroup(DT_TOPIC_GROUP, topicPartition.topic()) + .addGroup(DT_PARTITION_GROUP, topicPartition.partition() + ""); + metricGroup.gauge(DT_TOPIC_PARTITION_LAG_GAUGE, new KafkaTopicPartitionLagMetric(subscriptionState, topicPartition)); + } + + } + + private static String partitionLagMetricName(TopicPartition tp) { + return tp + ".records-lag"; + } + + private Object convert(JsonNode node, TypeInformation info) { + if (info.getTypeClass().equals(Types.BOOLEAN.getTypeClass())) { + return node.asBoolean(); + } else if (info.getTypeClass().equals(Types.STRING.getTypeClass())) { + return node.asText(); + } else if (info.getTypeClass().equals(Types.SQL_DATE.getTypeClass())) { + return Date.valueOf(node.asText()); + } else if (info.getTypeClass().equals(Types.SQL_TIME.getTypeClass())) { + // local zone + return Time.valueOf(node.asText()); + } else if (info.getTypeClass().equals(Types.SQL_TIMESTAMP.getTypeClass())) { + // local zone + return Timestamp.valueOf(node.asText()); + } else { + // for types that were specified without JSON schema + // e.g. POJOs + try { + return objectMapper.treeToValue(node, info.getTypeClass()); + } catch (JsonProcessingException e) { + throw new IllegalStateException("Unsupported type information '" + info + "' for node: " + node); + } + } + } +} diff --git a/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafkaConsumer.java b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafkaConsumer.java new file mode 100644 index 000000000..7d35a35b3 --- /dev/null +++ b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerKafkaConsumer.java @@ -0,0 +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 com.dtstack.flink.sql.source.kafka; + +import com.dtstack.flink.sql.source.AbsDeserialization; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; +import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; +import org.apache.flink.streaming.connectors.kafka.config.OffsetCommitMode; +import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.types.Row; +import org.apache.flink.util.SerializedValue; + +import java.util.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * @author: chuixue + * @create: 2019-11-05 10:58 + * @description: + **/ +public class CustomerKafkaConsumer extends FlinkKafkaConsumer { + + private static final long serialVersionUID = -2265366268827807739L; + + private CustomerJsonDeserialization customerJsonDeserialization; + + public CustomerKafkaConsumer(String topic, AbsDeserialization valueDeserializer, Properties props) { + super(Arrays.asList(topic.split(",")), valueDeserializer, props); + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + } + + public CustomerKafkaConsumer(Pattern subscriptionPattern, AbsDeserialization valueDeserializer, Properties props) { + super(subscriptionPattern, valueDeserializer, props); + this.customerJsonDeserialization = (CustomerJsonDeserialization) valueDeserializer; + } + + @Override + public void run(SourceContext sourceContext) throws Exception { + customerJsonDeserialization.setRuntimeContext(getRuntimeContext()); + customerJsonDeserialization.initMetric(); + super.run(sourceContext); + } + + @Override + protected AbstractFetcher createFetcher(SourceContext sourceContext, Map assignedPartitionsWithInitialOffsets, SerializedValue> watermarksPeriodic, SerializedValue> watermarksPunctuated, StreamingRuntimeContext runtimeContext, OffsetCommitMode offsetCommitMode, MetricGroup consumerMetricGroup, boolean useMetrics) throws Exception { + AbstractFetcher fetcher = super.createFetcher(sourceContext, assignedPartitionsWithInitialOffsets, watermarksPeriodic, watermarksPunctuated, runtimeContext, offsetCommitMode, consumerMetricGroup, useMetrics); + customerJsonDeserialization.setFetcher(fetcher); + return fetcher; + } +} diff --git a/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.java new file mode 100644 index 000000000..c26d99b62 --- /dev/null +++ b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/KafkaSource.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 com.dtstack.flink.sql.source.kafka; + +import com.dtstack.flink.sql.source.IStreamSourceGener; +import com.dtstack.flink.sql.source.kafka.table.KafkaSourceTableInfo; +import com.dtstack.flink.sql.table.SourceTableInfo; +import com.dtstack.flink.sql.util.DtStringUtil; +import com.dtstack.flink.sql.util.PluginUtil; +import org.apache.commons.lang3.BooleanUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.java.StreamTableEnvironment; +import org.apache.flink.types.Row; + +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Pattern; + +/** + * @author: chuixue + * @create: 2019-11-05 10:55 + * @description: + **/ +public class KafkaSource implements IStreamSourceGener

{ + + private static final String SOURCE_OPERATOR_NAME_TPL = "${topic}_${table}"; + + /** + * Get kafka data source, you need to provide the data field names, data types + * If you do not specify auto.offset.reset, the default use groupoffset + * + * @param sourceTableInfo + * @return + */ + @SuppressWarnings("rawtypes") + @Override + public Table genStreamSource(SourceTableInfo sourceTableInfo, StreamExecutionEnvironment env, StreamTableEnvironment tableEnv) { + + KafkaSourceTableInfo kafkaSourceTableInfo = (KafkaSourceTableInfo) sourceTableInfo; + String topicName = kafkaSourceTableInfo.getTopic(); + + Properties props = new Properties(); + props.setProperty("bootstrap.servers", kafkaSourceTableInfo.getBootstrapServers()); + if (DtStringUtil.isJosn(kafkaSourceTableInfo.getOffsetReset())) { + props.setProperty("auto.offset.reset", "none"); + } else { + props.setProperty("auto.offset.reset", kafkaSourceTableInfo.getOffsetReset()); + } + if (StringUtils.isNotBlank(kafkaSourceTableInfo.getGroupId())) { + props.setProperty("group.id", kafkaSourceTableInfo.getGroupId()); + } + + TypeInformation[] types = new TypeInformation[kafkaSourceTableInfo.getFields().length]; + for (int i = 0; i < kafkaSourceTableInfo.getFieldClasses().length; i++) { + types[i] = TypeInformation.of(kafkaSourceTableInfo.getFieldClasses()[i]); + } + + TypeInformation typeInformation = new RowTypeInfo(types, kafkaSourceTableInfo.getFields()); + + FlinkKafkaConsumer kafkaSrc; + if (BooleanUtils.isTrue(kafkaSourceTableInfo.getTopicIsPattern())) { + kafkaSrc = new CustomerKafkaConsumer(Pattern.compile(topicName), + new CustomerJsonDeserialization(typeInformation, kafkaSourceTableInfo.getPhysicalFields(), kafkaSourceTableInfo.getFieldExtraInfoList()), props); + } else { + kafkaSrc = new CustomerKafkaConsumer(topicName, + new CustomerJsonDeserialization(typeInformation, kafkaSourceTableInfo.getPhysicalFields(), kafkaSourceTableInfo.getFieldExtraInfoList()), props); + } + + //earliest,latest + if ("earliest".equalsIgnoreCase(kafkaSourceTableInfo.getOffsetReset())) { + kafkaSrc.setStartFromEarliest(); + } else if (DtStringUtil.isJosn(kafkaSourceTableInfo.getOffsetReset())) {// {"0":12312,"1":12321,"2":12312} + try { + Properties properties = PluginUtil.jsonStrToObject(kafkaSourceTableInfo.getOffsetReset(), Properties.class); + Map offsetMap = PluginUtil.ObjectToMap(properties); + Map specificStartupOffsets = new HashMap<>(); + for (Map.Entry entry : offsetMap.entrySet()) { + specificStartupOffsets.put(new KafkaTopicPartition(topicName, Integer.valueOf(entry.getKey())), Long.valueOf(entry.getValue().toString())); + } + kafkaSrc.setStartFromSpecificOffsets(specificStartupOffsets); + } catch (Exception e) { + throw new RuntimeException("not support offsetReset type:" + kafkaSourceTableInfo.getOffsetReset()); + } + } else { + kafkaSrc.setStartFromLatest(); + } + + String fields = StringUtils.join(kafkaSourceTableInfo.getFields(), ","); + String sourceOperatorName = SOURCE_OPERATOR_NAME_TPL.replace("${topic}", topicName).replace("${table}", sourceTableInfo.getName()); + + DataStreamSource kafkaSource = env.addSource(kafkaSrc, sourceOperatorName, typeInformation); + Integer parallelism = kafkaSourceTableInfo.getParallelism(); + if (parallelism != null) { + kafkaSource.setParallelism(parallelism); + } + return tableEnv.fromDataStream(kafkaSource, fields); + } +} diff --git a/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/metric/KafkaTopicPartitionLagMetric.java b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/metric/KafkaTopicPartitionLagMetric.java new file mode 100644 index 000000000..7810056d7 --- /dev/null +++ b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/metric/KafkaTopicPartitionLagMetric.java @@ -0,0 +1,46 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.metric; + +import org.apache.flink.metrics.Gauge; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.requests.IsolationLevel; + +/** + * @author: chuixue + * @create: 2019-11-05 11:09 + * @description: + **/ +public class KafkaTopicPartitionLagMetric implements Gauge { + + private SubscriptionState subscriptionState; + + private TopicPartition tp; + + public KafkaTopicPartitionLagMetric(SubscriptionState subscriptionState, TopicPartition tp){ + this.subscriptionState = subscriptionState; + this.tp = tp; + } + + @Override + public Long getValue() { + return subscriptionState.partitionLag(tp, IsolationLevel.READ_UNCOMMITTED); + } +} diff --git a/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java new file mode 100644 index 000000000..ba18ccf5b --- /dev/null +++ b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -0,0 +1,97 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.table; + +import com.dtstack.flink.sql.table.AbsSourceParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.ClassUtil; +import com.dtstack.flink.sql.util.MathUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * @author: chuixue + * @create: 2019-11-05 11:08 + * @description: + **/ +public class KafkaSourceParser extends AbsSourceParser { + + private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceParser.class); + + private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; + + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)(\\s+NOT\\s+NULL)?$"); + + static { + keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); + + keyHandlerMap.put(KAFKA_NEST_FIELD_KEY, KafkaSourceParser::dealNestField); + } + + /** + * add parser for alias field + * + * @param matcher + * @param tableInfo + */ + static void dealNestField(Matcher matcher, TableInfo tableInfo) { + String physicalField = matcher.group(1); + String fieldType = matcher.group(3); + String mappingField = matcher.group(4); + Class fieldClass = ClassUtil.stringConvertClass(fieldType); + boolean notNull = matcher.group(5) != null; + TableInfo.FieldExtraInfo fieldExtraInfo = new TableInfo.FieldExtraInfo(); + fieldExtraInfo.setNotNull(notNull); + + tableInfo.addPhysicalMappings(mappingField, physicalField); + tableInfo.addField(mappingField); + tableInfo.addFieldClass(fieldClass); + tableInfo.addFieldType(fieldType); + tableInfo.addFieldExtraInfo(fieldExtraInfo); + if (LOG.isInfoEnabled()) { + LOG.info(physicalField + "--->" + mappingField + " Class: " + fieldClass.toString()); + } + } + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) throws Exception { + KafkaSourceTableInfo kafkaSourceTableInfo = new KafkaSourceTableInfo(); + kafkaSourceTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, kafkaSourceTableInfo); + + kafkaSourceTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(KafkaSourceTableInfo.PARALLELISM_KEY.toLowerCase()))); + String bootstrapServer = MathUtil.getString(props.get(KafkaSourceTableInfo.BOOTSTRAPSERVERS_KEY.toLowerCase())); + if (bootstrapServer == null || bootstrapServer.trim().equals("")) { + throw new Exception("BootstrapServers can not be empty!"); + } else { + kafkaSourceTableInfo.setBootstrapServers(bootstrapServer); + } + kafkaSourceTableInfo.setGroupId(MathUtil.getString(props.get(KafkaSourceTableInfo.GROUPID_KEY.toLowerCase()))); + kafkaSourceTableInfo.setTopic(MathUtil.getString(props.get(KafkaSourceTableInfo.TOPIC_KEY.toLowerCase()))); + kafkaSourceTableInfo.setOffsetReset(MathUtil.getString(props.get(KafkaSourceTableInfo.OFFSETRESET_KEY.toLowerCase()))); + kafkaSourceTableInfo.setTopicIsPattern(MathUtil.getBoolean(props.get(KafkaSourceTableInfo.TOPICISPATTERN_KEY.toLowerCase()))); + kafkaSourceTableInfo.setTimeZone(MathUtil.getString(props.get(KafkaSourceTableInfo.TIME_ZONE_KEY.toLowerCase()))); + kafkaSourceTableInfo.check(); + return kafkaSourceTableInfo; + } +} diff --git a/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java new file mode 100644 index 000000000..33b704ac0 --- /dev/null +++ b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -0,0 +1,128 @@ +/* + * 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 com.dtstack.flink.sql.source.kafka.table; + +import com.dtstack.flink.sql.table.SourceTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * @author: chuixue + * @create: 2019-11-05 11:09 + * @description: + **/ +public class KafkaSourceTableInfo extends SourceTableInfo { + + //version + private static final String CURR_TYPE = "kafka"; + + public static final String BOOTSTRAPSERVERS_KEY = "bootstrapServers"; + + public static final String TOPIC_KEY = "topic"; + + public static final String GROUPID_KEY = "groupId"; + + public static final String OFFSETRESET_KEY = "offsetReset"; + + public static final String TOPICISPATTERN_KEY = "topicIsPattern"; + + private String bootstrapServers; + + private String topic; + + private String groupId; + + private Boolean topicIsPattern = false; + + public Boolean getTopicIsPattern() { + return topicIsPattern; + } + + public void setTopicIsPattern(Boolean topicIsPattern) { + if (topicIsPattern == null) return; + + this.topicIsPattern = topicIsPattern; + } + + //latest, earliest + private String offsetReset = "latest"; + + private String offset; + + public KafkaSourceTableInfo() { + super.setType(CURR_TYPE); + } + + + public String getBootstrapServers() { + return bootstrapServers; + } + + public void setBootstrapServers(String bootstrapServers) { + this.bootstrapServers = bootstrapServers; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public String getGroupId() { + return groupId; + } + + public void setGroupId(String groupId) { + this.groupId = groupId; + } + + public String getOffsetReset() { + return offsetReset; + } + + public void setOffsetReset(String offsetReset) { + if (offsetReset == null) { + return; + } + + this.offsetReset = offsetReset; + } + + public String getOffset() { + return offset; + } + + public void setOffset(String offset) { + this.offset = offset; + } + + @Override + public boolean check() { + Preconditions.checkNotNull(bootstrapServers, "kafka of bootstrapServers is required"); + Preconditions.checkNotNull(topic, "kafka of topic is required"); + return false; + } + + @Override + public String getType() { +// return super.getType() + SOURCE_SUFFIX; + return super.getType(); + } +} diff --git a/kafka/pom.xml b/kafka/pom.xml new file mode 100644 index 000000000..772671ff6 --- /dev/null +++ b/kafka/pom.xml @@ -0,0 +1,43 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + + 4.0.0 + + sql.kafka + pom + + + kafka-source + kafka-sink + + + + + org.apache.flink + flink-connector-kafka_2.11 + ${flink.version} + + + + junit + junit + 3.8.1 + test + + + + com.dtstack.flink + sql.core + 1.0-SNAPSHOT + provided + + + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index dba6b4898..15086c07e 100644 --- a/pom.xml +++ b/pom.xml @@ -13,6 +13,7 @@ kafka09 kafka10 kafka11 + kafka mysql hbase elasticsearch5 From aa6c7f2b29b2b2cac9c4f08da6bfb39969ed5e6f Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Tue, 5 Nov 2019 20:55:55 +0800 Subject: [PATCH 451/470] do not handle brackets in quotes --- .../main/java/com/dtstack/flink/sql/util/DtStringUtil.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index 8cf59f757..38af382c4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -83,10 +83,10 @@ public static List splitIgnoreQuota(String str, char delimiter){ } else if (c == '\'' && '\\' != flag && !inQuotes) { inSingleQuotes = !inSingleQuotes; b.append(c); - } else if (c == '(') { + } else if (c == '(' && !inSingleQuotes && !inQuotes) { bracketLeftNum++; b.append(c); - } else if (c == ')') { + } else if (c == ')' && !inSingleQuotes && !inQuotes) { bracketLeftNum--; b.append(c); } else { From 8722736796b6fae606f23b24d861102e3bca36a8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BF=AE=E7=AB=B9?= Date: Tue, 5 Nov 2019 21:05:04 +0800 Subject: [PATCH 452/470] =?UTF-8?q?=E6=9C=80=E6=96=B0kafka=E7=89=88?= =?UTF-8?q?=E6=9C=AC=E6=94=AF=E6=8C=81=E5=B5=8C=E5=A5=97json=E5=92=8C?= =?UTF-8?q?=E6=95=B0=E7=BB=84=E7=B1=BB=E5=9E=8B=E5=AD=97=E6=AE=B5=E8=A7=A3?= =?UTF-8?q?=E6=9E=90?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 2 +- docs/kafkaSource.md | 2 +- .../kafka/CustomerJsonDeserialization.java | 40 +++++++++++++------ .../source/kafka/table/KafkaSourceParser.java | 2 +- 4 files changed, 30 insertions(+), 16 deletions(-) diff --git a/README.md b/README.md index 3c4c527a8..459d7b79a 100644 --- a/README.md +++ b/README.md @@ -17,7 +17,7 @@ 2.修复yarnPer模式提交失败的异常。 # 已支持 - * 源表:kafka 0.9,1.x版本 + * 源表:kafka 0.9,1.x及以上版本 * 维表:mysql,SQlServer,oracle,hbase,mongo,redis,cassandra,serversocket * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra,console diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index 8f519fa2c..bb516ff62 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -21,7 +21,7 @@ CREATE TABLE tableName( ``` ## 2.支持的版本 - kafka08,kafka09,kafka10,kafka11 + kafka08,kafka09,kafka10,kafka11及以上版本 **kafka读取和写入的版本必须一致,否则会有兼容性错误。** ## 3.表结构定义 diff --git a/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index ef65f2882..6d3e57957 100644 --- a/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -30,6 +30,7 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; @@ -157,21 +158,27 @@ public Row deserialize(byte[] message) throws IOException { public JsonNode getIgnoreCase(String key) { String nodeMappingKey = rowAndFieldMapping.getOrDefault(key, key); - JsonNode node = nodeAndJsonNodeMapping.get(nodeMappingKey); - if(node == null){ - return null; - } - - JsonNodeType nodeType = node.getNodeType(); - - if (nodeType==JsonNodeType.ARRAY){ - throw new IllegalStateException("Unsupported type information array .") ; - } - - return node; + return nodeAndJsonNodeMapping.get(nodeMappingKey); } private void parseTree(JsonNode jsonNode, String prefix){ + if (jsonNode.isArray()) { + ArrayNode array = (ArrayNode) jsonNode; + for (int i = 0; i < array.size(); i++) { + JsonNode child = array.get(i); + String nodeKey = getNodeKey(prefix, i); + + if (child.isValueNode()) { + nodeAndJsonNodeMapping.put(nodeKey, child); + } else { + if (rowAndFieldMapping.containsValue(nodeKey)) { + nodeAndJsonNodeMapping.put(nodeKey, child); + } + parseTree(child, nodeKey); + } + } + return; + } Iterator iterator = jsonNode.fieldNames(); while (iterator.hasNext()){ @@ -182,7 +189,7 @@ private void parseTree(JsonNode jsonNode, String prefix){ if (child.isValueNode()){ nodeAndJsonNodeMapping.put(nodeKey, child); }else if(child.isArray()){ - nodeAndJsonNodeMapping.put(nodeKey, new TextNode(child.toString())); + parseTree(child, nodeKey); }else { parseTree(child, nodeKey); } @@ -197,6 +204,13 @@ private String getNodeKey(String prefix, String nodeName){ return prefix + "." + nodeName; } + private String getNodeKey(String prefix, int i) { + if (Strings.isNullOrEmpty(prefix)) { + return "[" + i + "]"; + } + return prefix + "[" + i + "]"; + } + public void setFetcher(AbstractFetcher fetcher) { this.fetcher = fetcher; } diff --git a/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java index ba18ccf5b..a99f49298 100644 --- a/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java +++ b/kafka/kafka-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceParser.java @@ -40,7 +40,7 @@ public class KafkaSourceParser extends AbsSourceParser { private static final String KAFKA_NEST_FIELD_KEY = "nestFieldKey"; - private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\w+\\.)*\\w+)\\s+(\\w+)\\s+AS\\s+(\\w+)(\\s+NOT\\s+NULL)?$"); + private static Pattern kafkaNestFieldKeyPattern = Pattern.compile("(?i)((@*\\S+\\.)*\\S+)\\s+(\\w+)\\s+AS\\s+(\\w+)(\\s+NOT\\s+NULL)?$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); From 851edecf6b5a0f1dc73d9dc8c0fab21026bafd5e Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 6 Nov 2019 14:03:22 +0800 Subject: [PATCH 453/470] plugin load mode code opt --- .../main/java/com/dtstack/flink/sql/Main.java | 6 ++--- .../flink/sql/enums/EPluginLoadMode.java | 23 ++++++++++++++++ .../flink/sql/enums/PluginLoadMode.java | 15 ----------- .../com/dtstack/flink/sql/option/Options.java | 4 +-- .../dtstack/flink/sql/util/PluginUtil.java | 26 ++++++++++--------- .../perjob/PerJobClusterClientBuilder.java | 10 ++++--- .../sql/launcher/perjob/PerJobSubmitter.java | 5 ---- 7 files changed, 48 insertions(+), 41 deletions(-) create mode 100644 core/src/main/java/com/dtstack/flink/sql/enums/EPluginLoadMode.java delete mode 100644 core/src/main/java/com/dtstack/flink/sql/enums/PluginLoadMode.java diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index c3ad1a97e..6f31dc5c7 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -25,7 +25,7 @@ import com.dtstack.flink.sql.constrant.ConfigConstrant; import com.dtstack.flink.sql.enums.ClusterMode; import com.dtstack.flink.sql.enums.ECacheType; -import com.dtstack.flink.sql.enums.PluginLoadMode; +import com.dtstack.flink.sql.enums.EPluginLoadMode; import com.dtstack.flink.sql.environment.MyLocalStreamEnvironment; import com.dtstack.flink.sql.exec.FlinkSQLExec; import com.dtstack.flink.sql.option.OptionParser; @@ -296,14 +296,14 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en } private static URL buildSourceAndSinkPathByLoadMode(String type, String suffix, String localSqlPluginPath, String remoteSqlPluginPath, String pluginLoadMode) throws Exception { - if (StringUtils.equalsIgnoreCase(pluginLoadMode, PluginLoadMode.classpath.name())) { + if (StringUtils.equalsIgnoreCase(pluginLoadMode, EPluginLoadMode.CLASSPATH.name())) { return PluginUtil.getRemoteJarFilePath(type, suffix, remoteSqlPluginPath, localSqlPluginPath); } return PluginUtil.getLocalJarFilePath(type, suffix, localSqlPluginPath); } private static URL buildSidePathByLoadMode(String type, String operator, String suffix, String localSqlPluginPath, String remoteSqlPluginPath, String pluginLoadMode) throws Exception { - if (StringUtils.equalsIgnoreCase(pluginLoadMode, PluginLoadMode.classpath.name())) { + if (StringUtils.equalsIgnoreCase(pluginLoadMode, EPluginLoadMode.CLASSPATH.name())) { return PluginUtil.getRemoteSideJarFilePath(type, operator, suffix, remoteSqlPluginPath, localSqlPluginPath); } return PluginUtil.getLocalSideJarFilePath(type, operator, suffix, localSqlPluginPath); diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/EPluginLoadMode.java b/core/src/main/java/com/dtstack/flink/sql/enums/EPluginLoadMode.java new file mode 100644 index 000000000..add47aaf2 --- /dev/null +++ b/core/src/main/java/com/dtstack/flink/sql/enums/EPluginLoadMode.java @@ -0,0 +1,23 @@ +package com.dtstack.flink.sql.enums; + +/** + * + * CLASSPATH: plugin jar depends on each machine node. + * SHIPFILE: plugin jar only depends on the client submitted by the task. + * + */ +public enum EPluginLoadMode { + + CLASSPATH(0), + SHIPFILE(1); + + private int type; + + EPluginLoadMode(int type){ + this.type = type; + } + + public int getType(){ + return this.type; + } +} diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/PluginLoadMode.java b/core/src/main/java/com/dtstack/flink/sql/enums/PluginLoadMode.java deleted file mode 100644 index 5dd6f2f42..000000000 --- a/core/src/main/java/com/dtstack/flink/sql/enums/PluginLoadMode.java +++ /dev/null @@ -1,15 +0,0 @@ -package com.dtstack.flink.sql.enums; - -public enum PluginLoadMode { - classpath(0),shipfile(1); - - private int type; - - PluginLoadMode(int type){ - this.type = type; - } - - public int getType(){ - return this.type; - } -} diff --git a/core/src/main/java/com/dtstack/flink/sql/option/Options.java b/core/src/main/java/com/dtstack/flink/sql/option/Options.java index 7100a17f2..a653aa42e 100644 --- a/core/src/main/java/com/dtstack/flink/sql/option/Options.java +++ b/core/src/main/java/com/dtstack/flink/sql/option/Options.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.option; import com.dtstack.flink.sql.enums.ClusterMode; -import com.dtstack.flink.sql.enums.PluginLoadMode; +import com.dtstack.flink.sql.enums.EPluginLoadMode; /** @@ -73,7 +73,7 @@ public class Options { private String yarnSessionConf = "{}"; @OptionRequired(description = "plugin load mode, by classpath or shipfile") - private String pluginLoadMode = PluginLoadMode.classpath.name(); + private String pluginLoadMode = EPluginLoadMode.CLASSPATH.name(); public String getMode() { return mode; diff --git a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java index 96a7f5768..569bf1dd3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java @@ -110,35 +110,37 @@ public static Properties stringToProperties(String str) throws IOException{ } public static URL getRemoteJarFilePath(String pluginType, String tableType, String remoteSqlRootDir, String localSqlPluginPath) throws Exception { - String dirName = pluginType + tableType.toLowerCase(); - String prefix = String.format("%s-%s", pluginType, tableType.toLowerCase()); - String jarPath = localSqlPluginPath + SP + dirName; - String jarName = getCoreJarFileName(jarPath, prefix); - return new URL("file:" + remoteSqlRootDir + SP + dirName + SP + jarName); + return buildFinalJarFilePath(pluginType, tableType, remoteSqlRootDir, localSqlPluginPath); } public static URL getLocalJarFilePath(String pluginType, String tableType, String localSqlPluginPath) throws Exception { + return buildFinalJarFilePath(pluginType, tableType, null, localSqlPluginPath); + } + + public static URL buildFinalJarFilePath(String pluginType, String tableType, String remoteSqlRootDir, String localSqlPluginPath) throws Exception { String dirName = pluginType + tableType.toLowerCase(); String prefix = String.format("%s-%s", pluginType, tableType.toLowerCase()); String jarPath = localSqlPluginPath + SP + dirName; String jarName = getCoreJarFileName(jarPath, prefix); - return new URL("file:" + jarPath + SP + jarName); + String sqlRootDir = remoteSqlRootDir == null ? localSqlPluginPath : remoteSqlRootDir; + return new URL("file:" + sqlRootDir + SP + dirName + SP + jarName); } public static URL getRemoteSideJarFilePath(String pluginType, String sideOperator, String tableType, String remoteSqlRootDir, String localSqlPluginPath) throws Exception { - String dirName = pluginType + sideOperator + tableType.toLowerCase(); - String prefix = String.format("%s-%s-%s", pluginType, sideOperator, tableType.toLowerCase()); - String jarPath = localSqlPluginPath + SP + dirName; - String jarName = getCoreJarFileName(jarPath, prefix); - return new URL("file:" + remoteSqlRootDir + SP + dirName + SP + jarName); + return buildFinalSideJarFilePath(pluginType, sideOperator, tableType, remoteSqlRootDir, localSqlPluginPath); } public static URL getLocalSideJarFilePath(String pluginType, String sideOperator, String tableType, String localSqlPluginPath) throws Exception { + return buildFinalSideJarFilePath(pluginType, sideOperator, tableType, null, localSqlPluginPath); + } + + public static URL buildFinalSideJarFilePath(String pluginType, String sideOperator, String tableType, String remoteSqlRootDir, String localSqlPluginPath) throws Exception { String dirName = pluginType + sideOperator + tableType.toLowerCase(); String prefix = String.format("%s-%s-%s", pluginType, sideOperator, tableType.toLowerCase()); String jarPath = localSqlPluginPath + SP + dirName; String jarName = getCoreJarFileName(jarPath, prefix); - return new URL("file:" + jarPath + SP + jarName); + String sqlRootDir = remoteSqlRootDir == null ? localSqlPluginPath : remoteSqlRootDir; + return new URL("file:" + sqlRootDir + SP + dirName + SP + jarName); } public static String upperCaseFirstChar(String str){ diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java index 82d56d2dc..3b58bf845 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java @@ -18,6 +18,7 @@ package com.dtstack.flink.sql.launcher.perjob; +import com.dtstack.flink.sql.enums.EPluginLoadMode; import com.dtstack.flink.sql.launcher.YarnConfLoader; import com.dtstack.flink.sql.option.Options; import org.apache.commons.lang3.StringUtils; @@ -47,8 +48,6 @@ */ public class PerJobClusterClientBuilder { - private static final String FLINK_PLUGIN_CLASSPATH_LOAD = "classpath"; - private YarnClient yarnClient; private YarnConfiguration yarnConf; @@ -93,11 +92,14 @@ public AbstractYarnClusterDescriptor createPerJobClusterDescriptor(Properties co } // classpath , all node need contain plugin jar String pluginLoadMode = launcherOptions.getPluginLoadMode(); - if (StringUtils.equalsIgnoreCase(pluginLoadMode, FLINK_PLUGIN_CLASSPATH_LOAD)){ + if (StringUtils.equalsIgnoreCase(pluginLoadMode, EPluginLoadMode.CLASSPATH.name())) { fillJobGraphClassPath(jobGraph); - } else { + } else if (StringUtils.equalsIgnoreCase(pluginLoadMode, EPluginLoadMode.SHIPFILE.name())) { List pluginPaths = getPluginPathToShipFiles(jobGraph); shipFiles.addAll(pluginPaths); + } else { + throw new IllegalArgumentException("Unsupported plugin loading mode " + pluginLoadMode + + " Currently only classpath and shipfile are supported."); } clusterDescriptor.addShipFiles(shipFiles); diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java index e4735621a..55b55be2f 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobSubmitter.java @@ -22,7 +22,6 @@ import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.io.Charsets; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.core.fs.Path; @@ -31,13 +30,9 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - -import java.net.MalformedURLException; -import java.net.URL; import java.net.URLDecoder; import java.util.Arrays; import java.util.List; -import java.util.Map; import java.util.Properties; /** From 815732d29a371d72e60d4b71a9e3392b118479ee Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 6 Nov 2019 14:06:15 +0800 Subject: [PATCH 454/470] add licens --- .../flink/sql/enums/EPluginLoadMode.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/enums/EPluginLoadMode.java b/core/src/main/java/com/dtstack/flink/sql/enums/EPluginLoadMode.java index add47aaf2..6cb027ac3 100644 --- a/core/src/main/java/com/dtstack/flink/sql/enums/EPluginLoadMode.java +++ b/core/src/main/java/com/dtstack/flink/sql/enums/EPluginLoadMode.java @@ -1,3 +1,21 @@ +/* + * 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 com.dtstack.flink.sql.enums; /** From a13e584dda5572b0790409cb3e2614e949c8c7b8 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 6 Nov 2019 14:11:33 +0800 Subject: [PATCH 455/470] test code --- .../sql/launcher/PluginLoadModeTest.java | 55 +++++++++++++++++++ 1 file changed, 55 insertions(+) create mode 100644 launcher/src/main/test/java/com/dtstack/flink/sql/launcher/PluginLoadModeTest.java diff --git a/launcher/src/main/test/java/com/dtstack/flink/sql/launcher/PluginLoadModeTest.java b/launcher/src/main/test/java/com/dtstack/flink/sql/launcher/PluginLoadModeTest.java new file mode 100644 index 000000000..0a153d018 --- /dev/null +++ b/launcher/src/main/test/java/com/dtstack/flink/sql/launcher/PluginLoadModeTest.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 com.dtstack.flink.sql.launcher; + + +/** + * yarnPer提交任务时指定pluginLoadMode + * Date: 2019/11/6 + * Company: www.dtstack.com + * @author maqi + */ +public class PluginLoadModeTest { + public static void testShipfileMode() throws Exception { + String[] sql = new String[]{"--mode", "yarnPer", "-sql", "/Users/maqi/tmp/json/group_tmp4.txt", "-name", "PluginLoadModeTest", + "-localSqlPluginPath", "/Users/maqi/code/dtstack/dt-center-flinkStreamSQL/plugins", + "-remoteSqlPluginPath", "/Users/maqi/code/dtstack/dt-center-flinkStreamSQL/plugins", + "-allowNonRestoredState", "false", "-flinkconf", "/Users/maqi/tmp/flink-1.8.1/conf", + "-confProp", "{\"sql.checkpoint.cleanup.mode\":\"false\",\"sql.checkpoint.interval\":10000,\"time.characteristic\":\"EventTime\"}", + "-yarnconf", "/Users/maqi/tmp/hadoop", "-flinkJarPath", "/Users/maqi/tmp/flink-1.8.1/lib", "-queue", "c", "-pluginLoadMode", "shipfile"}; + System.setProperty("HADOOP_USER_NAME", "admin"); + LauncherMain.main(sql); + } + + public static void testClasspathMode() throws Exception { + String[] sql = new String[]{"--mode", "yarnPer", "-sql", "/Users/maqi/tmp/json/group_tmp4.txt", "-name", "PluginLoadModeTest", + "-localSqlPluginPath", "/Users/maqi/code/dtstack/dt-center-flinkStreamSQL/plugins", + "-remoteSqlPluginPath", "/opt/dtstack/180_flinkplugin/sqlplugin", + "-allowNonRestoredState", "false", "-flinkconf", "/Users/maqi/tmp/flink-1.8.1/conf", + "-confProp", "{\"sql.checkpoint.cleanup.mode\":\"false\",\"sql.checkpoint.interval\":10000,\"time.characteristic\":\"EventTime\"}", + "-yarnconf", "/Users/maqi/tmp/hadoop", "-flinkJarPath", "/Users/maqi/tmp/flink-1.8.1/lib", "-queue", "c", "-pluginLoadMode", "classpath"}; + System.setProperty("HADOOP_USER_NAME", "admin"); + LauncherMain.main(sql); + } + + public static void main(String[] args) throws Exception { + testShipfileMode(); +// testClasspathMode(); + } +} From 57df9a8213f6e56f1c81b6dbe76c96803e1392bb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BF=AE=E7=AB=B9?= Date: Wed, 6 Nov 2019 16:58:26 +0800 Subject: [PATCH 456/470] =?UTF-8?q?=E6=95=B4=E7=90=86postgresql=E7=BB=B4?= =?UTF-8?q?=E8=A1=A8=E5=92=8C=E7=BB=93=E6=9E=9C=E8=A1=A8=E4=BB=A3=E7=A0=81?= =?UTF-8?q?=EF=BC=8Cmerge=20to=20v1.5.0=5Fdev?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 8 +- docs/postgresqlSide.md | 80 ++++++++++++ docs/postgresqlSink.md | 55 +++++++++ pom.xml | 1 + postgresql/pom.xml | 44 +++++++ postgresql/postgresql-side/pom.xml | 38 ++++++ .../postgresql-all-side/pom.xml | 92 ++++++++++++++ .../side/postgresql/PostgresqlAllReqRow.java | 70 +++++++++++ .../postgresql/PostgresqlAllSideInfo.java | 41 +++++++ .../postgresql-async-side/pom.xml | 92 ++++++++++++++ .../postgresql/PostgresqlAsyncReqRow.java | 74 ++++++++++++ .../postgresql/PostgresqlAsyncSideInfo.java | 42 +++++++ .../postgresql-side-core/pom.xml | 18 +++ .../table/PostgresqlSideParser.java | 45 +++++++ postgresql/postgresql-sink/pom.xml | 90 ++++++++++++++ .../sql/sink/postgresql/PostgresqlSink.java | 114 ++++++++++++++++++ .../table/PostgresqlSinkParser.java | 63 ++++++++++ .../postgresql/table/PostgresqlTableInfo.java | 76 ++++++++++++ 18 files changed, 1041 insertions(+), 2 deletions(-) create mode 100644 docs/postgresqlSide.md create mode 100644 docs/postgresqlSink.md create mode 100644 postgresql/pom.xml create mode 100644 postgresql/postgresql-side/pom.xml create mode 100644 postgresql/postgresql-side/postgresql-all-side/pom.xml create mode 100644 postgresql/postgresql-side/postgresql-all-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAllReqRow.java create mode 100644 postgresql/postgresql-side/postgresql-all-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAllSideInfo.java create mode 100644 postgresql/postgresql-side/postgresql-async-side/pom.xml create mode 100644 postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncReqRow.java create mode 100644 postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncSideInfo.java create mode 100644 postgresql/postgresql-side/postgresql-side-core/pom.xml create mode 100644 postgresql/postgresql-side/postgresql-side-core/src/main/java/com/dtstack/flink/sql/side/postgresql/table/PostgresqlSideParser.java create mode 100644 postgresql/postgresql-sink/pom.xml create mode 100644 postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/PostgresqlSink.java create mode 100644 postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/table/PostgresqlSinkParser.java create mode 100644 postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/table/PostgresqlTableInfo.java diff --git a/README.md b/README.md index 93edde5c3..f41baebd5 100644 --- a/README.md +++ b/README.md @@ -10,8 +10,8 @@ # 已支持 * 源表:kafka 0.9,1.x版本 - * 维表:mysql,SQlServer,oracle,hbase,mongo,redis,cassandra - * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra + * 维表:mysql, SQlServer, oracle, hbase, mongo, redis, cassandra, kudu, postgresql + * 结果表:mysql, SQlServer, oracle, hbase, elasticsearch5.x, mongo, redis, cassandra, kudu, postgresql # 后续开发计划 * 增加SQL支持CEP @@ -154,6 +154,8 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * [mongo 结果表插件](docs/mongoSink.md) * [redis 结果表插件](docs/redisSink.md) * [cassandra 结果表插件](docs/cassandraSink.md) +* [kudu 结果表插件](docs/kuduSink.md) +* [postgresql 结果表插件](docs/postgresqlSink.md) ### 2.3 维表插件 * [hbase 维表插件](docs/hbaseSide.md) @@ -161,6 +163,8 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * [mongo 维表插件](docs/mongoSide.md) * [redis 维表插件](docs/redisSide.md) * [cassandra 维表插件](docs/cassandraSide.md) +* [kudu 维表插件](docs/kuduSide.md) +* [postgresql 维表插件](docs/postgresqlSide.md) ## 3 性能指标(新增) diff --git a/docs/postgresqlSide.md b/docs/postgresqlSide.md new file mode 100644 index 000000000..68d10b869 --- /dev/null +++ b/docs/postgresqlSide.md @@ -0,0 +1,80 @@ + +## 1.格式: +``` + CREATE TABLE tableName( + colName cloType, + ... + PRIMARY KEY(keyInfo), + PERIOD FOR SYSTEM_TIME + )WITH( + type='postgresql', + url='jdbcUrl', + userName='dbUserName', + password='dbPwd', + tableName='tableName', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' + ); +``` + +# 2.支持版本 + postgresql-8.2+ + +## 3.表结构定义 + + |参数名称|含义| + |----|---| + | tableName | 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同)| + | colName | 列名称| + | colType | 列类型 [colType支持的类型](colType.md)| + | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| + | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| + +## 4.参数 + + |参数名称|含义|是否必填|默认值| + |----|---|---|----| + | type | 表明维表的类型[postgresql] |是|| + | url | 连接postgresql数据库 jdbcUrl |是|| + | userName | postgresql连接用户名 |是|| + | password | postgresql连接密码|是|| + | tableName | postgresql表名称|是|| + | tableName | postgresql 的表名称|是|| + | cache | 维表缓存策略(NONE/LRU/ALL)|否|NONE| + | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| + + ---------- + > 缓存策略 + * NONE: 不做内存缓存 + * LRU: + * cacheSize: 缓存的条目数量 + * cacheTTLMs:缓存的过期时间(ms) + + +## 5.样例 +``` +create table sideTable( + channel varchar, + xccount int, + PRIMARY KEY(channel), + PERIOD FOR SYSTEM_TIME + )WITH( + type='postgresql', + url='jdbc:postgresql://localhost:9001/test?sslmode=disable', + userName='dtstack', + password='abc123', + tableName='sidetest', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' + ); + + +``` + + diff --git a/docs/postgresqlSink.md b/docs/postgresqlSink.md new file mode 100644 index 000000000..da09c4c34 --- /dev/null +++ b/docs/postgresqlSink.md @@ -0,0 +1,55 @@ +## 1.格式: +``` +CREATE TABLE tableName( + colName colType, + ... + colNameX colType + )WITH( + type ='postgresql', + url ='jdbcUrl', + userName ='userName', + password ='pwd', + tableName ='tableName', + parallelism ='parllNum' + ); + +``` + +## 2.支持版本 + postgresql-8.2+ + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName| 在 sql 中使用的名称;即注册到flink-table-env上的名称| +| colName | 列名称| +| colType | 列类型 [colType支持的类型](colType.md)| + +## 4.参数: + +|参数名称|含义|是否必填|默认值| +|----|----|----|----| +| type |表明 输出表类型[postgresql]|是|| +| url | 连接postgresql数据库 jdbcUrl |是|| +| userName | postgresql连接用户名 |是|| +| password | postgresql连接密码|是|| +| tableName | postgresqll表名称|是|| +| parallelism | 并行度设置|否|1| +| isUpsert | 使用upsert模式插入数据(版本9.5之后才支持upsert) |否|false +| keyField | 设置更新主键字段名(isupsert为true时为必填项)|否| + +## 5.样例: +``` +CREATE TABLE MyResult( + channel VARCHAR, + pv VARCHAR + )WITH( + type ='postgresql', + url ='jdbc:postgresql://localhost:9001/test?sslmode=disable', + userName ='dtstack', + password ='abc123', + tableName ='pv2', + parallelism ='1' + ) + ``` diff --git a/pom.xml b/pom.xml index 0bb26671b..4f4efa79b 100644 --- a/pom.xml +++ b/pom.xml @@ -24,6 +24,7 @@ oracle cassandra kudu + postgresql diff --git a/postgresql/pom.xml b/postgresql/pom.xml new file mode 100644 index 000000000..53398b87b --- /dev/null +++ b/postgresql/pom.xml @@ -0,0 +1,44 @@ + + + + flink.sql + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + pom + sql.postgresql + + + postgresql-side + postgresql-sink + + + 1.0-SNAPSHOT + 42.2.2 + + + + + junit + junit + 3.8.1 + test + + + + com.dtstack.flink + sql.core + ${sql.core.version} + provided + + + org.postgresql + postgresql + ${postgresql.version} + + + diff --git a/postgresql/postgresql-side/pom.xml b/postgresql/postgresql-side/pom.xml new file mode 100644 index 000000000..0e3723ea6 --- /dev/null +++ b/postgresql/postgresql-side/pom.xml @@ -0,0 +1,38 @@ + + + + sql.postgresql + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.postgresql + 1.0-SNAPSHOT + postgresql-side + pom + + + postgresql-side-core + postgresql-async-side + postgresql-all-side + + + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.side.rdb + ${rdb.side.version} + + + + + diff --git a/postgresql/postgresql-side/postgresql-all-side/pom.xml b/postgresql/postgresql-side/postgresql-all-side/pom.xml new file mode 100644 index 000000000..d076d9300 --- /dev/null +++ b/postgresql/postgresql-side/postgresql-all-side/pom.xml @@ -0,0 +1,92 @@ + + + + sql.side.postgresql + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.all.postgresql + postgresql-all-side + + jar + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.side.postgresql.core + ${sql.side.postgresql.core.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + + diff --git a/postgresql/postgresql-side/postgresql-all-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAllReqRow.java b/postgresql/postgresql-side/postgresql-all-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAllReqRow.java new file mode 100644 index 000000000..6d68cfdca --- /dev/null +++ b/postgresql/postgresql-side/postgresql-all-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAllReqRow.java @@ -0,0 +1,70 @@ +/* + * 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 com.dtstack.flink.sql.side.postgresql; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; +import com.dtstack.flink.sql.util.DtStringUtil; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.util.List; +import java.util.Map; + +/** + * side operator with cache for all(period reload) + * Date: 2019-08-11 + * Company: mmg + * + * @author tcm + */ + +public class PostgresqlAllReqRow extends RdbAllReqRow { + + private static final long serialVersionUID = 2098635140857937717L; + + private static final Logger LOG = LoggerFactory.getLogger(PostgresqlAllReqRow.class); + + private static final String POSTGRESQL_DRIVER = "org.postgresql.Driver"; + + public PostgresqlAllReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new PostgresqlAllSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + @Override + public Connection getConn(String dbURL, String userName, String password) { + try { + Class.forName(POSTGRESQL_DRIVER); + //add param useCursorFetch=true + Map addParams = Maps.newHashMap(); + addParams.put("useCursorFetch", "true"); + String targetDbUrl = DtStringUtil.addJdbcParam(dbURL, addParams, true); + return DriverManager.getConnection(targetDbUrl, userName, password); + } catch (Exception e) { + LOG.error("", e); + throw new RuntimeException("", e); + } + } +} diff --git a/postgresql/postgresql-side/postgresql-all-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAllSideInfo.java b/postgresql/postgresql-side/postgresql-all-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAllSideInfo.java new file mode 100644 index 000000000..d383ee46d --- /dev/null +++ b/postgresql/postgresql-side/postgresql-all-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAllSideInfo.java @@ -0,0 +1,41 @@ +/* + * 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 com.dtstack.flink.sql.side.postgresql; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.all.RdbAllSideInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; + +import java.util.List; + +/** + * Reason: + * Date: 2019-08-11 + * Company: mmg + * + * @author tcm + */ + +public class PostgresqlAllSideInfo extends RdbAllSideInfo { + public PostgresqlAllSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } +} diff --git a/postgresql/postgresql-side/postgresql-async-side/pom.xml b/postgresql/postgresql-side/postgresql-async-side/pom.xml new file mode 100644 index 000000000..13296e7f3 --- /dev/null +++ b/postgresql/postgresql-side/postgresql-async-side/pom.xml @@ -0,0 +1,92 @@ + + + + sql.side.postgresql + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.async.postgresql + + postgresql-async-side + + jar + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.side.postgresql.core + ${sql.side.postgresql.core.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + diff --git a/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncReqRow.java b/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncReqRow.java new file mode 100644 index 000000000..02b333819 --- /dev/null +++ b/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncReqRow.java @@ -0,0 +1,74 @@ +/* + * 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 com.dtstack.flink.sql.side.postgresql; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.async.RdbAsyncReqRow; +import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; +import io.vertx.core.Vertx; +import io.vertx.core.VertxOptions; +import io.vertx.core.json.JsonObject; +import io.vertx.ext.jdbc.JDBCClient; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +/** + * Date: 2019-08-11 + * Company: mmg + * + * @author tcm + */ + +public class PostgresqlAsyncReqRow extends RdbAsyncReqRow { + + private static final Logger LOG = LoggerFactory.getLogger(PostgresqlAsyncReqRow.class); + + private final static String POSTGRESQL_DRIVER = "org.postgresql.Driver"; + + public PostgresqlAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(new PostgresqlAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); + } + + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + JsonObject pgClientConfig = new JsonObject(); + RdbSideTableInfo rdbSideTableInfo = (RdbSideTableInfo) sideInfo.getSideTableInfo(); + pgClientConfig.put("url", rdbSideTableInfo.getUrl()) + .put("driver_class", POSTGRESQL_DRIVER) + .put("max_pool_size", DEFAULT_MAX_DB_CONN_POOL_SIZE) + .put("user", rdbSideTableInfo.getUserName()) + .put("password", rdbSideTableInfo.getPassword()); + + VertxOptions vo = new VertxOptions(); + vo.setEventLoopPoolSize(DEFAULT_VERTX_EVENT_LOOP_POOL_SIZE); + vo.setWorkerPoolSize(DEFAULT_VERTX_WORKER_POOL_SIZE); + Vertx vertx = Vertx.vertx(vo); + setRdbSQLClient(JDBCClient.createNonShared(vertx, pgClientConfig)); + } + +} diff --git a/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncSideInfo.java b/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncSideInfo.java new file mode 100644 index 000000000..1d89f4894 --- /dev/null +++ b/postgresql/postgresql-side/postgresql-async-side/src/main/java/com/dtstack/flink/sql/side/postgresql/PostgresqlAsyncSideInfo.java @@ -0,0 +1,42 @@ +/* + * 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 com.dtstack.flink.sql.side.postgresql; + +import com.dtstack.flink.sql.side.FieldInfo; +import com.dtstack.flink.sql.side.JoinInfo; +import com.dtstack.flink.sql.side.SideTableInfo; +import com.dtstack.flink.sql.side.rdb.async.RdbAsyncSideInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; + +import java.util.List; + +/** + * Reason: + * Date: 2019-08-11 + * Company: mmg + * + * @author tcm + */ + +public class PostgresqlAsyncSideInfo extends RdbAsyncSideInfo { + + public PostgresqlAsyncSideInfo(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { + super(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo); + } +} diff --git a/postgresql/postgresql-side/postgresql-side-core/pom.xml b/postgresql/postgresql-side/postgresql-side-core/pom.xml new file mode 100644 index 000000000..6120767eb --- /dev/null +++ b/postgresql/postgresql-side/postgresql-side-core/pom.xml @@ -0,0 +1,18 @@ + + + + sql.side.postgresql + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.side.postgresql.core + 1.0-SNAPSHOT + jar + postgresql-side-core + + diff --git a/postgresql/postgresql-side/postgresql-side-core/src/main/java/com/dtstack/flink/sql/side/postgresql/table/PostgresqlSideParser.java b/postgresql/postgresql-side/postgresql-side-core/src/main/java/com/dtstack/flink/sql/side/postgresql/table/PostgresqlSideParser.java new file mode 100644 index 000000000..faee2c704 --- /dev/null +++ b/postgresql/postgresql-side/postgresql-side-core/src/main/java/com/dtstack/flink/sql/side/postgresql/table/PostgresqlSideParser.java @@ -0,0 +1,45 @@ +/* + * 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 com.dtstack.flink.sql.side.postgresql.table; + +import com.dtstack.flink.sql.side.rdb.table.RdbSideParser; +import com.dtstack.flink.sql.table.TableInfo; + +import java.util.Map; + +/** + * Reason: + * Date: 2019-08-11 + * Company: mmg + * + * @author tcm + */ + +public class PostgresqlSideParser extends RdbSideParser { + + private static final String CURR_TYPE = "postgresql"; + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + TableInfo pgTableInfo = super.getTableInfo(tableName, fieldsInfo, props); + pgTableInfo.setType(CURR_TYPE); + return pgTableInfo; + } +} diff --git a/postgresql/postgresql-sink/pom.xml b/postgresql/postgresql-sink/pom.xml new file mode 100644 index 000000000..d65368358 --- /dev/null +++ b/postgresql/postgresql-sink/pom.xml @@ -0,0 +1,90 @@ + + + sql.postgresql + com.dtstack.flink + 1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + sql.sink.postgresql + jar + + postgresql-sink + http://maven.apache.org + + + 1.0-SNAPSHOT + + + + + com.dtstack.flink + sql.sink.rdb + ${sql.sink.rdb.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 1.4 + + + package + + shade + + + + + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + maven-antrun-plugin + 1.2 + + + copy-resources + + package + + run + + + + + + + + + + + + + + + + + + diff --git a/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/PostgresqlSink.java b/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/PostgresqlSink.java new file mode 100644 index 000000000..eaafdc595 --- /dev/null +++ b/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/PostgresqlSink.java @@ -0,0 +1,114 @@ +/* + * 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 com.dtstack.flink.sql.sink.postgresql; + + +import com.dtstack.flink.sql.sink.IStreamSinkGener; +import com.dtstack.flink.sql.sink.postgresql.table.PostgresqlTableInfo; +import com.dtstack.flink.sql.sink.rdb.RdbSink; +import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; +import com.dtstack.flink.sql.table.TargetTableInfo; +import org.apache.commons.lang3.StringUtils; + +import java.util.List; +import java.util.Map; + +/** + * Date: 2019-08-22 + * Company: mmg + * + * @author tcm + */ + +public class PostgresqlSink extends RdbSink implements IStreamSinkGener { + + private static final String POSTGRESQL_DRIVER = "org.postgresql.Driver"; + + private boolean isUpsert; + + private String keyField; + + public PostgresqlSink() { + } + + @Override + public RdbSink genStreamSink(TargetTableInfo targetTableInfo) { + PostgresqlTableInfo pgTableInfo = (PostgresqlTableInfo) targetTableInfo; + this.isUpsert = pgTableInfo.isUpsert(); + this.keyField = pgTableInfo.getKeyField(); + super.genStreamSink(targetTableInfo); + return this; + } + + @Override + public RetractJDBCOutputFormat getOutputFormat() { + return new RetractJDBCOutputFormat(); + } + + @Override + public void buildSql(String tableName, List fields) { + buildInsertSql(tableName, fields); + } + + @Override + public String buildUpdateSql(String tableName, + List fieldNames, + Map> realIndexes, + List fullField) { + return null; + } + + private void buildInsertSql(String tableName, List fields) { + StringBuffer sqlBuffer = new StringBuffer(); + + sqlBuffer.append("insert into ".concat(tableName) + .concat(" (") + .concat(StringUtils.join(fields, ",")) + .concat(") ") + ); + sqlBuffer.append("values ("); + StringBuffer upsertFields = new StringBuffer(); + for (String fieldName : fields) { + sqlBuffer.append("?,"); + if (this.isUpsert) { + if (fieldName.equals(this.keyField)) { + continue; + } + upsertFields.append(String.format("%s=excluded.%s,", fieldName, fieldName)); + } + } + sqlBuffer.deleteCharAt(sqlBuffer.length() - 1); + sqlBuffer.append(")"); + + if (this.isUpsert) { + upsertFields.deleteCharAt(upsertFields.length() - 1); + sqlBuffer.append(" ON conflict(".concat(keyField).concat(")")); + sqlBuffer.append(" DO UPDATE SET "); + sqlBuffer.append(upsertFields); + } + this.sql = sqlBuffer.toString(); + } + + @Override + public String getDriverName() { + return POSTGRESQL_DRIVER; + } + +} diff --git a/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/table/PostgresqlSinkParser.java b/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/table/PostgresqlSinkParser.java new file mode 100644 index 000000000..f773b5a5c --- /dev/null +++ b/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/table/PostgresqlSinkParser.java @@ -0,0 +1,63 @@ +/* + * 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 com.dtstack.flink.sql.sink.postgresql.table; + +import com.dtstack.flink.sql.sink.rdb.table.RdbSinkParser; +import com.dtstack.flink.sql.table.TableInfo; +import com.dtstack.flink.sql.util.MathUtil; +import org.apache.commons.lang3.StringUtils; + +import java.util.Map; + +/** + * Date: 2019-08-22 + * Company: mmg + * + * @author tcm + */ + +public class PostgresqlSinkParser extends RdbSinkParser { + private static final String CURR_TYPE = "postgresql"; + + @Override + public TableInfo getTableInfo(String tableName, String fieldsInfo, Map props) { + PostgresqlTableInfo pgTableInfo = new PostgresqlTableInfo(); + pgTableInfo.setName(tableName); + parseFieldsInfo(fieldsInfo, pgTableInfo); + + pgTableInfo.setParallelism(MathUtil.getIntegerVal(props.get(PostgresqlTableInfo.PARALLELISM_KEY.toLowerCase()))); + pgTableInfo.setUrl(MathUtil.getString(props.get(PostgresqlTableInfo.URL_KEY.toLowerCase()))); + pgTableInfo.setTableName(MathUtil.getString(props.get(PostgresqlTableInfo.TABLE_NAME_KEY.toLowerCase()))); + pgTableInfo.setUserName(MathUtil.getString(props.get(PostgresqlTableInfo.USER_NAME_KEY.toLowerCase()))); + pgTableInfo.setPassword(MathUtil.getString(props.get(PostgresqlTableInfo.PASSWORD_KEY.toLowerCase()))); + pgTableInfo.setBatchSize(MathUtil.getIntegerVal(props.get(PostgresqlTableInfo.BATCH_SIZE_KEY.toLowerCase()))); + pgTableInfo.setBatchWaitInterval(MathUtil.getLongVal(props.get(PostgresqlTableInfo.BATCH_WAIT_INTERVAL_KEY.toLowerCase()))); + pgTableInfo.setBufferSize(MathUtil.getString(props.get(PostgresqlTableInfo.BUFFER_SIZE_KEY.toLowerCase()))); + pgTableInfo.setFlushIntervalMs(MathUtil.getString(props.get(PostgresqlTableInfo.FLUSH_INTERVALMS_KEY.toLowerCase()))); + + pgTableInfo.setKeyField(MathUtil.getString(props.get(PostgresqlTableInfo.TABLE_KEY_FIELD.toLowerCase()))); + + String isUpsertStr = (String) props.get(PostgresqlTableInfo.TABLE_IS_UPSERT.toLowerCase()); + pgTableInfo.setUpsert(!StringUtils.isEmpty(isUpsertStr) && isUpsertStr.equals("true") ? true : false); + + pgTableInfo.check(); + return pgTableInfo; + } +} diff --git a/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/table/PostgresqlTableInfo.java b/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/table/PostgresqlTableInfo.java new file mode 100644 index 000000000..78df0de14 --- /dev/null +++ b/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/table/PostgresqlTableInfo.java @@ -0,0 +1,76 @@ +/* + * 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 com.dtstack.flink.sql.sink.postgresql.table; + +import com.dtstack.flink.sql.sink.rdb.table.RdbTableInfo; +import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; + +/** + * Date: 2019-08-22 + * Company: mmg + * + * @author tcm + */ + +public class PostgresqlTableInfo extends RdbTableInfo { + + public static final String TABLE_IS_UPSERT = "isUpsert"; + + public static final String TABLE_KEY_FIELD = "keyField"; + + private static final String CURR_TYPE = "postgresql"; + + private boolean isUpsert; + + private String keyField; + + + public PostgresqlTableInfo() { + setType(CURR_TYPE); + } + + public boolean isUpsert() { + return isUpsert; + } + + public void setUpsert(boolean upsert) { + isUpsert = upsert; + } + + public String getKeyField() { + return keyField; + } + + public void setKeyField(String keyField) { + this.keyField = keyField; + } + + @Override + public boolean check() { + Preconditions.checkNotNull(getUrl(), "postgresql field of URL is required"); + Preconditions.checkNotNull(getTableName(), "postgresql field of tableName is required"); + Preconditions.checkNotNull(getUserName(), "postgresql field of userName is required"); + Preconditions.checkNotNull(getPassword(), "postgresql field of password is required"); + if (isUpsert()) { + Preconditions.checkNotNull(getKeyField(), "postgresql field of keyField is required"); + } + return true; + } +} From d057118fa8c9a93abfc0a9f15aa9ebece6cdf6f9 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Wed, 6 Nov 2019 20:20:55 +0800 Subject: [PATCH 457/470] add timestampadd and timestampdiff --- core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java | 2 ++ core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java | 2 ++ 2 files changed, 4 insertions(+) diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 7f3cbd394..0d72b392b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -520,6 +520,8 @@ private SqlNode replaceSelectFieldName(SqlNode selectNode, HashBasedTable Date: Thu, 7 Nov 2019 11:02:35 +0800 Subject: [PATCH 458/470] pgsql --- .../dtstack/flink/sql/sink/postgresql/PostgresqlSink.java | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/PostgresqlSink.java b/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/PostgresqlSink.java index eaafdc595..663816ca0 100644 --- a/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/PostgresqlSink.java +++ b/postgresql/postgresql-sink/src/main/java/com/dtstack/flink/sql/sink/postgresql/PostgresqlSink.java @@ -63,15 +63,12 @@ public RetractJDBCOutputFormat getOutputFormat() { } @Override - public void buildSql(String tableName, List fields) { + public void buildSql(String scheam, String tableName, List fields) { buildInsertSql(tableName, fields); } @Override - public String buildUpdateSql(String tableName, - List fieldNames, - Map> realIndexes, - List fullField) { + public String buildUpdateSql(String schema, String tableName, List fieldNames, Map> realIndexes, List fullField) { return null; } From 8f18eee359206bfaf2a5336d3a45327354f30d42 Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Thu, 7 Nov 2019 11:49:27 +0800 Subject: [PATCH 459/470] readme --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 3c39f9bf1..d15399ff7 100644 --- a/README.md +++ b/README.md @@ -18,8 +18,8 @@ # 已支持 * 源表:kafka 0.9、0.10、0.11、1.x版本 - * 维表:mysql, SQlServer,oracle, hbase, mongo, redis, cassandra, kudu, postgresql - * 结果表:mysql, SQlServer, oracle, hbase, elasticsearch5.x, mongo, redis, cassandra, kudu, postgresql + * 维表:mysql, SQlServer,oracle, hbase, mongo, redis, cassandra, serversocket, kudu, postgresql + * 结果表:mysql, SQlServer, oracle, hbase, elasticsearch5.x, mongo, redis, cassandra, console, kudu, postgresql # 后续开发计划 * 增加SQL支持CEP From fa181f5e5c1ee29ea3f9f72259873ec75847535a Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Fri, 8 Nov 2019 16:51:48 +0800 Subject: [PATCH 460/470] =?UTF-8?q?=E6=B5=81=E8=A1=A8=E5=89=8D=E7=BC=80?= =?UTF-8?q?=E6=89=AB=E6=8F=8F=E7=9A=84=E6=96=B9=E5=BC=8Fjoin=20hbase?= =?UTF-8?q?=E7=BB=B4=E8=A1=A8=E6=97=B6=EF=BC=8C=E5=A6=82=E6=9E=9C=E6=89=AB?= =?UTF-8?q?=E6=8F=8F=E5=88=B0=E4=BA=86=E5=A4=9A=E6=9D=A1=E8=AE=B0=E5=BD=95?= =?UTF-8?q?=EF=BC=8C=E5=8F=AA=E4=BC=9A=E8=BE=93=E5=85=A5=E4=B8=80=E6=9D=A1?= =?UTF-8?q?=E8=AE=B0=E5=BD=95?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../hbase/rowkeydealer/PreRowKeyModeDealerDealer.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java index d79f67ec5..4d1282b94 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java @@ -88,6 +88,7 @@ private String dealOneRow(ArrayList> args, String rowKeyStr, } List cacheContent = Lists.newArrayList(); + List rowList = Lists.newArrayList(); for(List oneRow : args){ try { @@ -120,8 +121,7 @@ private String dealOneRow(ArrayList> args, String rowKeyStr, if (openCache) { cacheContent.add(sideVal); } - - resultFuture.complete(Collections.singleton(row)); + rowList.add(row); } } catch (Exception e) { resultFuture.complete(null); @@ -130,6 +130,10 @@ private String dealOneRow(ArrayList> args, String rowKeyStr, } } + if (rowList.size() > 0){ + resultFuture.complete(rowList); + } + if(openCache){ sideCache.putCache(rowKeyStr, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); } From 5fdd67207657b43eb1783dc2ebae25318ffa20c2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BF=AE=E7=AB=B9?= Date: Fri, 8 Nov 2019 21:06:55 +0800 Subject: [PATCH 461/470] =?UTF-8?q?=E4=BF=AE=E6=94=B9redis=E7=BB=93?= =?UTF-8?q?=E6=9E=9C=E8=A1=A8=E5=92=8C=E7=BB=B4=E8=A1=A8bug?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/redisSide.md | 4 ++ docs/redisSink.md | 10 +++- .../flink/sql/side/redis/RedisAllReqRow.java | 2 +- .../sql/side/redis/RedisAsyncReqRow.java | 57 +++++++++++-------- .../sql/side/redis/table/RedisSideReqRow.java | 44 +++++++++++++- .../sql/sink/redis/RedisOutputFormat.java | 8 ++- .../sql/sink/redis/table/RedisSinkParser.java | 14 +++++ 7 files changed, 110 insertions(+), 29 deletions(-) diff --git a/docs/redisSide.md b/docs/redisSide.md index 52d637b70..58249e0ad 100644 --- a/docs/redisSide.md +++ b/docs/redisSide.md @@ -11,6 +11,7 @@ password = 'redisPwd', database = 'dbName', tableName ='sideTableName', + redisType = '1', cache ='LRU', cacheSize ='10000', cacheTTLMs ='60000' @@ -35,6 +36,8 @@ | type | 表明维表的类型[hbase\|mysql\|redis]|是|| | url | redis 的地址;格式ip:port[,ip:port]|是|| | password | redis 的密码 |是|| +| redisType | redis模式(1 单机,2 哨兵, 3 集群)| 是 | +| masterName | 主节点名称(哨兵模式下为必填项) | 否 | | database | reids 的数据库地址|否|| | tableName | redis 的表名称|是|| | cache | 维表缓存策略(NONE/LRU/ALL)|否|NONE| @@ -60,6 +63,7 @@ create table sideTable( url='172.16.10.79:6379', password='abc123', database='0', + redisType = '1', tableName='sidetest', cache = 'LRU', cacheTTLMs='10000' diff --git a/docs/redisSink.md b/docs/redisSink.md index a3c80914e..4f314bc3e 100644 --- a/docs/redisSink.md +++ b/docs/redisSink.md @@ -9,6 +9,7 @@ CREATE TABLE tableName( url = 'ip:port', database ='dbName', password ='pwd', + redisType='1', tableName ='tableName', parallelism ='parllNum' ); @@ -32,12 +33,14 @@ redis5.0 |参数名称|含义|是否必填|默认值| |----|---|---|-----| -|type | 表明 输出表类型[mysql\|hbase\|elasticsearch\|redis\]|是|| +| type | 表明 输出表类型[mysql\|hbase\|elasticsearch\|redis\]|是|| | url | redis 的地址;格式ip:port[,ip:port]|是|| | password | redis 的密码 |是|| +| redisType | redis模式(1 单机,2 哨兵, 3 集群)| 是 | +| masterName | 主节点名称(哨兵模式下为必填项) | 否 | | database | reids 的数据库地址|否|| | tableName | redis 的表名称|是|| -|parallelism | 并行度设置|否|1| +| parallelism | 并行度设置|否|1| ## 5.样例: @@ -51,7 +54,8 @@ redis5.0 url='172.16.10.79:6379', password='abc123', database='0', - tableName='sinktoredis', + redisType='1', + tableName='sinktoredis' ); ``` \ No newline at end of file diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java index 54ecf8952..48d5e0f04 100644 --- a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java @@ -98,7 +98,7 @@ public void flatMap(Row row, Collector out) throws Exception { out.collect(null); } String columnName = sideInfo.getEqualFieldList().get(conValIndex); - inputParams.put(columnName, (String) equalObj); + inputParams.put(columnName, equalObj.toString()); } String key = buildKey(inputParams); diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index b5faff102..f3338aba8 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -39,7 +39,7 @@ import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; -import java.sql.Timestamp; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; @@ -123,12 +123,18 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except for (int i = 0; i < sideInfo.getEqualValIndex().size(); i++) { Integer conValIndex = sideInfo.getEqualValIndex().get(i); Object equalObj = input.getField(conValIndex); + + String value = ""; + if(equalObj == null){ resultFuture.complete(null); + value = "null"; + } else { + value = equalObj.toString(); } keyData.add(sideInfo.getEqualFieldList().get(i)); - keyData.add((String) equalObj); + keyData.add(value); } String key = buildCacheKey(keyData); @@ -158,29 +164,34 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except Map keyValue = Maps.newHashMap(); List value = async.keys(key + ":*").get(); String[] values = value.toArray(new String[value.size()]); - RedisFuture>> future = ((RedisStringAsyncCommands) async).mget(values); - future.thenAccept(new Consumer>>() { - @Override - public void accept(List> keyValues) { - if (keyValues.size() != 0){ - for (int i=0; i>> future = ((RedisStringAsyncCommands) async).mget(values); + future.thenAccept(new Consumer>>() { + @Override + public void accept(List> keyValues) { + if (keyValues.size() != 0) { + for (int i = 0; i < keyValues.size(); i++) { + String[] splitKeys = keyValues.get(i).getKey().split(":"); + keyValue.put(splitKeys[1], splitKeys[2]); + keyValue.put(splitKeys[3], keyValues.get(i).getValue()); + } + Row row = fillData(input, keyValue); + resultFuture.complete(Collections.singleton(row)); + if (openCache()) { + putCache(key, CacheObj.buildCacheObj(ECacheContentType.MultiLine, keyValue)); + } + } else { + dealMissKey(input, resultFuture); + if (openCache()) { + putCache(key, CacheMissVal.getMissKeyObj()); + } } } - } - }); + }); + } } private String buildCacheKey(List keyData) { diff --git a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideReqRow.java b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideReqRow.java index 40dae6c68..5007868e9 100644 --- a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideReqRow.java +++ b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideReqRow.java @@ -24,6 +24,8 @@ import org.apache.flink.types.Row; import java.io.Serializable; +import java.math.BigDecimal; +import java.sql.Date; import java.sql.Timestamp; import java.util.Map; @@ -63,10 +65,50 @@ public Row fillData(Row input, Object sideInput) { row.setField(entry.getKey(), null); }else{ String key = sideInfo.getSideFieldNameIndex().get(entry.getKey()); - row.setField(entry.getKey(), sideInputMap.get(key)); + setRowField(row, entry.getKey(), sideInfo, sideInputMap.get(key)); } } return row; } + + public void setRowField(Row row, Integer index, SideInfo sideInfo, String value) { + Integer keyIndex = sideInfo.getSideFieldIndex().get(index); + String classType = sideInfo.getSideTableInfo().getFieldClassList().get(keyIndex).getName(); + switch (classType){ + case "java.lang.Integer": + row.setField(index, Integer.valueOf(value)); + break; + case "java.lang.String": + row.setField(index, value); + break; + case "java.lang.Double": + row.setField(index, Double.valueOf(value)); + break; + case "java.lang.Long": + row.setField(index, Long.valueOf(value)); + break; + case "java.lang.Byte": + row.setField(index, Byte.valueOf(value)); + break; + case "java.lang.Short": + row.setField(index, Short.valueOf(value)); + break; + case "java.lang.Float": + row.setField(index, Float.valueOf(value)); + break; + case "java.math.BigDecimal": + row.setField(index, BigDecimal.valueOf(Long.valueOf(value))); + break; + case "java.sql.Timestamp": + row.setField(index, Timestamp.valueOf(value)); + break; + case "java.sql.Date": + row.setField(index, Date.valueOf(value)); + break; + default: + throw new RuntimeException("no support field type. the type: " + classType); + } + } + } diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java index 9077c989e..bb3963edb 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/RedisOutputFormat.java @@ -171,7 +171,13 @@ public void writeRecord(Tuple2 record) throws IOException { for (int i = 0; i < fieldNames.length; i++) { StringBuilder key = new StringBuilder(); key.append(tableName).append(":").append(perKey).append(":").append(fieldNames[i]); - jedis.set(key.toString(), row.getField(i).toString()); + + String value = "null"; + Object field = row.getField(i); + if (field != null) { + value = field.toString(); + } + jedis.set(key.toString(), value); } if (outRecords.getCount()%rowLenth == 0){ diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java index c114b946e..2fcd20dd8 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisSinkParser.java @@ -21,7 +21,10 @@ import com.dtstack.flink.sql.table.AbsTableParser; import com.dtstack.flink.sql.table.TableInfo; import com.dtstack.flink.sql.util.MathUtil; +import org.apache.commons.lang3.StringUtils; +import java.util.ArrayList; +import java.util.Arrays; import java.util.Map; public class RedisSinkParser extends AbsTableParser { @@ -42,6 +45,17 @@ public TableInfo getTableInfo(String tableName, String fieldsInfo, Map primaryKeysList = null; + if (!StringUtils.isEmpty(primaryKeysStr)) { + String[] primaryKeysArray = primaryKeysStr.split(","); + primaryKeysList = new ArrayList(Arrays.asList(primaryKeysArray)); + } else { + primaryKeysList = new ArrayList<>(); + } + redisTableInfo.setPrimaryKeys(primaryKeysList); + return redisTableInfo; } } From d25231727eaa5de9efec821e3b002aafc781d232 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Mon, 11 Nov 2019 11:09:26 +0800 Subject: [PATCH 462/470] Update README.md --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index 3c4c527a8..d21c3fdfd 100644 --- a/README.md +++ b/README.md @@ -22,7 +22,6 @@ * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra,console # 后续开发计划 - * 增加SQL支持CEP * 维表快照 * sql优化(谓词下移等) * kafka avro格式 From fd4650760ce307ae3515db1790c3be96dcead5d6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BF=AE=E7=AB=B9?= Date: Mon, 11 Nov 2019 12:04:21 +0800 Subject: [PATCH 463/470] =?UTF-8?q?=E8=A7=A3=E5=86=B3redis=E7=BB=B4?= =?UTF-8?q?=E8=A1=A8=E5=86=85=E8=BF=9E=E6=8E=A5=E7=9A=84bug?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index f3338aba8..d6f1c6862 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -31,6 +31,7 @@ import io.lettuce.core.api.async.RedisStringAsyncCommands; import io.lettuce.core.cluster.RedisClusterClient; import io.lettuce.core.cluster.api.StatefulRedisClusterConnection; +import org.apache.calcite.sql.JoinType; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; @@ -165,6 +166,9 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except List value = async.keys(key + ":*").get(); String[] values = value.toArray(new String[value.size()]); if (values.length == 0){ + if (sideInfo.getJoinType() != JoinType.LEFT) { + return; + } Row row = fillData(input, null); resultFuture.complete(Collections.singleton(row)); } else { From b8081fd6c23948f945d5cfbfc236ea10579b0d37 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BF=AE=E7=AB=B9?= Date: Mon, 11 Nov 2019 13:40:01 +0800 Subject: [PATCH 464/470] =?UTF-8?q?=E8=A7=A3=E5=86=B3redis=20async=20side?= =?UTF-8?q?=E5=86=85=E8=BF=9E=E6=8E=A5=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/side/redis/RedisAsyncReqRow.java | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index d6f1c6862..8aef1a136 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -125,15 +125,11 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except Integer conValIndex = sideInfo.getEqualValIndex().get(i); Object equalObj = input.getField(conValIndex); - String value = ""; - if(equalObj == null){ - resultFuture.complete(null); - value = "null"; - } else { - value = equalObj.toString(); + dealMissKey(input, resultFuture); + return; } - + String value = equalObj.toString(); keyData.add(sideInfo.getEqualFieldList().get(i)); keyData.add(value); } @@ -166,11 +162,7 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except List value = async.keys(key + ":*").get(); String[] values = value.toArray(new String[value.size()]); if (values.length == 0){ - if (sideInfo.getJoinType() != JoinType.LEFT) { - return; - } - Row row = fillData(input, null); - resultFuture.complete(Collections.singleton(row)); + dealMissKey(input, resultFuture); } else { RedisFuture>> future = ((RedisStringAsyncCommands) async).mget(values); future.thenAccept(new Consumer>>() { From 40c863fc4b8005402ed42ae230b7b43f770c20ad Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Mon, 11 Nov 2019 14:20:46 +0800 Subject: [PATCH 465/470] readme --- README.md | 5 ++- docs/clickhouseSide.md | 85 ++++++++++++++++++++++++++++++++++++++++++ docs/clickhouseSink.md | 53 ++++++++++++++++++++++++++ docs/kafkaSource.md | 2 +- docs/mysqlSide.md | 4 +- docs/mysqlSink.md | 4 +- 6 files changed, 146 insertions(+), 7 deletions(-) create mode 100644 docs/clickhouseSide.md create mode 100644 docs/clickhouseSink.md diff --git a/README.md b/README.md index d15399ff7..54f95f2c6 100644 --- a/README.md +++ b/README.md @@ -22,7 +22,6 @@ * 结果表:mysql, SQlServer, oracle, hbase, elasticsearch5.x, mongo, redis, cassandra, console, kudu, postgresql # 后续开发计划 - * 增加SQL支持CEP * 维表快照 * sql优化(谓词下移等) * kafka avro格式 @@ -39,7 +38,7 @@ ### 1.2 执行环境 * Java: JDK8及以上 -* Flink集群: 1.4,1.5(单机模式不需要安装Flink集群) +* Flink集群: 1.4,1.5,1.8(单机模式不需要安装Flink集群) * 操作系统:理论上不限 ### 1.3 打包 @@ -167,6 +166,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * [cassandra 结果表插件](docs/cassandraSink.md) * [kudu 结果表插件](docs/kuduSink.md) * [postgresql 结果表插件](docs/postgresqlSink.md) +* [clickhouse 结果表插件](docs/clickhouseSink.md) ### 2.3 维表插件 * [hbase 维表插件](docs/hbaseSide.md) @@ -176,6 +176,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * [cassandra 维表插件](docs/cassandraSide.md) * [kudu 维表插件](docs/kuduSide.md) * [postgresql 维表插件](docs/postgresqlSide.md) +* [clickhouse 维表插件](docs/clickhouseSide.md) ## 3 性能指标(新增) diff --git a/docs/clickhouseSide.md b/docs/clickhouseSide.md new file mode 100644 index 000000000..63d3cc3da --- /dev/null +++ b/docs/clickhouseSide.md @@ -0,0 +1,85 @@ + +## 1.格式: +``` + CREATE TABLE tableName( + colName cloType, + ... + PRIMARY KEY(keyInfo), + PERIOD FOR SYSTEM_TIME + )WITH( + type='clickhouse', + url='jdbcUrl', + userName='dbUserName', + password='dbPwd', + tableName='tableName', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + parallelism ='1', + partitionedJoin='false' + ); +``` + +# 2.支持版本 + 19.14.x、19.15.x、19.16.x + +## 3.表结构定义 + + |参数名称|含义| + |----|---| + | tableName | clickhouse表名称| + | colName | 列名称| + | colType | 列类型 [colType支持的类型](colType.md)| + | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| + | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| + +## 4.参数 + + |参数名称|含义|是否必填|默认值| + |----|---|---|----| + | type | 表明维表的类型 clickhouse |是|| + | url | 连接clickhouse数据库 jdbcUrl |是|| + | userName | clickhouse连接用户名 |是|| + | password | clickhouse连接密码|是|| + | tableName | clickhouse表名称|是|| + | tableName | clickhouse 的表名称|是|| + | cache | 维表缓存策略(NONE/LRU)|否|NONE| + | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| + + ---------- + > 缓存策略 + * NONE: 不做内存缓存 + * LRU: + * cacheSize: 缓存的条目数量 + * cacheTTLMs:缓存的过期时间(ms) + * cacheMode: (unordered|ordered)异步加载是有序还是无序,默认有序。 + * asyncCapacity:异步请求容量,默认1000 + * asyncTimeout:异步请求超时时间,默认10000毫秒 + +## 5.样例 +``` +create table sideTable( + channel varchar, + xccount int, + PRIMARY KEY(channel), + PERIOD FOR SYSTEM_TIME + )WITH( + type='clickhouse', + url='jdbc:clickhouse://172.16.8.104:3306/test?charset=utf8', + userName='dtstack', + password='abc123', + tableName='sidetest', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + cacheMode='unordered', + asyncCapacity='1000', + asyncTimeout='10000' + parallelism ='1', + partitionedJoin='false' + ); + + +``` + + diff --git a/docs/clickhouseSink.md b/docs/clickhouseSink.md new file mode 100644 index 000000000..d9774727f --- /dev/null +++ b/docs/clickhouseSink.md @@ -0,0 +1,53 @@ +## 1.格式: +``` +CREATE TABLE tableName( + colName colType, + ... + colNameX colType + )WITH( + type ='clickhouse', + url ='jdbcUrl', + userName ='userName', + password ='pwd', + tableName ='tableName', + parallelism ='parllNum' + ); + +``` + +## 2.支持版本 + 19.14.x、19.15.x、19.16.x + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName| clickhouse表名称| +| colName | 列名称| +| colType | 列类型 [colType支持的类型](colType.md)| + +## 4.参数: + +|参数名称|含义|是否必填|默认值| +|----|----|----|----| +|type |表明 输出表类型 clickhouse |是|| +|url | 连接clickhouse 数据库 jdbcUrl |是|| +|userName | clickhouse 连接用户名 |是|| +| password | clickhouse 连接密码|是|| +| tableName | clickhouse 表名称|是|| +| parallelism | 并行度设置|否|1| + +## 5.样例: +``` +CREATE TABLE MyResult( + channel VARCHAR, + pv VARCHAR + )WITH( + type ='clickhouse', + url ='jdbc:clickhouse://172.16.8.104:3306/test?charset=utf8', + userName ='dtstack', + password ='abc123', + tableName ='pv2', + parallelism ='1' + ) + ``` \ No newline at end of file diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index bb516ff62..3ddd9a3ab 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -38,7 +38,7 @@ CREATE TABLE tableName( |参数名称|含义|是否必填|默认值| |----|---|---|---| -|type | kafka09 | 是|| +|type | kafka09 | 是|kafka08、kafka09、kafka10、kafka11、kafka(对应kafka1.0及以上版本)| |kafka.group.id | 需要读取的 groupId 名称|否|| |kafka.bootstrap.servers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| |kafka.zookeeper.quorum | kafka zk地址信息(多个之间用逗号分隔)|是|| diff --git a/docs/mysqlSide.md b/docs/mysqlSide.md index b17c72bb6..f0eb16090 100644 --- a/docs/mysqlSide.md +++ b/docs/mysqlSide.md @@ -27,7 +27,7 @@ |参数名称|含义| |----|---| - | tableName | 注册到flink的表名称(可选填;不填默认和hbase对应的表名称相同)| + | tableName | mysql表名称| | colName | 列名称| | colType | 列类型 [colType支持的类型](colType.md)| | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| @@ -37,7 +37,7 @@ |参数名称|含义|是否必填|默认值| |----|---|---|----| - | type | 表明维表的类型[hbase\|mysql] |是|| + | type | 表明维表的类型 mysql |是|| | url | 连接mysql数据库 jdbcUrl |是|| | userName | mysql连接用户名 |是|| | password | mysql连接密码|是|| diff --git a/docs/mysqlSink.md b/docs/mysqlSink.md index 192c4944a..33355fddb 100644 --- a/docs/mysqlSink.md +++ b/docs/mysqlSink.md @@ -22,7 +22,7 @@ CREATE TABLE tableName( |参数名称|含义| |----|---| -| tableName| 在 sql 中使用的名称;即注册到flink-table-env上的名称| +| tableName| mysql表名称| | colName | 列名称| | colType | 列类型 [colType支持的类型](colType.md)| @@ -30,7 +30,7 @@ CREATE TABLE tableName( |参数名称|含义|是否必填|默认值| |----|----|----|----| -|type |表明 输出表类型[mysql\|hbase\|elasticsearch]|是|| +|type |表明 输出表类型mysql|是|| |url | 连接mysql数据库 jdbcUrl |是|| |userName | mysql连接用户名 |是|| | password | mysql连接密码|是|| From 43a356baa6b4d52f88071c5c620b502de1c6d172 Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Mon, 11 Nov 2019 15:54:02 +0800 Subject: [PATCH 466/470] readme format --- README.md | 4 ++-- docs/elasticsearchSink.md | 2 +- docs/hbaseSide.md | 2 +- docs/hbaseSink.md | 2 +- docs/kafkaSource.md | 10 +++++----- docs/kuduSide.md | 26 +++++++++++++------------- docs/kuduSink.md | 2 +- docs/mysqlSink.md | 2 +- docs/redisSide.md | 2 +- docs/redisSink.md | 2 +- 10 files changed, 27 insertions(+), 27 deletions(-) diff --git a/README.md b/README.md index 54f95f2c6..ed9bbb05b 100644 --- a/README.md +++ b/README.md @@ -18,8 +18,8 @@ # 已支持 * 源表:kafka 0.9、0.10、0.11、1.x版本 - * 维表:mysql, SQlServer,oracle, hbase, mongo, redis, cassandra, serversocket, kudu, postgresql - * 结果表:mysql, SQlServer, oracle, hbase, elasticsearch5.x, mongo, redis, cassandra, console, kudu, postgresql + * 维表:mysql, SQlServer,oracle, hbase, mongo, redis, cassandra, serversocket, kudu, postgresql, clickhouse + * 结果表:mysql, SQlServer, oracle, hbase, elasticsearch5.x, mongo, redis, cassandra, console, kudu, postgresql, clickhouse # 后续开发计划 * 维表快照 diff --git a/docs/elasticsearchSink.md b/docs/elasticsearchSink.md index 9a406245a..69d69af75 100644 --- a/docs/elasticsearchSink.md +++ b/docs/elasticsearchSink.md @@ -27,7 +27,7 @@ CREATE TABLE tableName( ## 4.参数: |参数名称|含义|是否必填|默认值| |----|---|---|----| -|type|表明 输出表类型[mysql\|hbase\|elasticsearch]|是|| +|type|表明 输出表类型[mysq|hbase|elasticsearch]|是|| |address | 连接ES Transport地址(tcp地址)|是|| |cluster | ES 集群名称 |是|| |index | 选择的ES上的index名称|是|| diff --git a/docs/hbaseSide.md b/docs/hbaseSide.md index 07b03026a..0c4e545f9 100644 --- a/docs/hbaseSide.md +++ b/docs/hbaseSide.md @@ -35,7 +35,7 @@ |参数名称|含义|是否必填|默认值| |----|---|---|----| -| type | 表明维表的类型[hbase\|mysql]|是|| +| type | 表明维表的类型[hbase|mysql]|是|| | zookeeperQuorum | hbase 的zk地址;格式ip:port[;ip:port]|是|| | zookeeperParent | hbase 的zk parent路径|是|| | tableName | hbase 的表名称|是|| diff --git a/docs/hbaseSink.md b/docs/hbaseSink.md index b41abb281..c6a15e766 100644 --- a/docs/hbaseSink.md +++ b/docs/hbaseSink.md @@ -30,7 +30,7 @@ hbase2.0 |参数名称|含义|是否必填|默认值| |----|---|---|-----| -|type | 表明 输出表类型[mysql\|hbase\|elasticsearch]|是|| +|type | 表明 输出表类型[mysq|hbase|elasticsearch]|是|| |zookeeperQuorum | hbase zk地址,多个直接用逗号隔开|是|| |zookeeperParent | zkParent 路径|是|| |tableName | 关联的hbase表名称|是|| diff --git a/docs/kafkaSource.md b/docs/kafkaSource.md index 3ddd9a3ab..580eda6b4 100644 --- a/docs/kafkaSource.md +++ b/docs/kafkaSource.md @@ -32,7 +32,7 @@ CREATE TABLE tableName( | colName | 列名称| | colType | 列类型 [colType支持的类型](colType.md)| | function(colNameX) as aliasName | 支持在定义列信息的时候根据已有列类型生成新的列(函数可以使用系统函数和已经注册的UDF)| -| WATERMARK FOR colName AS withOffset( colName , delayTime ) | 标识输入流生的watermake生成规则,根据指定的colName(当前支持列的类型为Long \| Timestamp) 和delayTime生成waterMark 同时会在注册表的使用附带上rowtime字段(如果未指定则默认添加proctime字段);注意:添加该标识的使用必须设置系统参数 time.characteristic:EventTime; delayTime: 数据最大延迟时间(ms)| +| WATERMARK FOR colName AS withOffset( colName , delayTime ) | 标识输入流生的watermake生成规则,根据指定的colName(当前支持列的类型为Long | Timestamp) 和delayTime生成waterMark 同时会在注册表的使用附带上rowtime字段(如果未指定则默认添加proctime字段);注意:添加该标识的使用必须设置系统参数 time.characteristic:EventTime; delayTime: 数据最大延迟时间(ms)| ## 4.参数: @@ -43,8 +43,8 @@ CREATE TABLE tableName( |kafka.bootstrap.servers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| |kafka.zookeeper.quorum | kafka zk地址信息(多个之间用逗号分隔)|是|| |kafka.topic | 需要读取的 topic 名称|是|| -|patterntopic | topic是否是正则表达式格式(true|false) |否| false -|kafka.auto.offset.reset | 读取的topic 的offset初始位置[latest\|earliest\|指定offset值({"0":12312,"1":12321,"2":12312},{"partition_no":offset_value})]|否|latest| +|patterntopic | topic是否是正则表达式格式(true|false) |否| false +|kafka.auto.offset.reset | 读取的topic 的offset初始位置[latest|earliest|指定offset值({"0":12312,"1":12321,"2":12312},{"partition_no":offset_value})]|否|latest| |parallelism | 并行度设置|否|1| |sourcedatatype | 数据类型|否|json| |timezone|时区设置[timezone支持的参数](timeZone.md)|否|'Asia/Shanghai' @@ -149,7 +149,7 @@ CREATE TABLE MyTable( |kafka.bootstrap.servers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| |kafka.zookeeper.quorum | kafka zk地址信息(多个之间用逗号分隔)|是|| |kafka.topic | 需要读取的 topic 名称|是|| -|kafka.auto.offset.reset | 读取的topic 的offset初始位置[latest\|earliest]|否|latest| +|kafka.auto.offset.reset | 读取的topic 的offset初始位置[latest|earliest]|否|latest| |parallelism | 并行度设置 |否|1| |sourcedatatype | 数据类型|是 |csv| |fielddelimiter | 字段分隔符|是 || @@ -208,7 +208,7 @@ create table kafka_stream( |kafka.bootstrap.servers | kafka bootstrap-server 地址信息(多个用逗号隔开)|是|| |kafka.zookeeper.quorum | kafka zk地址信息(多个之间用逗号分隔)|是|| |kafka.topic | 需要读取的 topic 名称|是|| -|kafka.auto.offset.reset | 读取的topic 的offset初始位置[latest\|earliest]|否|latest| +|kafka.auto.offset.reset | 读取的topic 的offset初始位置[latest|earliest]|否|latest| |parallelism | 并行度设置|否|1| |sourcedatatype | 数据类型|否|text| **kafka相关参数可以自定义,使用kafka.开头即可。** diff --git a/docs/kuduSide.md b/docs/kuduSide.md index fece4a480..5a73596ad 100644 --- a/docs/kuduSide.md +++ b/docs/kuduSide.md @@ -64,21 +64,21 @@ kudu 1.9.0+cdh6.2.0 |参数名称|含义|是否必填|默认值| |----|---|---|-----| -|type | 表明维表的类型[hbase\|mysql|\kudu]|是|| +|type | 表明维表的类型[hbase|mysql|kudu]|是|| | kuduMasters | kudu master节点的地址;格式ip[ip,ip2]|是|| | tableName | kudu 的表名称|是|| -| workerCount | 工作线程数 |否| -| defaultOperationTimeoutMs | 写入操作超时时间 |否| -| defaultSocketReadTimeoutMs | socket读取超时时间 |否| -| primaryKey | 需要过滤的主键 ALL模式独有 |否| -| lowerBoundPrimaryKey | 需要过滤的主键的最小值 ALL模式独有 |否| -| upperBoundPrimaryKey | 需要过滤的主键的最大值(不包含) ALL模式独有 |否| -| workerCount | 工作线程数 |否| -| defaultOperationTimeoutMs | 写入操作超时时间 |否| -| defaultSocketReadTimeoutMs | socket读取超时时间 |否| -| batchSizeBytes |返回数据的大小 | 否| -| limitNum |返回数据的条数 | 否| -| isFaultTolerant |查询是否容错 查询失败是否扫描第二个副本 默认false 容错 | 否| +| workerCount | 工作线程数 |否|| +| defaultOperationTimeoutMs | 写入操作超时时间 |否|| +| defaultSocketReadTimeoutMs | socket读取超时时间 |否|| +| primaryKey | 需要过滤的主键 ALL模式独有 |否|| +| lowerBoundPrimaryKey | 需要过滤的主键的最小值 ALL模式独有 |否|| +| upperBoundPrimaryKey | 需要过滤的主键的最大值(不包含) ALL模式独有 |否|| +| workerCount | 工作线程数 |否|| +| defaultOperationTimeoutMs | 写入操作超时时间 |否|| +| defaultSocketReadTimeoutMs | socket读取超时时间 |否|| +| batchSizeBytes |返回数据的大小 | 否|| +| limitNum |返回数据的条数 | 否|| +| isFaultTolerant |查询是否容错 查询失败是否扫描第二个副本 默认false 容错 | 否|| | cache | 维表缓存策略(NONE/LRU/ALL)|否|NONE| | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| diff --git a/docs/kuduSink.md b/docs/kuduSink.md index ba607c7ec..eaf155892 100644 --- a/docs/kuduSink.md +++ b/docs/kuduSink.md @@ -34,7 +34,7 @@ kudu 1.9.0+cdh6.2.0 |参数名称|含义|是否必填|默认值| |----|---|---|-----| -|type | 表明 输出表类型[mysql\|hbase\|elasticsearch\redis\|kudu\]|是|| +|type | 表名 输出表类型[mysq|hbase|elasticsearch|redis|kudu]|是|| | kuduMasters | kudu master节点的地址;格式ip[ip,ip2]|是|| | tableName | kudu 的表名称|是|| | writeMode | 写入kudu的模式 insert|update|upsert |否 |upsert diff --git a/docs/mysqlSink.md b/docs/mysqlSink.md index 33355fddb..3218f9371 100644 --- a/docs/mysqlSink.md +++ b/docs/mysqlSink.md @@ -30,7 +30,7 @@ CREATE TABLE tableName( |参数名称|含义|是否必填|默认值| |----|----|----|----| -|type |表明 输出表类型mysql|是|| +|type |表名 输出表类型[mysq|hbase|elasticsearch]|是|| |url | 连接mysql数据库 jdbcUrl |是|| |userName | mysql连接用户名 |是|| | password | mysql连接密码|是|| diff --git a/docs/redisSide.md b/docs/redisSide.md index 58249e0ad..9c7f4b47e 100644 --- a/docs/redisSide.md +++ b/docs/redisSide.md @@ -33,7 +33,7 @@ |参数名称|含义|是否必填|默认值| |----|---|---|----| -| type | 表明维表的类型[hbase\|mysql\|redis]|是|| +| type | 表明维表的类型[hbase|mysql|redis]|是|| | url | redis 的地址;格式ip:port[,ip:port]|是|| | password | redis 的密码 |是|| | redisType | redis模式(1 单机,2 哨兵, 3 集群)| 是 | diff --git a/docs/redisSink.md b/docs/redisSink.md index 4f314bc3e..6a754e5c6 100644 --- a/docs/redisSink.md +++ b/docs/redisSink.md @@ -33,7 +33,7 @@ redis5.0 |参数名称|含义|是否必填|默认值| |----|---|---|-----| -| type | 表明 输出表类型[mysql\|hbase\|elasticsearch\|redis\]|是|| +| type | 表名 输出表类型[mysq|hbase|elasticsearch|redis]|是|| | url | redis 的地址;格式ip:port[,ip:port]|是|| | password | redis 的密码 |是|| | redisType | redis模式(1 单机,2 哨兵, 3 集群)| 是 | From d7794e54d189692ca85d73789b1134be9e48ee43 Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Mon, 11 Nov 2019 16:04:00 +0800 Subject: [PATCH 467/470] kudusink readme --- docs/kuduSink.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/kuduSink.md b/docs/kuduSink.md index eaf155892..990dfdd7d 100644 --- a/docs/kuduSink.md +++ b/docs/kuduSink.md @@ -37,7 +37,7 @@ kudu 1.9.0+cdh6.2.0 |type | 表名 输出表类型[mysq|hbase|elasticsearch|redis|kudu]|是|| | kuduMasters | kudu master节点的地址;格式ip[ip,ip2]|是|| | tableName | kudu 的表名称|是|| -| writeMode | 写入kudu的模式 insert|update|upsert |否 |upsert +| writeMode | 写入kudu的模式 insert|update|upsert |否 |upsert | workerCount | 工作线程数 |否| | defaultOperationTimeoutMs | 写入操作超时时间 |否| | defaultSocketReadTimeoutMs | socket读取超时时间 |否| From d0b1c5489a6fa4a46fa3f60c87ebb43e95bfd224 Mon Sep 17 00:00:00 2001 From: todd5167 <313328862@qq.com> Date: Mon, 11 Nov 2019 16:06:30 +0800 Subject: [PATCH 468/470] remove flink shade class --- .../sql/side/cassandra/CassandraAllReqRow.java | 4 ++-- .../sql/side/cassandra/CassandraAllSideInfo.java | 4 +--- .../sql/side/cassandra/CassandraAsyncReqRow.java | 2 +- .../side/cassandra/CassandraAsyncSideInfo.java | 2 +- .../cassandra/table/CassandraSideTableInfo.java | 2 +- .../sink/cassandra/table/CassandraTableInfo.java | 2 +- core/pom.xml | 6 ------ .../src/main/java/com/dtstack/flink/sql/Main.java | 10 +++++----- .../dtstack/flink/sql/option/OptionParser.java | 2 +- .../flink/sql/parser/CreateTableParser.java | 2 +- .../flink/sql/parser/CreateTmpTableParser.java | 3 +-- .../dtstack/flink/sql/parser/InsertSqlParser.java | 2 +- .../com/dtstack/flink/sql/parser/SqlParser.java | 4 ++-- .../com/dtstack/flink/sql/parser/SqlTree.java | 4 ++-- .../dtstack/flink/sql/side/FieldReplaceInfo.java | 2 +- .../java/com/dtstack/flink/sql/side/JoinInfo.java | 2 +- .../com/dtstack/flink/sql/side/JoinScope.java | 4 ++-- .../dtstack/flink/sql/side/ParserJoinField.java | 2 +- .../java/com/dtstack/flink/sql/side/SideInfo.java | 4 ++-- .../com/dtstack/flink/sql/side/SideSQLParser.java | 9 ++++----- .../com/dtstack/flink/sql/side/SideSqlExec.java | 6 +++--- .../flink/sql/side/cache/LRUSideCache.java | 4 ++-- .../flink/sql/table/AbsSideTableParser.java | 2 -- .../dtstack/flink/sql/table/AbsTableParser.java | 5 ++--- .../dtstack/flink/sql/table/SourceTableInfo.java | 6 +++--- .../com/dtstack/flink/sql/table/TableInfo.java | 4 ++-- .../dtstack/flink/sql/table/TableInfoParser.java | 4 ++-- .../com/dtstack/flink/sql/util/DtStringUtil.java | 6 +++--- .../com/dtstack/flink/sql/util/ParseUtils.java | 1 - .../com/dtstack/flink/sql/util/PluginUtil.java | 8 ++++---- .../sql/watermarker/WaterMarkerAssigner.java | 5 +---- .../dtstack/flink/sql/side/SideSqlExecTest.java | 6 ++---- .../dtstack/flink/sql/side/TestSideSqlParser.java | 2 +- .../table/ElasticsearchTableInfo.java | 2 +- .../flink/sql/side/hbase/HbaseAllReqRow.java | 2 +- .../flink/sql/side/hbase/HbaseAllSideInfo.java | 4 +--- .../flink/sql/side/hbase/HbaseAsyncSideInfo.java | 6 ++---- .../hbase/rowkeydealer/AbsRowKeyModeDealer.java | 2 +- .../rowkeydealer/PreRowKeyModeDealerDealer.java | 2 +- .../hbase/rowkeydealer/RowKeyEqualModeDealer.java | 2 +- .../flink/sql/side/hbase/RowKeyBuilder.java | 2 +- .../sql/sink/hbase/table/HbaseTableInfo.java | 2 +- .../CustomerCsvDeserialization.java | 2 +- .../CustomerJsonDeserialization.java | 4 ++-- .../source/kafka/table/KafkaSourceTableInfo.java | 2 +- .../sink/kafka/CustomerFlinkKafkaProducer09.java | 10 ---------- .../kafka/CustomerJsonRowSerializationSchema.java | 10 +++++----- .../sink/kafka/CustomerKafka09JsonTableSink.java | 1 - .../sql/sink/kafka/table/KafkaSinkTableInfo.java | 2 +- .../source/kafka/CustomerJsonDeserialization.java | 14 +++++++------- .../source/kafka/table/KafkaSourceTableInfo.java | 2 +- .../sink/kafka/CustomerFlinkKafkaProducer010.java | 8 -------- .../kafka/CustomerJsonRowSerializationSchema.java | 10 +++++----- .../sink/kafka/CustomerKafka10JsonTableSink.java | 2 -- .../source/kafka/CustomerJsonDeserialization.java | 14 +++++++------- .../source/kafka/table/KafkaSourceTableInfo.java | 2 +- .../kafka/CustomerJsonRowSerializationSchema.java | 10 +++++----- .../source/kafka/CustomerJsonDeserialization.java | 15 +++++++-------- .../source/kafka/table/KafkaSourceTableInfo.java | 2 +- .../dtstack/flink/sql/launcher/LauncherMain.java | 2 +- .../perjob/PerJobClusterClientBuilder.java | 2 +- .../flink/sql/side/mongo/MongoAllReqRow.java | 4 ++-- .../flink/sql/side/mongo/MongoAllSideInfo.java | 4 +--- .../flink/sql/side/mongo/MongoAsyncReqRow.java | 2 +- .../flink/sql/side/mongo/MongoAsyncSideInfo.java | 2 +- .../sql/side/mongo/table/MongoSideTableInfo.java | 2 +- .../sql/sink/mongo/table/MongoTableInfo.java | 2 +- .../flink/sql/side/mysql/MysqlAllReqRow.java | 2 +- .../flink/sql/side/oracle/OracleAllReqRow.java | 2 +- .../sql/side/oracle/OracleAsyncSideInfo.java | 5 +---- .../dtstack/flink/sql/sink/oracle/OracleSink.java | 2 +- .../flink/sql/side/rdb/all/RdbAllReqRow.java | 4 ++-- .../flink/sql/side/rdb/all/RdbAllSideInfo.java | 4 +--- .../flink/sql/side/rdb/async/RdbAsyncReqRow.java | 3 +-- .../sql/side/rdb/async/RdbAsyncSideInfo.java | 2 +- .../sql/side/rdb/table/RdbSideTableInfo.java | 2 +- .../sql/sink/rdb/format/ExtendOutputFormat.java | 2 +- .../sink/rdb/format/RetractJDBCOutputFormat.java | 4 ++-- .../flink/sql/sink/rdb/table/RdbTableInfo.java | 2 +- .../flink/sql/side/redis/RedisAllReqRow.java | 4 +--- .../flink/sql/side/redis/RedisAllSideInfo.java | 4 +--- .../flink/sql/side/redis/RedisAsyncReqRow.java | 6 ++---- .../flink/sql/side/redis/RedisAsyncSideInfo.java | 4 +--- .../sql/side/redis/table/RedisSideTableInfo.java | 2 +- .../sql/sink/redis/table/RedisTableInfo.java | 2 +- .../CustomerSocketTextStreamFunction.java | 4 ++-- .../table/ServersocketSourceTableInfo.java | 2 +- .../sql/side/sqlserver/SqlserverAllReqRow.java | 3 +-- 88 files changed, 144 insertions(+), 206 deletions(-) diff --git a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java index 3a7e56902..eecb1350e 100644 --- a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java +++ b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllReqRow.java @@ -36,8 +36,8 @@ import org.apache.calcite.sql.JoinType; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.apache.flink.util.Collector; diff --git a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java index 99a286f9f..fa665f9a0 100644 --- a/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java +++ b/cassandra/cassandra-side/cassandra-all-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAllSideInfo.java @@ -24,12 +24,10 @@ import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.cassandra.table.CassandraSideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import java.util.List; diff --git a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java index a4b04ed93..c2528d67b 100644 --- a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java +++ b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncReqRow.java @@ -45,7 +45,7 @@ import io.vertx.core.json.JsonArray; import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; -import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; diff --git a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java index 8ff801064..3557f0f73 100644 --- a/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java +++ b/cassandra/cassandra-side/cassandra-async-side/src/main/java/com/dtstack/flink/sql/side/cassandra/CassandraAsyncSideInfo.java @@ -29,7 +29,7 @@ import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import java.util.List; diff --git a/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideTableInfo.java b/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideTableInfo.java index b1b36f7e8..205a0f769 100644 --- a/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideTableInfo.java +++ b/cassandra/cassandra-side/cassandra-side-core/src/main/java/com/dtstack/flink/sql/side/cassandra/table/CassandraSideTableInfo.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.side.cassandra.table; import com.dtstack.flink.sql.side.SideTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; /** * Reason: diff --git a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java index 7d52b23bb..c6626c42a 100644 --- a/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java +++ b/cassandra/cassandra-sink/src/main/java/com/dtstack/flink/sql/sink/cassandra/table/CassandraTableInfo.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.sink.cassandra.table; import com.dtstack.flink.sql.table.TargetTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; /** * Reason: diff --git a/core/pom.xml b/core/pom.xml index 8d829a438..c357b4c99 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -89,12 +89,6 @@ ${flink.version} - - org.apache.flink - flink-shaded-hadoop2 - 1.7.2 - - org.apache.flink flink-yarn_2.11 diff --git a/core/src/main/java/com/dtstack/flink/sql/Main.java b/core/src/main/java/com/dtstack/flink/sql/Main.java index 6f31dc5c7..a08df9b05 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -58,11 +58,11 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.client.program.ContextEnvironment; import org.apache.flink.configuration.Configuration; -import org.apache.flink.shaded.guava18.com.google.common.base.Strings; -import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; -import org.apache.flink.shaded.guava18.com.google.common.collect.Sets; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamContextEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; diff --git a/core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java b/core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java index 10e34a5e6..07860b608 100644 --- a/core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/option/OptionParser.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.option; -import avro.shaded.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import com.dtstack.flink.sql.util.PluginUtil; import org.apache.commons.cli.BasicParser; import org.apache.commons.cli.CommandLine; diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java index 5e126e786..ae6e1f708 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTableParser.java @@ -21,7 +21,7 @@ package com.dtstack.flink.sql.parser; import com.dtstack.flink.sql.util.DtStringUtil; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import com.google.common.collect.Maps; import java.util.List; import java.util.Map; diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java index db18986b7..de7141eb5 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/CreateTmpTableParser.java @@ -25,8 +25,7 @@ import org.apache.calcite.sql.*; import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; -import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; - +import com.google.common.collect.Lists; import java.util.List; import java.util.regex.Matcher; import java.util.regex.Pattern; diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java index e946c2b5a..a7c6db9eb 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/InsertSqlParser.java @@ -25,7 +25,7 @@ import org.apache.calcite.sql.parser.SqlParseException; import org.apache.calcite.sql.parser.SqlParser; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import java.util.List; diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java index e9fb68cfe..a76c1b31a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/SqlParser.java @@ -25,8 +25,8 @@ import com.dtstack.flink.sql.table.TableInfoParser; import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.collect.Lists; -import org.apache.flink.shaded.guava18.com.google.common.base.Strings; +import com.google.common.collect.Lists; +import com.google.common.base.Strings; import java.util.List; import java.util.Set; diff --git a/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java b/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java index 754de0819..1b64b7c68 100644 --- a/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java +++ b/core/src/main/java/com/dtstack/flink/sql/parser/SqlTree.java @@ -22,8 +22,8 @@ import com.dtstack.flink.sql.table.TableInfo; -import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.collect.Maps; -import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Lists; import java.util.List; import java.util.Map; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/FieldReplaceInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/FieldReplaceInfo.java index bc716ddaa..37b23d046 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/FieldReplaceInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/FieldReplaceInfo.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.side; -import org.apache.flink.calcite.shaded.com.google.common.collect.HashBasedTable; +import com.google.common.collect.HashBasedTable; /** * Reason: diff --git a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java index f2c08d1b2..6fde02493 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/JoinInfo.java @@ -22,7 +22,7 @@ import org.apache.calcite.sql.JoinType; import org.apache.calcite.sql.SqlNode; -import org.apache.flink.calcite.shaded.com.google.common.base.Strings; +import com.google.common.base.Strings; import java.io.Serializable; import java.util.Map; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/JoinScope.java b/core/src/main/java/com/dtstack/flink/sql/side/JoinScope.java index ba07e714a..c7a73e0d7 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/JoinScope.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/JoinScope.java @@ -23,8 +23,8 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import java.util.List; import java.util.Map; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java b/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java index 27d82e779..74d303c24 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/ParserJoinField.java @@ -27,7 +27,7 @@ import org.apache.calcite.sql.SqlSelect; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import java.util.Iterator; import java.util.List; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java b/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java index 97e5e555f..df41e1663 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideInfo.java @@ -27,8 +27,8 @@ import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import java.io.Serializable; import java.util.List; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java index 5ec79ec3b..c881d6344 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSQLParser.java @@ -41,14 +41,13 @@ import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.calcite.shaded.com.google.common.base.Strings; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; -import org.apache.flink.calcite.shaded.com.google.common.collect.Queues; +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Queues; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Queue; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java index 7f3cbd394..47c88c27a 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/SideSqlExec.java @@ -48,9 +48,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.HashBasedTable; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import com.google.common.collect.HashBasedTable; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.java.StreamTableEnvironment; diff --git a/core/src/main/java/com/dtstack/flink/sql/side/cache/LRUSideCache.java b/core/src/main/java/com/dtstack/flink/sql/side/cache/LRUSideCache.java index 475536a4c..700e13bb2 100644 --- a/core/src/main/java/com/dtstack/flink/sql/side/cache/LRUSideCache.java +++ b/core/src/main/java/com/dtstack/flink/sql/side/cache/LRUSideCache.java @@ -21,8 +21,8 @@ package com.dtstack.flink.sql.side.cache; import com.dtstack.flink.sql.side.SideTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.cache.Cache; -import org.apache.flink.calcite.shaded.com.google.common.cache.CacheBuilder; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; import java.util.concurrent.TimeUnit; diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java index ae8135256..3c4199c8c 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsSideTableParser.java @@ -23,8 +23,6 @@ import com.dtstack.flink.sql.enums.ECacheType; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.util.MathUtil; -import org.apache.commons.lang3.StringUtils; - import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; diff --git a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java index fc29932bf..cfabbc6fb 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/AbsTableParser.java @@ -22,9 +22,8 @@ import com.dtstack.flink.sql.util.ClassUtil; import com.dtstack.flink.sql.util.DtStringUtil; -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.collect.Maps; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import java.util.List; import java.util.Map; diff --git a/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java index d809dc3b5..9a41fa0a1 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/SourceTableInfo.java @@ -20,9 +20,9 @@ package com.dtstack.flink.sql.table; -import org.apache.flink.calcite.shaded.com.google.common.base.Strings; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.flink.util.StringUtils; import java.util.ArrayList; diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java index ebda7d120..a2454b893 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfo.java @@ -20,8 +20,8 @@ package com.dtstack.flink.sql.table; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import java.io.Serializable; import java.util.List; diff --git a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java b/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java index 4c2f67f68..ae98d90ae 100644 --- a/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java +++ b/core/src/main/java/com/dtstack/flink/sql/table/TableInfoParser.java @@ -27,8 +27,8 @@ import com.dtstack.flink.sql.sink.StreamSinkFactory; import com.dtstack.flink.sql.source.StreamSourceFactory; import com.dtstack.flink.sql.util.MathUtil; -import org.apache.flink.calcite.shaded.com.google.common.base.Strings; -import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.collect.Maps; +import com.google.common.base.Strings; +import com.google.common.collect.Maps; import java.util.Map; import java.util.regex.Matcher; diff --git a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java index 1f571dd49..a78cf977b 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/DtStringUtil.java @@ -22,9 +22,9 @@ import com.dtstack.flink.sql.enums.ColumnType; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.calcite.shaded.com.google.common.base.Strings; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import com.google.common.collect.Maps; +import com.fasterxml.jackson.databind.ObjectMapper; import java.sql.Timestamp; import java.math.BigDecimal; import java.util.ArrayList; diff --git a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java index bc4a0e577..ffde35bc1 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/ParseUtils.java @@ -47,7 +47,6 @@ import org.apache.calcite.sql.SqlNodeList; import org.apache.calcite.sql.fun.SqlCase; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.calcite.shaded.com.google.common.collect.HashBasedTable; import java.util.List; import java.util.Map; diff --git a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java index 569bf1dd3..f1c26b047 100644 --- a/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java +++ b/core/src/main/java/com/dtstack/flink/sql/util/PluginUtil.java @@ -22,10 +22,10 @@ import com.dtstack.flink.sql.classloader.DtClassLoader; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerationException; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParseException; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonMappingException; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.core.JsonGenerationException; +import com.fasterxml.jackson.core.JsonParseException; +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.ObjectMapper; import java.io.ByteArrayInputStream; import java.io.File; diff --git a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java index 4f386d75f..d444c1bac 100644 --- a/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java +++ b/core/src/main/java/com/dtstack/flink/sql/watermarker/WaterMarkerAssigner.java @@ -21,13 +21,10 @@ package com.dtstack.flink.sql.watermarker; import com.dtstack.flink.sql.table.SourceTableInfo; -import com.google.common.collect.Lists; -import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.shaded.guava18.com.google.common.base.Strings; +import com.google.common.base.Strings; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; diff --git a/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java b/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java index 8eb09ee18..3bab778ae 100644 --- a/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java +++ b/core/src/test/java/com/dtstack/flink/sql/side/SideSqlExecTest.java @@ -22,12 +22,10 @@ import com.dtstack.flink.sql.Main; import com.dtstack.flink.sql.parser.SqlParser; import com.dtstack.flink.sql.parser.SqlTree; -import org.apache.flink.calcite.shaded.com.google.common.base.Charsets; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; import org.junit.Test; -import java.io.UnsupportedEncodingException; -import java.net.URLDecoder; import java.net.URLEncoder; import java.util.List; diff --git a/core/src/test/java/com/dtstack/flink/sql/side/TestSideSqlParser.java b/core/src/test/java/com/dtstack/flink/sql/side/TestSideSqlParser.java index 01e33a3b6..18a7d2d86 100644 --- a/core/src/test/java/com/dtstack/flink/sql/side/TestSideSqlParser.java +++ b/core/src/test/java/com/dtstack/flink/sql/side/TestSideSqlParser.java @@ -21,7 +21,7 @@ package com.dtstack.flink.sql.side; import org.apache.calcite.sql.parser.SqlParseException; -import org.apache.flink.calcite.shaded.com.google.common.collect.Sets; +import com.google.common.collect.Sets; import org.junit.Test; import java.util.Set; diff --git a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java index 19f158c3a..e616163ec 100644 --- a/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java +++ b/elasticsearch5/elasticsearch5-sink/src/main/java/com/dtstack/flink/sql/sink/elasticsearch/table/ElasticsearchTableInfo.java @@ -22,7 +22,7 @@ import com.dtstack.flink.sql.table.TargetTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; /** * @date 2018/09/12 diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java index c9adc87b9..0fe2d1720 100644 --- a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllReqRow.java @@ -24,7 +24,7 @@ import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo; import org.apache.commons.collections.map.HashedMap; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import com.google.common.collect.Maps; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.apache.flink.util.Collector; diff --git a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java index dde3f41ba..ea51f46e4 100644 --- a/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java +++ b/hbase/hbase-side/hbase-all-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAllSideInfo.java @@ -25,11 +25,9 @@ import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import java.util.List; diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncSideInfo.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncSideInfo.java index 84a6358d1..2bfdd0d44 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncSideInfo.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/HbaseAsyncSideInfo.java @@ -6,12 +6,10 @@ import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.hbase.table.HbaseSideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import java.util.List; import java.util.Map; diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbsRowKeyModeDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbsRowKeyModeDealer.java index 5fe61af4f..d73b31e75 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbsRowKeyModeDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/AbsRowKeyModeDealer.java @@ -23,7 +23,7 @@ import com.dtstack.flink.sql.side.FieldInfo; import com.dtstack.flink.sql.side.cache.AbsSideCache; import org.apache.calcite.sql.JoinType; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import com.google.common.collect.Maps; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.types.Row; import org.hbase.async.HBaseClient; diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java index d79f67ec5..7503f95c0 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/PreRowKeyModeDealerDealer.java @@ -28,7 +28,7 @@ import com.dtstack.flink.sql.side.hbase.utils.HbaseUtils; import com.google.common.collect.Maps; import org.apache.calcite.sql.JoinType; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.types.Row; import org.hbase.async.BinaryPrefixComparator; diff --git a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java index fe3149e3d..8604db18e 100644 --- a/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java +++ b/hbase/hbase-side/hbase-async-side/src/main/java/com/dtstack/flink/sql/side/hbase/rowkeydealer/RowKeyEqualModeDealer.java @@ -28,7 +28,7 @@ import com.dtstack.flink.sql.side.hbase.utils.HbaseUtils; import com.google.common.collect.Maps; import org.apache.calcite.sql.JoinType; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.types.Row; import org.hbase.async.GetRequest; diff --git a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java index af2053fe6..114b7fa6a 100644 --- a/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java +++ b/hbase/hbase-side/hbase-side-core/src/main/java/com/dtstack/flink/sql/side/hbase/RowKeyBuilder.java @@ -21,7 +21,7 @@ package com.dtstack.flink.sql.side.hbase; import com.dtstack.flink.sql.side.hbase.enums.EReplaceType; -import org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import java.io.Serializable; import java.util.List; diff --git a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java index 4e286fd45..ed28f781d 100644 --- a/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java +++ b/hbase/hbase-sink/src/main/java/com/dtstack/flink/sql/sink/hbase/table/HbaseTableInfo.java @@ -22,7 +22,7 @@ import com.dtstack.flink.sql.table.TargetTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; import java.util.Map; /** diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java index 0ca57e0b3..f9f4c897c 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerCsvDeserialization.java @@ -25,7 +25,7 @@ import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java index 900717b98..3f0c10737 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/deserialization/CustomerJsonDeserialization.java @@ -24,8 +24,8 @@ import com.dtstack.flink.sql.source.AbsDeserialization; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.types.Row; import org.apache.kafka.common.TopicPartition; diff --git a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 9f7e36f56..93e9d52f7 100644 --- a/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka08/kafka08-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -21,7 +21,7 @@ package com.dtstack.flink.sql.source.kafka.table; import com.dtstack.flink.sql.table.SourceTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; import java.util.HashMap; import java.util.Map; diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer09.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer09.java index b026bf2c6..1660063c4 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer09.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer09.java @@ -23,17 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.MeterView; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer09; -import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricWrapper; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; -import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.Metric; -import org.apache.kafka.common.MetricName; - -import java.util.Map; import java.util.Properties; /** diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java index 079d13a84..be9538464 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java @@ -27,11 +27,11 @@ import org.apache.flink.formats.json.JsonRowDeserializationSchema; import org.apache.flink.formats.json.JsonRowSchemaConverter; import org.apache.flink.metrics.Counter; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ContainerNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ContainerNode; +import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka09JsonTableSink.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka09JsonTableSink.java index 47ff5dab4..ffd44e9f7 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka09JsonTableSink.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka09JsonTableSink.java @@ -22,7 +22,6 @@ import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase; import org.apache.flink.streaming.connectors.kafka.Kafka09TableSink; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.utils.TableConnectorUtils; diff --git a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java index d4b3e5b62..0e6ba70bd 100644 --- a/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java +++ b/kafka09/kafka09-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/table/KafkaSinkTableInfo.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.sink.kafka.table; import com.dtstack.flink.sql.table.TargetTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; import java.util.HashMap; import java.util.Map; diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index ba8cc62a9..a8c3828eb 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -27,14 +27,14 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Strings; +import com.google.common.base.Strings; import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; +import com.google.common.collect.Maps; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.JsonNodeType; +import com.fasterxml.jackson.databind.node.TextNode; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.types.Row; diff --git a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 9081b956b..e6098fb3c 100644 --- a/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka09/kafka09-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.source.kafka.table; import com.dtstack.flink.sql.table.SourceTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; /** * Reason: diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer010.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer010.java index 50da17099..2f11c355d 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer010.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerFlinkKafkaProducer010.java @@ -23,15 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.MeterView; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer010; -import org.apache.flink.streaming.connectors.kafka.internals.metrics.KafkaMetricWrapper; -import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; -import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.Metric; -import org.apache.kafka.common.MetricName; import java.util.Map; import java.util.Properties; diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java index 7d1d45345..8033c851e 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java @@ -27,11 +27,11 @@ import org.apache.flink.formats.json.JsonRowDeserializationSchema; import org.apache.flink.formats.json.JsonRowSchemaConverter; import org.apache.flink.metrics.Counter; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ContainerNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ContainerNode; +import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; diff --git a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java index 01762079a..d3edc2d86 100644 --- a/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java +++ b/kafka10/kafka10-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerKafka10JsonTableSink.java @@ -19,13 +19,11 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducerBase; import org.apache.flink.streaming.connectors.kafka.Kafka010TableSink; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.util.TableConnectorUtil; import org.apache.flink.table.utils.TableConnectorUtils; import org.apache.flink.types.Row; diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 99373737f..527fbc33a 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -27,14 +27,14 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Strings; +import com.google.common.base.Strings; import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; +import com.google.common.collect.Maps; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.JsonNodeType; +import com.fasterxml.jackson.databind.node.TextNode; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.types.Row; diff --git a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 113159450..da78a2af6 100644 --- a/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -21,7 +21,7 @@ package com.dtstack.flink.sql.source.kafka.table; import com.dtstack.flink.sql.table.SourceTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; /** * Reason: diff --git a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java index 4b77cac76..0d575a5fd 100644 --- a/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java +++ b/kafka11/kafka11-sink/src/main/java/com/dtstack/flink/sql/sink/kafka/CustomerJsonRowSerializationSchema.java @@ -28,11 +28,11 @@ import org.apache.flink.formats.json.JsonRowDeserializationSchema; import org.apache.flink.formats.json.JsonRowSchemaConverter; import org.apache.flink.metrics.Counter; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ContainerNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ContainerNode; +import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index c6ce5fefe..500e3c35c 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java @@ -27,14 +27,14 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Strings; +import com.google.common.base.Strings; import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.JsonNodeType; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.TextNode; +import com.google.common.collect.Maps; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.JsonNodeType; +import com.fasterxml.jackson.databind.node.TextNode; import org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread; import org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher; import org.apache.flink.types.Row; @@ -45,7 +45,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.lang.reflect.Array; import java.lang.reflect.Field; import java.sql.Date; import java.sql.Time; diff --git a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java index 4173063de..7a166695b 100644 --- a/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java +++ b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/table/KafkaSourceTableInfo.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.source.kafka.table; import com.dtstack.flink.sql.table.SourceTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; /** * Reason: diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java index ce48b388c..0edd01434 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/LauncherMain.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.launcher; -import avro.shaded.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.TypeReference; import com.dtstack.flink.sql.enums.ClusterMode; diff --git a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java index 3b58bf845..5dc0971bc 100644 --- a/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java +++ b/launcher/src/main/java/com/dtstack/flink/sql/launcher/perjob/PerJobClusterClientBuilder.java @@ -24,7 +24,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.configuration.Configuration; -import org.apache.flink.hadoop.shaded.com.google.common.base.Strings; +import com.google.common.base.Strings; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java index d1a38ca5f..4dc7c26b4 100644 --- a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllReqRow.java @@ -36,8 +36,8 @@ import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.apache.flink.util.Collector; diff --git a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java index b3ef1f7e7..6191114b5 100644 --- a/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java +++ b/mongo/mongo-side/mongo-all-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAllSideInfo.java @@ -24,12 +24,10 @@ import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.mongo.table.MongoSideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import java.util.List; diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java index 226885ed7..150053246 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncReqRow.java @@ -40,7 +40,7 @@ import com.mongodb.connection.ConnectionPoolSettings; import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; diff --git a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java index 1590912a1..365026972 100644 --- a/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java +++ b/mongo/mongo-side/mongo-async-side/src/main/java/com/dtstack/flink/sql/side/mongo/MongoAsyncSideInfo.java @@ -29,7 +29,7 @@ import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import java.util.List; diff --git a/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideTableInfo.java b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideTableInfo.java index 721960003..a5c834469 100644 --- a/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideTableInfo.java +++ b/mongo/mongo-side/mongo-side-core/src/main/java/com/dtstack/flink/sql/side/mongo/table/MongoSideTableInfo.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.side.mongo.table; import com.dtstack.flink.sql.side.SideTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; /** * Reason: diff --git a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoTableInfo.java b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoTableInfo.java index 91de9eba6..02a96d6bb 100644 --- a/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoTableInfo.java +++ b/mongo/mongo-sink/src/main/java/com/dtstack/flink/sql/sink/mongo/table/MongoTableInfo.java @@ -20,7 +20,7 @@ package com.dtstack.flink.sql.sink.mongo.table; import com.dtstack.flink.sql.table.TargetTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; /** * Reason: diff --git a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java index a60f5774e..b6b7f45cb 100644 --- a/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java +++ b/mysql/mysql-side/mysql-all-side/src/main/java/com/dtstack/flink/sql/side/mysql/MysqlAllReqRow.java @@ -24,7 +24,7 @@ import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java index 278e5e665..18d9ba045 100644 --- a/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java +++ b/oracle/oracle-side/oracle-all-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAllReqRow.java @@ -24,7 +24,7 @@ import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java index 891aebe61..c9d2f98d9 100644 --- a/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java +++ b/oracle/oracle-side/oracle-async-side/src/main/java/com/dtstack/flink/sql/side/oracle/OracleAsyncSideInfo.java @@ -25,13 +25,10 @@ import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import com.dtstack.flink.sql.util.DtStringUtil; import com.dtstack.flink.sql.util.ParseUtils; -import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; -import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import com.google.common.collect.Lists; -import java.util.Arrays; import java.util.List; diff --git a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java index 129286e63..c30dc9c60 100644 --- a/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java +++ b/oracle/oracle-sink/src/main/java/com/dtstack/flink/sql/sink/oracle/OracleSink.java @@ -23,7 +23,7 @@ import com.dtstack.flink.sql.sink.rdb.format.RetractJDBCOutputFormat; import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import java.util.ArrayList; import java.util.Iterator; diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java index 6c3f205a1..a8f4d1ae4 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllReqRow.java @@ -24,8 +24,8 @@ import com.dtstack.flink.sql.side.rdb.util.SwitchUtil; import org.apache.calcite.sql.JoinType; import org.apache.commons.collections.CollectionUtils; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; import org.apache.flink.util.Collector; diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java index 9ea79802d..c10b394e5 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/all/RdbAllSideInfo.java @@ -24,12 +24,10 @@ import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.side.rdb.table.RdbSideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import java.util.List; diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java index 323bed3ea..d997f7d58 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncReqRow.java @@ -26,8 +26,7 @@ import io.vertx.core.json.JsonArray; import io.vertx.ext.sql.SQLClient; import io.vertx.ext.sql.SQLConnection; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java index 2a8859ca1..eb09d613e 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/async/RdbAsyncSideInfo.java @@ -29,7 +29,7 @@ import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import java.util.List; diff --git a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java index 174c992f5..0dfbef325 100644 --- a/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java +++ b/rdb/rdb-side/src/main/java/com/dtstack/flink/sql/side/rdb/table/RdbSideTableInfo.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.side.rdb.table; import com.dtstack.flink.sql.side.SideTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; /** * Reason: diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java index 80f7f2018..6265bbadd 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/ExtendOutputFormat.java @@ -20,7 +20,7 @@ import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.commons.lang3.StringUtils; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import com.google.common.collect.Maps; import java.sql.ResultSet; import java.sql.SQLException; diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java index 41bb37792..292d30ca7 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/format/RetractJDBCOutputFormat.java @@ -23,8 +23,8 @@ import org.apache.commons.lang3.StringUtils; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.flink.types.Row; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java index 9ace0afd0..d43104fcb 100644 --- a/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java +++ b/rdb/rdb-sink/src/main/java/com/dtstack/flink/sql/sink/rdb/table/RdbTableInfo.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.sink.rdb.table; import com.dtstack.flink.sql.table.TargetTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; /** * Reason: diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java index 54ecf8952..cbfe807dd 100644 --- a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllReqRow.java @@ -24,8 +24,7 @@ import org.apache.calcite.sql.JoinType; import org.apache.commons.pool2.impl.GenericObjectPoolConfig; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; -import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; +import com.google.common.collect.Maps; import org.apache.flink.types.Row; import org.apache.flink.util.Collector; import org.slf4j.Logger; @@ -35,7 +34,6 @@ import java.io.Closeable; import java.io.IOException; import java.sql.SQLException; -import java.sql.Timestamp; import java.util.*; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; diff --git a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java index b449d8c02..81e7b1f4f 100644 --- a/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java +++ b/redis5/redis5-side/redis-all-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAllSideInfo.java @@ -23,11 +23,9 @@ import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import java.util.List; diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java index 4da17eb22..aecab0516 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncReqRow.java @@ -32,14 +32,12 @@ import io.lettuce.core.cluster.RedisClusterClient; import io.lettuce.core.cluster.api.StatefulRedisClusterConnection; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; -import org.apache.flink.calcite.shaded.com.google.common.collect.Maps; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.async.ResultFuture; -import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo; import org.apache.flink.types.Row; -import java.sql.Timestamp; import java.util.Collections; import java.util.List; import java.util.Map; diff --git a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java index 2c85aaf3a..9bdefe8c5 100644 --- a/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java +++ b/redis5/redis5-side/redis-async-side/src/main/java/com/dtstack/flink/sql/side/redis/RedisAsyncSideInfo.java @@ -23,11 +23,9 @@ import com.dtstack.flink.sql.side.SideInfo; import com.dtstack.flink.sql.side.SideTableInfo; import com.dtstack.flink.sql.util.ParseUtils; -import org.apache.calcite.sql.SqlBasicCall; -import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.calcite.shaded.com.google.common.collect.Lists; +import com.google.common.collect.Lists; import java.util.List; diff --git a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideTableInfo.java b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideTableInfo.java index c463febec..88cfcb8a2 100644 --- a/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideTableInfo.java +++ b/redis5/redis5-side/redis-side-core/src/main/java/com/dtstack/flink/sql/side/redis/table/RedisSideTableInfo.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.side.redis.table; import com.dtstack.flink.sql.side.SideTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; public class RedisSideTableInfo extends SideTableInfo { diff --git a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java index 7afb51cdd..82def4115 100644 --- a/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java +++ b/redis5/redis5-sink/src/main/java/com/dtstack/flink/sql/sink/redis/table/RedisTableInfo.java @@ -19,7 +19,7 @@ package com.dtstack.flink.sql.sink.redis.table; import com.dtstack.flink.sql.table.TargetTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; public class RedisTableInfo extends TargetTableInfo { diff --git a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java index ce9a44895..b76e94159 100644 --- a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/CustomerSocketTextStreamFunction.java @@ -20,8 +20,8 @@ import com.dtstack.flink.sql.source.serversocket.table.ServersocketSourceTableInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.types.Row; import org.apache.flink.util.IOUtils; diff --git a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceTableInfo.java b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceTableInfo.java index 2ff9a6739..796728eb3 100644 --- a/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceTableInfo.java +++ b/serversocket/serversocket-source/src/main/java/com/dtstack/flink/sql/source/serversocket/table/ServersocketSourceTableInfo.java @@ -18,7 +18,7 @@ package com.dtstack.flink.sql.source.serversocket.table; import com.dtstack.flink.sql.table.SourceTableInfo; -import org.apache.flink.calcite.shaded.com.google.common.base.Preconditions; +import com.google.common.base.Preconditions; /** * Reason: diff --git a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java index 837236b61..961539fde 100644 --- a/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java +++ b/sqlserver/sqlserver-side/sqlserver-all-side/src/main/java/com/dtstack/flink/sql/side/sqlserver/SqlserverAllReqRow.java @@ -24,10 +24,9 @@ import com.dtstack.flink.sql.side.rdb.all.RdbAllReqRow; import com.dtstack.flink.sql.util.DtStringUtil; import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; +import com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.sql.Connection; import java.sql.DriverManager; import java.util.List; From 8c63e2e65cdb2c552f42f9899e72fc5077c0ea2e Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Mon, 11 Nov 2019 17:55:49 +0800 Subject: [PATCH 469/470] =?UTF-8?q?readme=E5=A2=9E=E5=8A=A0=E6=96=B0?= =?UTF-8?q?=E7=89=88=E6=9C=AC=E4=B8=AD=E7=9A=84=E6=96=B0=E5=8A=9F=E8=83=BD?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 24 ++++++++++++++++++------ 1 file changed, 18 insertions(+), 6 deletions(-) diff --git a/README.md b/README.md index ed9bbb05b..358ed8bc6 100644 --- a/README.md +++ b/README.md @@ -8,13 +8,21 @@ > > * 扩展了输入和输出的性能指标到promethus ## 新特性: - 1.kafka源表支持not null语法,支持字符串类型的时间转换。 - 2.rdb维表与DB建立连接时,周期进行连接,防止连接断开。rdbsink写入时,对连接进行检查。 - 3.异步维表支持非等值连接,比如:<>,<,>。 + * 1.kafka源表支持not null语法,支持字符串类型的时间转换。 + * 2.rdb维表与DB建立连接时,周期进行连接,防止连接断开。rdbsink写入时,对连接进行检查。 + * 3.异步维表支持非等值连接,比如:<>,<,>。 + * 4.增加kafka数组解析 + * 5.增加kafka1.0以上版本的支持 + * 6.增加postgresql、kudu、clickhouse维表、结果表的支持 + * 7.支持插件的依赖方式,参考pluginLoadMode参数 + * 8.支持cep处理 + * 9.支持udaf + * 10.支持谓词下移 ## BUG修复: - 1.修复不能解析sql中orderby,union语法。 - 2.修复yarnPer模式提交失败的异常。 + * 1.修复不能解析sql中orderby,union语法。 + * 2.修复yarnPer模式提交失败的异常。 + * 3.一些bug的修复 # 已支持 * 源表:kafka 0.9、0.10、0.11、1.x版本 @@ -23,7 +31,6 @@ # 后续开发计划 * 维表快照 - * sql优化(谓词下移等) * kafka avro格式 * topN @@ -148,6 +155,11 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * 必选:否 * 默认值:false +* **pluginLoadMode** + * 描述:per_job 模式下的插件包加载方式。classpath:从每台机器加载插件包,shipfile:将需要插件从提交的节点上传到hdfs,不需要每台安装插件 + * 必选:否 + * 默认值:classpath + * **yarnSessionConf** * 描述:yarn session 模式下指定的运行的一些参数,[可参考](https://ci.apache.org/projects/flink/flink-docs-release-1.8/ops/cli.html),目前只支持指定yid * 必选:否 From 265eae8a505fd97427ff611e7baac3e890bd2b9c Mon Sep 17 00:00:00 2001 From: simenliuxing <492341344@qq.com> Date: Mon, 11 Nov 2019 21:34:51 +0800 Subject: [PATCH 470/470] =?UTF-8?q?readme=20oracle=20side=E3=80=81sink=20d?= =?UTF-8?q?oc?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- README.md | 2 ++ docs/oracleSide.md | 85 ++++++++++++++++++++++++++++++++++++++++++++++ docs/oracleSink.md | 55 ++++++++++++++++++++++++++++++ 3 files changed, 142 insertions(+) create mode 100644 docs/oracleSide.md create mode 100644 docs/oracleSink.md diff --git a/README.md b/README.md index 358ed8bc6..6474dc052 100644 --- a/README.md +++ b/README.md @@ -173,6 +173,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack * [elasticsearch 结果表插件](docs/elasticsearchSink.md) * [hbase 结果表插件](docs/hbaseSink.md) * [mysql 结果表插件](docs/mysqlSink.md) +* [oracle 结果表插件](docs/oracleSink.md) * [mongo 结果表插件](docs/mongoSink.md) * [redis 结果表插件](docs/redisSink.md) * [cassandra 结果表插件](docs/cassandraSink.md) @@ -183,6 +184,7 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack ### 2.3 维表插件 * [hbase 维表插件](docs/hbaseSide.md) * [mysql 维表插件](docs/mysqlSide.md) +* [oracle 维表插件](docs/oracleSide.md) * [mongo 维表插件](docs/mongoSide.md) * [redis 维表插件](docs/redisSide.md) * [cassandra 维表插件](docs/cassandraSide.md) diff --git a/docs/oracleSide.md b/docs/oracleSide.md new file mode 100644 index 000000000..74fc56680 --- /dev/null +++ b/docs/oracleSide.md @@ -0,0 +1,85 @@ + +## 1.格式: +``` + CREATE TABLE tableName( + colName cloType, + ... + PRIMARY KEY(keyInfo), + PERIOD FOR SYSTEM_TIME + )WITH( + type='oracle', + url='jdbcUrl', + userName='dbUserName', + password='dbPwd', + tableName='tableName', + cache ='LRU', + schema = 'MQTEST', + parallelism ='1', + partitionedJoin='false' + ); +``` + +# 2.支持版本 + 10g 11g + +## 3.表结构定义 + + |参数名称|含义| + |----|---| + | tableName | oracle表名称| + | colName | 列名称| + | colType | 列类型 [colType支持的类型](colType.md)| + | PERIOD FOR SYSTEM_TIME | 关键字表明该定义的表为维表信息| + | PRIMARY KEY(keyInfo) | 维表主键定义;多个列之间用逗号隔开| + +## 4.参数 + + |参数名称|含义|是否必填|默认值| + |----|---|---|----| + | type | 表明维表的类型 oracle |是|| + | url | 连接oracle数据库 jdbcUrl |是|| + | userName | oracle连接用户名 |是|| + | password | oracle连接密码|是|| + | tableName | oracle表名称|是|| + | schema | oracle 的schema|否|当前登录用户| + | cache | 维表缓存策略(NONE/LRU)|否|NONE| + | partitionedJoin | 是否在維表join之前先根据 設定的key 做一次keyby操作(可以減少维表的数据缓存量)|否|false| + + ---------- + > 缓存策略 + * NONE: 不做内存缓存 + * LRU: + * cacheSize: 缓存的条目数量 + * cacheTTLMs:缓存的过期时间(ms) + * cacheMode: (unordered|ordered)异步加载是有序还是无序,默认有序。 + * asyncCapacity:异步请求容量,默认1000 + * asyncTimeout:异步请求超时时间,默认10000毫秒 + +## 5.样例 +``` +create table sideTable( + channel varchar, + xccount int, + PRIMARY KEY(channel), + PERIOD FOR SYSTEM_TIME + )WITH( + type='oracle', + url='jdbc:oracle:thin:@xx.xx.xx.xx:1521:orcl', + userName='xx', + password='xx', + tableName='sidetest', + cache ='LRU', + cacheSize ='10000', + cacheTTLMs ='60000', + cacheMode='unordered', + asyncCapacity='1000', + asyncTimeout='10000' + parallelism ='1', + partitionedJoin='false', + schema = 'MQTEST' + ); + + +``` + + diff --git a/docs/oracleSink.md b/docs/oracleSink.md new file mode 100644 index 000000000..47ddd8371 --- /dev/null +++ b/docs/oracleSink.md @@ -0,0 +1,55 @@ +## 1.格式: +``` +CREATE TABLE tableName( + colName colType, + ... + colNameX colType + )WITH( + type ='oracle', + url ='jdbcUrl', + userName ='userName', + password ='pwd', + tableName ='tableName', + parallelism ='parllNum' + ); + +``` + +## 2.支持版本 + 10g 11g + +## 3.表结构定义 + +|参数名称|含义| +|----|---| +| tableName| oracle表名称| +| colName | 列名称| +| colType | 列类型 [colType支持的类型](colType.md)| + +## 4.参数: + +|参数名称|含义|是否必填|默认值| +|----|----|----|----| +|type |表名 输出表类型[mysq|hbase|elasticsearch|oracle]|是|| +|url | 连接oracle数据库 jdbcUrl |是|| +|userName | oracle连接用户名 |是|| +| password | oracle连接密码|是|| +| tableName | oracle表名称|是|| +| schema | oracle 的schema|否|当前登录用户| +| parallelism | 并行度设置|否|1| + +## 5.样例: +``` +CREATE TABLE MyResult( + channel VARCHAR, + pv VARCHAR + )WITH( + type ='oracle', + url ='jdbc:oracle:thin:@xx.xx.xx.xx:1521:orcl', + userName ='dtstack', + password ='abc123', + tableName ='pv2', + schema = 'MQTEST', + parallelism ='1' + ) + ``` \ No newline at end of file