From c198b8486de1eadc5fac2e8daa9294a0c80c08b1 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 12 Sep 2018 13:51:37 +0800 Subject: [PATCH 001/250] 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 586557c05f6a0269ecc8401bbc72a5f93cc2bebb Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 12 Sep 2018 14:01:37 +0800 Subject: [PATCH 002/250] 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 3f66d3baa4f7b0395f096fef5b25922d4d99144b Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 12 Sep 2018 17:51:46 +0800 Subject: [PATCH 003/250] 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 f54ba693d38b7c08674ab1e33e2ac2810197ac9e Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 12 Sep 2018 18:03:10 +0800 Subject: [PATCH 004/250] 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 69e2c3f356824b3e394944ded4e7f5a1b8f5f537 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 12 Sep 2018 20:03:53 +0800 Subject: [PATCH 005/250] =?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 ce44fbe52ec12ae967747be83e3ff44ae0cfdf5c Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 12 Sep 2018 21:34:16 +0800 Subject: [PATCH 006/250] =?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 71e441ac322e6baf7643a7869fee964ceb40b22a Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 12 Sep 2018 22:00:19 +0800 Subject: [PATCH 007/250] 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 efb111866e28fbd15fc1c75702ef6f81b2cfb4f3 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 13 Sep 2018 11:37:21 +0800 Subject: [PATCH 008/250] 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 45dffb6f210385d3b1e6915a76bd9e511ffb2adc Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 13 Sep 2018 11:45:50 +0800 Subject: [PATCH 009/250] =?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 0e334149fd5079f6fdc233d1f7cfdfadd43fec30 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 13 Sep 2018 11:47:31 +0800 Subject: [PATCH 010/250] 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 22f8f79b7faf0eda6c8aaf75d737cf562e866665 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 13 Sep 2018 15:03:33 +0800 Subject: [PATCH 011/250] 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 a78bf25d8788f855603f0967223a932a0e3a4ad2 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 13 Sep 2018 16:18:38 +0800 Subject: [PATCH 012/250] 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 c913c96af7669499b86c0569aa2bf3b118461ae1 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 13 Sep 2018 16:48:33 +0800 Subject: [PATCH 013/250] =?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 1827f97a578d8d7db8ad2b94f9d051f566e5adac Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 13 Sep 2018 16:55:15 +0800 Subject: [PATCH 014/250] =?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 7f8542a2e7137b67a7505c16d0624ffa9b883fcb Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 13 Sep 2018 16:58:28 +0800 Subject: [PATCH 015/250] =?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 58614cc556249cddafcd6409f1f0e16a77557890 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 13 Sep 2018 17:01:18 +0800 Subject: [PATCH 016/250] =?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 8938b746e276a084d0f0cb91d56fd398912223a5 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 14 Sep 2018 11:10:18 +0800 Subject: [PATCH 017/250] 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 4f1baba26ac835001b1323c5f09fd2b94ed553f4 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 14 Sep 2018 14:02:22 +0800 Subject: [PATCH 018/250] 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 0eee8fd233073decf80ddefce736852c9a797ba4 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 14 Sep 2018 14:28:59 +0800 Subject: [PATCH 019/250] 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 2a6688db17bf2d0037f2e617d14408d6fa4c0618 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 14 Sep 2018 14:43:33 +0800 Subject: [PATCH 020/250] 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 5d16aeb927362d0909d441640666df82b4781e92 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 14 Sep 2018 16:06:48 +0800 Subject: [PATCH 021/250] 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 1f154a82b4756bed66e13d124c816a94a3aefe32 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Tue, 18 Sep 2018 14:21:28 +0800 Subject: [PATCH 022/250] 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 ba341eaddd6f683e050147a14a6a3ce2e9b615b5 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Tue, 18 Sep 2018 15:47:30 +0800 Subject: [PATCH 023/250] 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 0a650e2b7f79e880b38d95692e3529b202adc42e Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 18 Sep 2018 16:14:06 +0800 Subject: [PATCH 024/250] =?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 b0256c968d6e6f9bf0110f9c8a153daa2e95d469 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 20 Sep 2018 16:34:56 +0800 Subject: [PATCH 025/250] 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 294e614e98dbb93375100a4998638046b5bd0de0 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 20 Sep 2018 17:28:07 +0800 Subject: [PATCH 026/250] 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 b112db7337765ac0ecafec8c158c692c1adabd76 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 20 Sep 2018 19:49:46 +0800 Subject: [PATCH 027/250] 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 91fb4902a218475eb866dfcd3e9bdfe9802460f1 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 27 Sep 2018 15:37:23 +0800 Subject: [PATCH 028/250] 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 01a4ee3ad1123be7540293b7d98fd13281208777 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 28 Sep 2018 11:31:17 +0800 Subject: [PATCH 029/250] 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 5bd8de4735db56b7df68b9299422b8015630b89d Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 28 Sep 2018 11:34:35 +0800 Subject: [PATCH 030/250] 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 4c50f0714a4fa203cd6d3758c3fabf7e3afe8e20 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 28 Sep 2018 14:12:02 +0800 Subject: [PATCH 031/250] 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 8ec483d91b9c31fa280fdb076372ec9d39c6a790 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 28 Sep 2018 14:17:58 +0800 Subject: [PATCH 032/250] 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 f6895a83121eb499a6d74d730cbc39f7e00c41a0 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 28 Sep 2018 14:28:01 +0800 Subject: [PATCH 033/250] 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 ad1cfe6ff0592b91c7606f1979394d64b7d872c2 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 28 Sep 2018 14:30:12 +0800 Subject: [PATCH 034/250] 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 ad37018f40775bfc55f18de90c7372142f7d9e48 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 28 Sep 2018 14:31:00 +0800 Subject: [PATCH 035/250] 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 4baea266e008160e60495def81a04e4d83b68c50 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 28 Sep 2018 19:36:05 +0800 Subject: [PATCH 036/250] 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 6286a9f62e629652d91c68c2453582c5faf73d7b Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 28 Sep 2018 19:48:05 +0800 Subject: [PATCH 037/250] 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 418c62ccb2b92c52bb32458cefc307d36be33bc5 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 28 Sep 2018 20:02:28 +0800 Subject: [PATCH 038/250] 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 f7acdad9642d6eaf0b5e711f16e5d7270fa003ad Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sun, 30 Sep 2018 10:36:15 +0800 Subject: [PATCH 039/250] 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 c5053badbe2500407a9cdf004006a9a1c2f13d17 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sun, 30 Sep 2018 10:44:29 +0800 Subject: [PATCH 040/250] 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 98f683cbac05e6e3d543e5241d20e19cea45fe29 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sun, 30 Sep 2018 11:00:23 +0800 Subject: [PATCH 041/250] 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 38f83a8c84a7b4109e5108e078e0e8568be42283 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sun, 30 Sep 2018 11:33:16 +0800 Subject: [PATCH 042/250] 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 0bebb7fa2beff5b7564b1de28da3c4f46ba29ac2 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Tue, 9 Oct 2018 14:57:06 +0800 Subject: [PATCH 043/250] 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 859549f1c8977cd07009e430ee44d69981489e0a Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 10 Oct 2018 15:23:16 +0800 Subject: [PATCH 044/250] 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 f237049f1350a1a4768a1ee42ecf1957778d0649 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 10 Oct 2018 15:27:14 +0800 Subject: [PATCH 045/250] 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 9725a670b9aa01112fdd36f5f7546490facb7042 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 10 Oct 2018 15:56:41 +0800 Subject: [PATCH 046/250] 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 356f6051166fddceee67e71012b5d3e98184029d Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 10 Oct 2018 16:04:06 +0800 Subject: [PATCH 047/250] 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 b1d08c8eefcf4024d193d5dfd373a4207b8a436a Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 10 Oct 2018 16:10:30 +0800 Subject: [PATCH 048/250] 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 e50936b9c16079fe60ef4f69b4dcada9f4b0f088 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 10 Oct 2018 20:00:26 +0800 Subject: [PATCH 049/250] 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 216a131a0bfc1dcbad9b202add2afcf23d3bd28b Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 10 Oct 2018 20:21:25 +0800 Subject: [PATCH 050/250] 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 b5be9c0e277569b33698d1545137409cf0d65bd7 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 11 Oct 2018 09:17:37 +0800 Subject: [PATCH 051/250] 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 2b7695f19975eabfc0b5c9ff1cdfe8a0355b865a Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 11 Oct 2018 09:19:07 +0800 Subject: [PATCH 052/250] 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 ae2f779f93483e7bf9677b807f7a6fb65165e7cf Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 11 Oct 2018 09:19:41 +0800 Subject: [PATCH 053/250] 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 4f98da05c5559cffe74427dbed517c8ac3cc432f Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 11 Oct 2018 17:40:45 +0800 Subject: [PATCH 054/250] 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 a64da8637c11bf964116968e5bc8a19912308e9b Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 11 Oct 2018 17:52:46 +0800 Subject: [PATCH 055/250] 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 1680163cb1296ab7ec24edfc39ba3ef79b980ae9 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Tue, 16 Oct 2018 17:06:07 +0800 Subject: [PATCH 056/250] 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 cbd9d9fe701d80088ca3cb1737f105242fc73a64 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Tue, 16 Oct 2018 17:20:11 +0800 Subject: [PATCH 057/250] 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 74f0cf62ee2ce7a5be3547b30f6056d4334e6e36 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 18 Oct 2018 13:55:42 +0800 Subject: [PATCH 058/250] 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 a7698b51d3b5546279e42a83ae674876405fa602 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 18 Oct 2018 16:12:18 +0800 Subject: [PATCH 059/250] =?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 4ec4f94da8c1aba3377dbe46e80a826c92a1056a Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 18 Oct 2018 21:24:56 +0800 Subject: [PATCH 060/250] 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 05db24e66efb01738fcadb17cac51c1c9feedce2 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 18 Oct 2018 21:39:34 +0800 Subject: [PATCH 061/250] 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 437ff39a0c845b17021bac021605d050b32366e1 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 18 Oct 2018 21:40:29 +0800 Subject: [PATCH 062/250] 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 b4f424c97eb2a82b16455186a4f909da3a70531e Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 18 Oct 2018 21:43:17 +0800 Subject: [PATCH 063/250] 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 82fd7fec6659be7986c9b68e738d18b4ce0855c5 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/250] 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 c50bd9a3d46e6ebbe9f0a6b5416f87f4e73c4af5 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 19 Oct 2018 10:13:20 +0800 Subject: [PATCH 065/250] 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 e9d9bece8a1565fea2667fed41c486f44f872cdb Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 19 Oct 2018 16:41:35 +0800 Subject: [PATCH 066/250] 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 dbd0ca905f3abf18dad7891bbf47bf92f408fdab Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 19 Oct 2018 20:20:17 +0800 Subject: [PATCH 067/250] 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 a89f0a09bbf80da40448d64aa1aa3f6b596c5cf3 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/250] 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/250] 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 f84aeae6f8035d6b9c6db7f2f8b72a63e21bd01f Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 20 Oct 2018 14:18:31 +0800 Subject: [PATCH 070/250] 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 05dc2d87268bea47f367b0cca6f5ad258827c93d 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/250] 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 d709c295adc4b48ae9155dad8b71b29d8463f787 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/250] 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 2875e4991205ca0935ecaddb748c40b526bcdf1f Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Sun, 21 Oct 2018 22:50:25 +0800 Subject: [PATCH 073/250] 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 eee0a9a1e9146074e60d9c69af36085fab703712 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 22 Oct 2018 16:53:30 +0800 Subject: [PATCH 074/250] 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 272dfe87fe4f93b05db1e147db049df37ab185a9 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 22 Oct 2018 17:22:56 +0800 Subject: [PATCH 075/250] 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 8f2c6b164cb8ea8dc38b0d25857af7d7536b3cdc Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 22 Oct 2018 17:30:35 +0800 Subject: [PATCH 076/250] 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 92b403734815b8bdb75267864ff46b93cf462c3d Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 22 Oct 2018 17:39:04 +0800 Subject: [PATCH 077/250] 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 8e570ff537dbeb58e8b888dfafb480b3f641a262 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Mon, 22 Oct 2018 19:25:52 +0800 Subject: [PATCH 078/250] 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 6f7ac17c279965f97da72f0aa989d6d4bad679c5 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Mon, 22 Oct 2018 21:08:21 +0800 Subject: [PATCH 079/250] 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 b935f28fdc3902c283fdb5e917ae2e36e7f0b4ff 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 080/250] 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 b7c01465a69f14344f669b0abf69f5a8d7a77827 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 24 Oct 2018 09:58:23 +0800 Subject: [PATCH 081/250] 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 2d9985bcf3230a05b41f45a3e6f6c38239bce046 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 24 Oct 2018 14:43:05 +0800 Subject: [PATCH 082/250] 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 8a109e4c6db646d84f41ac472cef343fb0abdfef Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 24 Oct 2018 14:46:01 +0800 Subject: [PATCH 083/250] 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 9138a7e4132f017ba8710c7c38a38c134abf674c Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 25 Oct 2018 21:16:27 +0800 Subject: [PATCH 084/250] 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 480179345d8e1d9bc3657fb64c637e05af07d27f Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 1 Nov 2018 14:09:41 +0800 Subject: [PATCH 085/250] 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 f1fbf8d7e7a70f73a3b80d5c80a905da57b0f066 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 2 Nov 2018 16:02:43 +0800 Subject: [PATCH 086/250] 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 cbbb44646bdae862eceac69655bc5722251e7f12 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 2 Nov 2018 16:07:46 +0800 Subject: [PATCH 087/250] 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 7e904b512a349d274471eb5e8992ee93961bba39 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 2 Nov 2018 16:12:04 +0800 Subject: [PATCH 088/250] 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 827e78c964d672ae2225a83d43192871c37038c6 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 2 Nov 2018 16:13:04 +0800 Subject: [PATCH 089/250] 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 660ff348bd4162ea3b5d960459814baf20d8404f Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 3 Nov 2018 11:22:31 +0800 Subject: [PATCH 090/250] 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 aea00f721b9acf656bc7fe84565330f30e713604 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 3 Nov 2018 11:31:23 +0800 Subject: [PATCH 091/250] 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 3b8a531436a44f51c2738f8b34b2f743bb268a58 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 092/250] 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 093/250] 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 94ea2a1bea0b2c2e6ad7ce9abe94bff13853b56c Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Tue, 6 Nov 2018 20:23:40 +0800 Subject: [PATCH 094/250] 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 7e16c3319f9a902b3c7f48382c5fa530c5f0557f Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Tue, 6 Nov 2018 20:26:04 +0800 Subject: [PATCH 095/250] 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 49ac9d95077fe8201b753a0ad70d2fd513fef9cf 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 096/250] 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 226474c350e8ca0859750692064833ab9ea9487d Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Wed, 7 Nov 2018 13:58:45 +0800 Subject: [PATCH 097/250] =?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 a5df67c813f4d9223f70330df30a6425fc4db783 Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Wed, 7 Nov 2018 17:02:04 +0800 Subject: [PATCH 098/250] 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 c312e57313816b395b16b190938c1299ae82c0eb Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Wed, 7 Nov 2018 17:58:01 +0800 Subject: [PATCH 099/250] 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 f819c3c364d3d67741926a33f01ae18c7f8b25c0 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 7 Nov 2018 21:33:39 +0800 Subject: [PATCH 100/250] 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 6650e03f97a9b717a03304523307d80ab7d51125 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 7 Nov 2018 21:53:01 +0800 Subject: [PATCH 101/250] 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 8c09fdb564e77bd57c11b4197a14d2d43a1cc470 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 8 Nov 2018 10:05:32 +0800 Subject: [PATCH 102/250] 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 9828b9405548f6a862bbcde19925a4e78014bada Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 8 Nov 2018 16:56:45 +0800 Subject: [PATCH 103/250] 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 c27f901aac28ac156de83c018feccb4766f49e86 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 8 Nov 2018 16:58:16 +0800 Subject: [PATCH 104/250] 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 620c16bc4affa9f2c3bff9c614c6b433e83744e4 Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Thu, 8 Nov 2018 23:10:14 +0800 Subject: [PATCH 105/250] 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 4fd55827d6478acafe83b6eda467aeea8f99bb39 Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Fri, 9 Nov 2018 11:08:32 +0800 Subject: [PATCH 106/250] 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 3119b13767cd6c5b53c29279085219c6b6eaac38 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 9 Nov 2018 11:26:19 +0800 Subject: [PATCH 107/250] 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 accc454f756f03724bf1de2790cd0142819e45d2 Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Fri, 9 Nov 2018 16:02:42 +0800 Subject: [PATCH 108/250] 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 cb6e3fea9130111bd18ade881bc9113f95dd83bb Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Fri, 9 Nov 2018 16:09:56 +0800 Subject: [PATCH 109/250] 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 bffa0a7b051362190ed9b4aa46bacd9211397fc1 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 110/250] 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 a01beb9ecbc4967a2a28ebbaf549719e84ca37f6 Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Mon, 12 Nov 2018 12:44:57 +0800 Subject: [PATCH 111/250] 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 fa160a229285da5ac998c28e6fd2036efc3e551a Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 12 Nov 2018 21:07:06 +0800 Subject: [PATCH 112/250] 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 a343a3246a724124a811813e0c89a0adb8e8d03a 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 113/250] 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 5da737377a8ac3cb64076c2f88f91e9931d18e1e Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 13 Nov 2018 14:55:11 +0800 Subject: [PATCH 114/250] =?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 aa49a67579db3b691d04b805cf48285765f3dab5 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 115/250] 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 ba347adb620b983c41f6412b30ae034924cefcff 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 116/250] 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 d672a9874ce82baff4419d539038b2abd191d92d Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 14 Nov 2018 20:29:39 +0800 Subject: [PATCH 117/250] 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 0caa82e0e34be39534a80f2a801d3b958fe47dd2 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 118/250] 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 fe9634e58c3f4e8e82b6b68185e8f6be013411c2 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 119/250] 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 3a7e89476e079596e7981a4181de1cf2fa5faba2 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 120/250] 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 05f7530698eb7ed033b7b412dc5a19c96db8dc96 Mon Sep 17 00:00:00 2001 From: xuqianjin Date: Thu, 15 Nov 2018 14:23:16 +0800 Subject: [PATCH 121/250] 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 6312499a01f3acc714a6018d8d04fa4023fc521f 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 122/250] 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 d0d48652c6cadb39649a5797af3b8b266aff6dc0 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 123/250] 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 8a0e2230c70e97e78886748015d2ffb3f60e6cba 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 124/250] 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 5425de26466ab455ae28d85615775a1703d8989a Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 16 Nov 2018 14:24:02 +0800 Subject: [PATCH 125/250] 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 9f4ddaebf3d77e783e954871c058b55d67839e46 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 126/250] 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 a631062f3bdd4c0560fa3cda7f47d5a7946db24d Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 16 Nov 2018 16:53:31 +0800 Subject: [PATCH 127/250] 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 0a9b3208b062b9a996222c24a152d148214eaaae Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 16 Nov 2018 17:13:02 +0800 Subject: [PATCH 128/250] 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 f51540bd391a58990a610186b05baeec59f14295 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 16 Nov 2018 18:56:46 +0800 Subject: [PATCH 129/250] 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 130/250] 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 1ef29b0dc3e80cfa5a737af7e5d8c6ae2c9244c6 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 17 Nov 2018 14:33:23 +0800 Subject: [PATCH 131/250] 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 b4a2693a2f7371c03f63a10f661442fbef5766a0 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Sat, 17 Nov 2018 15:07:19 +0800 Subject: [PATCH 132/250] 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 c94f2bf08c52273cda85a93d4480fb2972a8b60c Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 17 Nov 2018 15:13:41 +0800 Subject: [PATCH 133/250] 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 e9c870a3c4515a59ac7c48baef382e74a9331954 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 17 Nov 2018 17:01:37 +0800 Subject: [PATCH 134/250] 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 c3a92d25684662c6832693d5d26005a380303307 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 17 Nov 2018 17:37:14 +0800 Subject: [PATCH 135/250] 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 53c8b20db1183afe271873a53a8435cd6d7ca8ec Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 19 Nov 2018 14:08:21 +0800 Subject: [PATCH 136/250] 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 277bedcbc7a5ff91c74649eee305db0366b449e4 Mon Sep 17 00:00:00 2001 From: zhihui-ge <2972333955@qq.com> Date: Mon, 19 Nov 2018 20:00:41 +0800 Subject: [PATCH 137/250] 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 b138bac5551a3707a02957023566ae607a5162ad Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 19 Nov 2018 22:13:05 +0800 Subject: [PATCH 138/250] 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 35f8567eddf602e4776c5af4b08705fe722d4763 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 20 Nov 2018 21:29:33 +0800 Subject: [PATCH 139/250] 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 0af546ddef66bcdeb86698823534a885e1305a88 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 22 Nov 2018 10:21:08 +0800 Subject: [PATCH 140/250] 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 3b5298c99186c3a79daea963a567e8d5ec118c7a Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 22 Nov 2018 10:57:36 +0800 Subject: [PATCH 141/250] 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 88f4287e9e7bda0644b541b63dc662a3e2d402d4 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 22 Nov 2018 16:02:16 +0800 Subject: [PATCH 142/250] 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 143/250] 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 5da0748aef12ba962680790bbcc6a94a009f03dc Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 22 Nov 2018 17:17:28 +0800 Subject: [PATCH 144/250] 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 d00f5b45f4daae4e25bf5f7455ea3334ef697601 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 22 Nov 2018 21:50:15 +0800 Subject: [PATCH 145/250] 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 146/250] 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 57c64d9e93b53d8dd81ade9f55081d687e695a8e Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Fri, 23 Nov 2018 17:01:45 +0800 Subject: [PATCH 147/250] 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 eb90a4b21a04836f4cb30e137d07869126ab68cd Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Fri, 23 Nov 2018 17:17:10 +0800 Subject: [PATCH 148/250] 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 154cf4cd8d3e25a679ed085009d8aae06f775162 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 23 Nov 2018 18:03:57 +0800 Subject: [PATCH 149/250] 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 86af0ec838569446699f5a209be60de2d33b54a9 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 23 Nov 2018 21:31:32 +0800 Subject: [PATCH 150/250] 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 3be45371565cf13b3bed676c2136b7104be3e953 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 26 Nov 2018 11:02:00 +0800 Subject: [PATCH 151/250] 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 152/250] 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 96c56176f10abc524bb790b101a0dcaa43d31c86 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Mon, 26 Nov 2018 11:12:29 +0800 Subject: [PATCH 153/250] 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 251440bdb530b19b03a91f0207be4e670ace1614 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Mon, 26 Nov 2018 21:42:33 +0800 Subject: [PATCH 154/250] 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 742312ae7ef4d9ba5154e6f85972ee8bdb559171 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Mon, 26 Nov 2018 21:50:30 +0800 Subject: [PATCH 155/250] 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 3cea7224cdaee072128c47e83841b11be807f1cc Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Mon, 26 Nov 2018 21:59:15 +0800 Subject: [PATCH 156/250] 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 34f28c5f619af789033f045027e20754e1ffbfb6 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Tue, 27 Nov 2018 20:37:12 +0800 Subject: [PATCH 157/250] 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 e6035bf03dda240171d32aff7638214deb0c57f8 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Tue, 27 Nov 2018 20:45:30 +0800 Subject: [PATCH 158/250] 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 f9b6efdc4e17f8852543a075c798c55ac7d47244 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Tue, 27 Nov 2018 21:16:12 +0800 Subject: [PATCH 159/250] 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 2c2ae4dbca2057fa661ec9d50c097d53d12ed384 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Tue, 27 Nov 2018 21:19:25 +0800 Subject: [PATCH 160/250] 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 1bf0b00e98c619c037c57bea13d831f6fd283d72 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 28 Nov 2018 09:36:23 +0800 Subject: [PATCH 161/250] 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 197d52149abbdb613b6a47ac1e5b338c444d8ca2 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 28 Nov 2018 09:37:00 +0800 Subject: [PATCH 162/250] 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 c1f29d2457b7513ea0667d184900251492b99239 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 28 Nov 2018 09:49:47 +0800 Subject: [PATCH 163/250] 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 2a533b80c800c99fe0f4d3b5608a74434c4013b2 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 28 Nov 2018 09:58:57 +0800 Subject: [PATCH 164/250] 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 235d7429182ece2108be3c77aa18e8e9c790749e Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Wed, 28 Nov 2018 16:58:02 +0800 Subject: [PATCH 165/250] 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 166/250] 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 f696bfbf507975ad59a8514af13b27e2d06990c6 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 29 Nov 2018 10:29:22 +0800 Subject: [PATCH 167/250] 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 56f70aec7fd4c646cac2ef845e034e6bdbfe240a Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 29 Nov 2018 14:59:59 +0800 Subject: [PATCH 168/250] 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 3832dca86bda1c3c393797d3472b7d38ae8f0212 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Thu, 29 Nov 2018 20:25:08 +0800 Subject: [PATCH 169/250] 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 25cc3451e6f40c3fd47a04c33a7fa968fd163c95 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Fri, 30 Nov 2018 17:57:21 +0800 Subject: [PATCH 170/250] 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 f4871822249e4140d363fd3d0647f054deba0833 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Sat, 1 Dec 2018 12:17:29 +0800 Subject: [PATCH 171/250] 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 379b9a53ea9d3c3b8286d8d30d0c65f0444d2a7a Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Fri, 30 Nov 2018 19:07:10 +0800 Subject: [PATCH 172/250] 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 7a498c9ef808952b0b2213fa2bfacf0b7b3448bd Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Mon, 3 Dec 2018 09:47:19 +0800 Subject: [PATCH 173/250] 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 72df82e3ea1d141d2d7ba5599b43f8a1f169e347 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 3 Dec 2018 16:33:08 +0800 Subject: [PATCH 174/250] 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 175/250] 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 1e9b14dd43a1ac0f0092db8f7d87a75af18d2bf1 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 4 Dec 2018 10:42:40 +0800 Subject: [PATCH 176/250] 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 508050c72b9e8cb4b8f5f5d159bb01b4587f5f3c Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 4 Dec 2018 11:29:39 +0800 Subject: [PATCH 177/250] 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 be12caaf278758308c2d0a26bbdf1bd3410b0b95 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Tue, 4 Dec 2018 17:26:36 +0800 Subject: [PATCH 178/250] 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 d79838612905ed4eda5a3abe23ee1f96b659c344 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Tue, 4 Dec 2018 21:29:58 +0800 Subject: [PATCH 179/250] 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 930e9195cf82d43cdff495bcc64bbc0a6bfa0e14 Mon Sep 17 00:00:00 2001 From: renjianxu <313328862@qq.com> Date: Tue, 4 Dec 2018 21:39:04 +0800 Subject: [PATCH 180/250] 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 302670a51a9e1849a35a0387fa575cfe1a192cf0 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Wed, 5 Dec 2018 11:39:05 +0800 Subject: [PATCH 181/250] 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 182/250] 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 a455b675f67b2f74e4a381976d20833a141079d1 Mon Sep 17 00:00:00 2001 From: XuQianJin-Stars Date: Wed, 5 Dec 2018 18:52:20 +0800 Subject: [PATCH 183/250] 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 d91101ca514af2e6c57967d0f3ff2b6677a2beaf Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Wed, 5 Dec 2018 20:53:51 +0800 Subject: [PATCH 184/250] 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 b382bbe2efb67f8ccf8111c0d7f208add2eb6352 Mon Sep 17 00:00:00 2001 From: XuQianJin-Stars Date: Thu, 6 Dec 2018 09:54:18 +0800 Subject: [PATCH 185/250] 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 765bff604ed014ff0d6932f6e08e637605658751 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 6 Dec 2018 10:24:58 +0800 Subject: [PATCH 186/250] 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 b41cc0ea7f3eb01dfaef5c8cf6ed6a67e6c2ec78 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 10 Dec 2018 11:36:58 +0800 Subject: [PATCH 187/250] 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 a75a0b7c38585af6c15552c9d68f45325c9a73e7 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Mon, 10 Dec 2018 17:05:34 +0800 Subject: [PATCH 188/250] 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 20:36:55 +0800 Subject: [PATCH 189/250] 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 f223993275b3973d4fdf8c3ef2ca1e176c008a06 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Tue, 11 Dec 2018 09:34:06 +0800 Subject: [PATCH 190/250] 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 a635352d15fcc5b023f8b058d4957982afcde223 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 11 Dec 2018 14:43:53 +0800 Subject: [PATCH 191/250] 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 192/250] 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 72a90a92039e24e9f77bb671380435e4c89d8b10 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 11 Dec 2018 17:18:42 +0800 Subject: [PATCH 193/250] 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 d783df65f96f1c3f33029b3da1d0bb27c7245560 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 11 Dec 2018 18:15:30 +0800 Subject: [PATCH 194/250] 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 c9ecc83150f531f1d5d3fbdb27b44e5112021223 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 11 Dec 2018 18:40:19 +0800 Subject: [PATCH 195/250] 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 ab515e73b1e298f3195b3c4c758021cc7345d181 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Wed, 12 Dec 2018 13:51:52 +0800 Subject: [PATCH 196/250] 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 3ae08ec17b0f0dcfbfce43644221bca285a3bc59 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 12 Dec 2018 17:42:53 +0800 Subject: [PATCH 197/250] 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 7406d68de0c190416902f96369cd3d2e3840944a Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 12 Dec 2018 19:07:21 +0800 Subject: [PATCH 198/250] 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 a4b18db48ea87e14065ca8f72074c6afd53e6618 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Thu, 13 Dec 2018 15:16:59 +0800 Subject: [PATCH 199/250] 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 0016a51db36d9971e7fdbe8b5c72bc239195020f Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 14 Dec 2018 16:40:22 +0800 Subject: [PATCH 200/250] 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 83e7ddc74b292fb755bd24b321dcf37276e57711 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 20 Dec 2018 19:33:49 +0800 Subject: [PATCH 201/250] 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 61d81e8df39842881921ded0eb6aa6842393254e Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 8 Jan 2019 20:24:29 +0800 Subject: [PATCH 202/250] 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 85c27376e1b0c8b46e29901c87ed8efbfebfa92c Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 9 Jan 2019 11:09:24 +0800 Subject: [PATCH 203/250] 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 b82edb98e8c7bc6c470c64e5a646fdd19cf01605 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 10 Jan 2019 13:54:21 +0800 Subject: [PATCH 204/250] 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 12bd803ebf9993def741d8e1ee84aa6dc13165f6 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 10 Jan 2019 14:18:33 +0800 Subject: [PATCH 205/250] 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 84a44b240e01154d70054ec7aba42d9ae32a63b0 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 10 Jan 2019 17:53:36 +0800 Subject: [PATCH 206/250] 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 70b1c778c095cfafb97426dab591f3b519da6464 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 11 Jan 2019 17:24:33 +0800 Subject: [PATCH 207/250] =?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 075064a11d0f2087cf1f5c9820f789fe34e094ff Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Fri, 11 Jan 2019 20:57:21 +0800 Subject: [PATCH 208/250] 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 6144eaee4b238763d5dd91f0018fc0a5a0445635 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Fri, 18 Jan 2019 16:58:46 +0800 Subject: [PATCH 209/250] 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 6f4cb9ac8254902e3e3070328b84516d22b52ac8 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 20 Feb 2019 10:48:28 +0800 Subject: [PATCH 210/250] Update README.md --- README.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/README.md b/README.md index 93edde5c3..c32188d55 100644 --- a/README.md +++ b/README.md @@ -11,21 +11,19 @@ # 已支持 * 源表: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 # 后续开发计划 * 增加SQL支持CEP * 维表快照 * sql优化(谓词下移等) * serverSocket 源表 - * console 结果表 * kafka avro格式 * topN ## 1 快速起步 ### 1.1 运行模式 - * 单机模式:对应Flink集群的单机模式 * standalone模式:对应Flink集群的分布式模式 * yarn模式:对应Flink集群的yarn模式 From 626176b946fe45f505e7fbc4066c167807c77e0e Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 20 Feb 2019 10:49:10 +0800 Subject: [PATCH 211/250] Update README.md --- README.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/README.md b/README.md index c32188d55..810be11b5 100644 --- a/README.md +++ b/README.md @@ -9,7 +9,7 @@ # 已支持 - * 源表:kafka 0.9,1.x版本 + * 源表:kafka 0.9,1.x版本,serverSocket * 维表:mysql,SQlServer,oracle,hbase,mongo,redis,cassandra * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra,console @@ -17,7 +17,6 @@ * 增加SQL支持CEP * 维表快照 * sql优化(谓词下移等) - * serverSocket 源表 * kafka avro格式 * topN From e63eae5ea95f02d7cf527bd066592b59f7ce1d05 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 28 Feb 2019 12:01:53 +0800 Subject: [PATCH 212/250] Lex.MYSQL --- core/src/main/java/com/dtstack/flink/sql/Main.java | 10 +++++----- 1 file changed, 5 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 37ffa4df4..533b9a6d4 100644 --- a/core/src/main/java/com/dtstack/flink/sql/Main.java +++ b/core/src/main/java/com/dtstack/flink/sql/Main.java @@ -101,6 +101,11 @@ public class Main { private static final int delayInterval = 10; //sec + private static org.apache.calcite.sql.parser.SqlParser.Config config = org.apache.calcite.sql.parser.SqlParser + .configBuilder() + .setLex(Lex.MYSQL) + .build(); + public static void main(String[] args) throws Exception { Options options = new Options(); @@ -189,11 +194,6 @@ 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,config).parseStmt(); String tmpSql = ((SqlInsert) sqlNode).getSource().toString(); tmp.setExecSql(tmpSql); From ea3f36777012ec6f540d143bad9bfcb38b92ed59 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 28 Feb 2019 14:32:35 +0800 Subject: [PATCH 213/250] null --- .../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 ec369b28f..1e603c983 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 @@ -98,7 +98,7 @@ public Row deserialize(byte[] message) throws IOException { try { numInRecord.inc(); - numInBytes.inc(message.length); + if(message!=null){numInBytes.inc(message.length);} JsonNode root = objectMapper.readTree(message); Row row = new Row(fieldNames.length); 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..925ef7154 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 @@ -98,7 +98,7 @@ public Row deserialize(byte[] message) throws IOException { try { numInRecord.inc(); - numInBytes.inc(message.length); + if(message!=null){numInBytes.inc(message.length);} JsonNode root = objectMapper.readTree(message); 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 78f1c17ff..002cff392 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 @@ -100,7 +100,7 @@ public Row deserialize(byte[] message) throws IOException { try { numInRecord.inc(); - numInBytes.inc(message.length); + if(message!=null){numInBytes.inc(message.length);} JsonNode root = objectMapper.readTree(message); Row row = new Row(fieldNames.length); From 03343209990752707344d626dba61eb160b5d628 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 23 Mar 2019 14:29:13 +0800 Subject: [PATCH 214/250] =?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 b971db10d5a4dc1017200e62ae19662fd4a9fc05 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 23 Mar 2019 15:33:41 +0800 Subject: [PATCH 215/250] =?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 b941f397e660e04d204f94dec50ef072d35838e7 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Fri, 29 Mar 2019 11:26:30 +0800 Subject: [PATCH 216/250] 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 8c40de04c50f368187e24603738be4fd86f2ed66 Mon Sep 17 00:00:00 2001 From: yanxi0227 Date: Mon, 1 Apr 2019 14:40:58 +0800 Subject: [PATCH 217/250] 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 218/250] 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 3c1d7b98fdb4bbcafb783865a3ca3d6724a0a253 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 4 Apr 2019 11:57:41 +0800 Subject: [PATCH 219/250] =?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 d5c5bf11d7aa94505f58a1d4a6c33674933abaf4 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 4 Apr 2019 14:56:01 +0800 Subject: [PATCH 220/250] =?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 fe961ecc60202aecb541276d5955c43b4255e6e0 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Mon, 8 Apr 2019 20:14:19 +0800 Subject: [PATCH 221/250] 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 ef2f2d2fa66a1429555bf8f42f3b1d807e7d9483 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Mon, 8 Apr 2019 20:14:59 +0800 Subject: [PATCH 222/250] 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 3ba6572eb3ddb23ccad6ac647093d8f9f4360af7 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 14 Dec 2018 16:40:22 +0800 Subject: [PATCH 223/250] 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 3da8711dbcdcfaf408da48334fda287a00516a52 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 10 Apr 2019 09:56:07 +0800 Subject: [PATCH 224/250] 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 dfe3f6bb63718d030096ef0fb1b9afb0b7f691c1 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 10 Apr 2019 16:08:56 +0800 Subject: [PATCH 225/250] 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 226/250] 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 4ea3aecd45cf54c2b257891b7c2c897720f79aeb Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 11 Apr 2019 14:34:51 +0800 Subject: [PATCH 227/250] 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 f8b279863d52a9fc560c69ae02a7023ada0e3d47 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 11 Apr 2019 17:43:26 +0800 Subject: [PATCH 228/250] 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 bd2351cce85962cb10e534246886beb19531d7be Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Thu, 11 Apr 2019 17:53:38 +0800 Subject: [PATCH 229/250] 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 230/250] 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 38e13769a5a9841047b562123ea54aa3b620effb Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Sat, 20 Apr 2019 17:02:25 +0800 Subject: [PATCH 231/250] 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 c801fbe0548193136b3db542e91cad7d7eadd072 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 22 Apr 2019 16:46:02 +0800 Subject: [PATCH 232/250] =?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 41a0c067c941d460addff0cc211e627820985ae2 Mon Sep 17 00:00:00 2001 From: wenbao <1010467452@qq.com> Date: Tue, 23 Apr 2019 17:35:07 +0800 Subject: [PATCH 233/250] =?UTF-8?q?=E5=AE=8C=E6=88=90kudu=20sink=E5=92=8Cs?= =?UTF-8?q?ide=E7=9A=84=E6=9C=AC=E5=9C=B0=E5=BC=80=E5=8F=91=E5=92=8C?= =?UTF-8?q?=E6=B5=8B=E8=AF=95=20=E6=B7=BB=E5=8A=A0=E8=AF=B4=E6=98=8E?= =?UTF-8?q?=E6=96=87=E6=A1=A3?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/kuduSide.md | 127 ++++++ docs/kuduSink.md | 62 +++ 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 | 299 +++++++++++++ .../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 ++++ .../KuduOutputFormat.java | 308 ++++++++++++++ .../KuduSink.java | 104 +++++ .../table/KuduSinkParser.java | 54 +++ .../table/KuduTableInfo.java | 92 ++++ kudu/pom.xml | 45 ++ pom.xml | 1 + 19 files changed, 2228 insertions(+) create mode 100644 docs/kuduSide.md create mode 100644 docs/kuduSink.md 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/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 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..95b8a9924 --- /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 + + + + + + + + + + *:* + + 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..228c9827a --- /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 + + + + + + + + + + *:* + + 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..b996fea93 --- /dev/null +++ b/kudu/kudu-side/kudu-async-side/src/main/java/com/dtstack/flink/sql/side/kudu/KuduAsyncReqRow.java @@ -0,0 +1,299 @@ +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 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.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 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; +import java.util.concurrent.atomic.AtomicInteger; + +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 client; + + private KuduTable table; + + private KuduSideTableInfo kuduSideTableInfo; + + private KuduScanner.KuduScannerBuilder scannerBuilder; + + private KuduClient syncClient; + + private AtomicInteger atomicInteger = new AtomicInteger(0); + + + 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); + } + AsyncKuduClient client = asyncKuduClientBuilder.build(); + syncClient = client.syncClient(); + if (!syncClient.tableExists(tableName)) { + throw new IllegalArgumentException("Table Open Failed , please check table exists"); + } + table = syncClient.openTable(tableName); + LOG.info("connect kudu is successed!"); + } + scannerBuilder = syncClient.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; + } + } + String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); + List> cacheContent = Lists.newArrayList(); + KuduScanner kuduScanner = scannerBuilder.build(); + List rowList = Lists.newArrayList(); + //判断是否调用prc获取数据 + while (kuduScanner.hasMoreRows()) { + RowResultIterator results = kuduScanner.nextRows(); + //每次遍历一整条数据 + while (results.hasNext()) { + atomicInteger.incrementAndGet(); + 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); + } + } + Row row = fillData(input, oneRow); + if (openCache()) { + cacheContent.add(oneRow); + } + rowList.add(row); + } + } + if (0 != atomicInteger.get()) { + 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()); + } + } + atomicInteger.set(0); + } + + + @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 != syncClient) { + try { + syncClient.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); + } + } +} 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..81215d5c4 --- /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 + + + + + + + + + + *:* + + 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 7ef6c1fdd..8d6990165 100644 --- a/pom.xml +++ b/pom.xml @@ -23,6 +23,7 @@ sqlserver oracle cassandra + kudu From 0ebbe2f4f0e9c4eebb913f75eb191bfa96f9a302 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 24 Apr 2019 14:13:11 +0800 Subject: [PATCH 234/250] 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 17891c0992e1ab68f04f44d7385a9b14915ca89f Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Wed, 24 Apr 2019 21:03:43 +0800 Subject: [PATCH 235/250] 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 3b9b0bac139428bb874c67a6fac20eb68a730c82 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Fri, 26 Apr 2019 11:52:35 +0800 Subject: [PATCH 236/250] 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 5f8cc690e66810df7f8177f26ff771760393e152 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Sun, 28 Apr 2019 20:14:56 +0800 Subject: [PATCH 237/250] README.MD --- README.md | 270 ++---------------------------------------------------- 1 file changed, 10 insertions(+), 260 deletions(-) diff --git a/README.md b/README.md index 93edde5c3..06be83be9 100644 --- a/README.md +++ b/README.md @@ -8,266 +8,16 @@ > > * 扩展了输入和输出的性能指标到promethus -# 已支持 - * 源表:kafka 0.9,1.x版本 - * 维表:mysql,SQlServer,oracle,hbase,mongo,redis,cassandra - * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra +# 内容更新 -# 后续开发计划 - * 增加SQL支持CEP - * 维表快照 - * sql优化(谓词下移等) - * serverSocket 源表 - * console 结果表 - * kafka avro格式 - * topN +## 新特性: + 1. 支持从kafka读取嵌套JSON格式数据,暂不支持数组类型字段。例如: info.name varchar as info_name。 + 2. 支持kafka结果表数据写入。 + 3. 支持为ROWTIME绑定时区,默认为本地时区。例如:timezone="America/New_York" + 4. 支持从kafka自定义偏移量中消费数据。 + 5. -## 1 快速起步 -### 1.1 运行模式 +## BUG修复: - -* 单机模式:对应Flink集群的单机模式 -* standalone模式:对应Flink集群的分布式模式 -* yarn模式:对应Flink集群的yarn模式 - -### 1.2 执行环境 - -* Java: JDK8及以上 -* Flink集群: 1.4,1.5(单机模式不需要安装Flink集群) -* 操作系统:理论上不限 - -### 1.3 打包 - -进入项目根目录,使用maven打包: - -``` -mvn clean package -Dmaven.test.skip - -打包结束后,项目根目录下会产生plugins目录,plugins目录下存放编译好的数据同步插件包,在lib目下存放job提交的包 -``` - -### 1.4 启动 - -#### 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\} -``` - -#### 1.4.2 命令行参数选项 - -* **mode** - * 描述:执行模式,也就是flink集群的工作模式 - * local: 本地模式 - * standalone: 提交到独立部署模式的flink集群 - * yarn: 提交到yarn模式的flink集群(即提交到已有flink集群) - * yarnPer: yarn per_job模式提交(即创建新flink application) - * 必选:否 - * 默认值:local - -* **name** - * 描述:flink 任务对应名称。 - * 必选:是 - * 默认值:无 - -* **sql** - * 描述:执行flink sql 的主体语句。 - * 必选:是 - * 默认值:无 - -* **localSqlPluginPath** - * 描述:本地插件根目录地址,也就是打包后产生的plugins目录。 - * 必选:是 - * 默认值:无 - -* **remoteSqlPluginPath** - * 描述:flink执行集群上的插件根目录地址(将打包好的插件存放到各个flink节点上,如果是yarn集群需要存放到所有的nodemanager上)。 - * 必选:否 - * 默认值:无 - -* **addjar** - * 描述:扩展jar路径,当前主要是UDF定义的jar; - * 格式:json - * 必选:否 - * 默认值:无 - -* **confProp** - * 描述:一些参数设置 - * 格式: json - * 必选:是 (如无参数填写空json即可) - * 默认值:无 - * 可选参数: - * 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:// - * 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** - * 描述:flink配置文件所在的目录(单机模式下不需要),如/hadoop/flink-1.4.0/conf - * 必选:否 - * 默认值:无 - -* **yarnconf** - * 描述:Hadoop配置文件(包括hdfs和yarn)所在的目录(单机模式下不需要),如/hadoop/etc/hadoop - * 必选:否 - * 默认值:无 - -* **savePointPath** - * 描述:任务恢复点的路径 - * 必选:否 - * 默认值:无 - -* **allowNonRestoredState** - * 描述:指示保存点是否允许非还原状态的标志 - * 必选:否 - * 默认值:false - -* **flinkJarPath** - * 描述:per_job 模式提交需要指定本地的flink jar存放路径 - * 必选:否 - * 默认值:false - -* **queue** - * 描述:per_job 模式下指定的yarn queue - * 必选:否 - * 默认值:false - -## 2 结构 -### 2.1 源表插件 -* [kafka 源表插件](docs/kafkaSource.md) - -### 2.2 结果表插件 -* [elasticsearch 结果表插件](docs/elasticsearchSink.md) -* [hbase 结果表插件](docs/hbaseSink.md) -* [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 性能指标(新增) - -### 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 - - -## 4 样例 - -``` - -CREATE (scala|table) FUNCTION CHARACTER_LENGTH WITH com.dtstack.Kun; - - -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 ='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 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 ='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 -``` - -# 招聘 -1.大数据平台开发工程师,想了解岗位详细信息可以添加本人微信号ysqwhiletrue,注明招聘,如有意者发送简历至sishu@dtstack.com。 - + 1. oracle维表获取索引问题。 + 2. UDF类加载异常问题。 \ No newline at end of file From 09958d8b1386e9955a787bd2e47775c349231255 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 7 May 2019 10:22:29 +0800 Subject: [PATCH 238/250] 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 767626a8cf001a809fe4dd04b95b0019dc8656ce Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 7 May 2019 10:29:19 +0800 Subject: [PATCH 239/250] 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 baf23f941f532b4d2cde3f1b82ba88cbb0b38a2c Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Tue, 7 May 2019 16:06:20 +0800 Subject: [PATCH 240/250] =?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 3567a64e0e914578f46260d576feb938b8af998a Mon Sep 17 00:00:00 2001 From: wenbao <1010467452@qq.com> Date: Tue, 7 May 2019 16:25:25 +0800 Subject: [PATCH 241/250] =?UTF-8?q?=E4=B8=8E=E7=8E=B0=E6=9C=89pom=E4=BF=9D?= =?UTF-8?q?=E6=8C=81=E4=B8=80=E8=87=B4=20=E6=B7=BB=E5=8A=A0-${git.branch}?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- kudu/kudu-side/kudu-all-side/pom.xml | 2 +- kudu/kudu-side/kudu-async-side/pom.xml | 4 ++-- kudu/kudu-sink/pom.xml | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/kudu/kudu-side/kudu-all-side/pom.xml b/kudu/kudu-side/kudu-all-side/pom.xml index 95b8a9924..7629d0c75 100644 --- a/kudu/kudu-side/kudu-all-side/pom.xml +++ b/kudu/kudu-side/kudu-all-side/pom.xml @@ -77,7 +77,7 @@ + tofile="${basedir}/../../../plugins/kuduallside/${project.name}-${git.branch}.jar"/> diff --git a/kudu/kudu-side/kudu-async-side/pom.xml b/kudu/kudu-side/kudu-async-side/pom.xml index 228c9827a..7e2d96396 100644 --- a/kudu/kudu-side/kudu-async-side/pom.xml +++ b/kudu/kudu-side/kudu-async-side/pom.xml @@ -90,12 +90,12 @@ - + + tofile="${basedir}/../../../plugins/kuduasyncside/${project.name}-${git.branch}.jar"/> diff --git a/kudu/kudu-sink/pom.xml b/kudu/kudu-sink/pom.xml index 81215d5c4..3225352ec 100644 --- a/kudu/kudu-sink/pom.xml +++ b/kudu/kudu-sink/pom.xml @@ -70,7 +70,7 @@ + tofile="${basedir}/../../plugins/kudusink/${project.name}-${git.branch}.jar"/> From 9ec682f3ca12835117fb014785826d85f83e5689 Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 7 May 2019 18:53:46 +0800 Subject: [PATCH 242/250] 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 d0804c0eed111ab4df019c5c373524d0e2e7499b Mon Sep 17 00:00:00 2001 From: maqi <313328862@qq.com> Date: Tue, 7 May 2019 19:35:41 +0800 Subject: [PATCH 243/250] modify README.md --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 06be83be9..3ecb021da 100644 --- a/README.md +++ b/README.md @@ -15,9 +15,9 @@ 2. 支持kafka结果表数据写入。 3. 支持为ROWTIME绑定时区,默认为本地时区。例如:timezone="America/New_York" 4. 支持从kafka自定义偏移量中消费数据。 - 5. ## BUG修复: 1. oracle维表获取索引问题。 - 2. UDF类加载异常问题。 \ No newline at end of file + 2. Perjob模式下UDF类加载异常问题。 + 3. queue 参数设置无效的问题。 \ No newline at end of file From c786837fa494f10454cbb5c2ace2b44eff44dacb Mon Sep 17 00:00:00 2001 From: wenbao <1010467452@qq.com> Date: Fri, 24 May 2019 11:26:20 +0800 Subject: [PATCH 244/250] =?UTF-8?q?=E5=B0=86=E5=8E=9F=E6=9C=89=E7=9A=84?= =?UTF-8?q?=E5=BC=82=E6=AD=A5=E6=9F=A5=E8=AF=A2=E4=BF=AE=E6=94=B9=E4=B8=BA?= =?UTF-8?q?callBack=E6=96=B9=E5=BC=8F?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../flink/sql/side/kudu/KuduAsyncReqRow.java | 129 ++++++++++-------- 1 file changed, 75 insertions(+), 54 deletions(-) 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 index b996fea93..10fee59f4 100644 --- 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 @@ -4,11 +4,13 @@ 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.calcite.shaded.com.google.common.collect.Maps; 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; @@ -25,7 +27,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; public class KuduAsyncReqRow extends AsyncReqRow { @@ -42,18 +43,13 @@ public class KuduAsyncReqRow extends AsyncReqRow { private static final long serialVersionUID = 5028583854989267753L; - // private AsyncKuduClient client; + private AsyncKuduClient asyncClient; private KuduTable table; private KuduSideTableInfo kuduSideTableInfo; - private KuduScanner.KuduScannerBuilder scannerBuilder; - - private KuduClient syncClient; - - private AtomicInteger atomicInteger = new AtomicInteger(0); - + private AsyncKuduScanner.AsyncKuduScannerBuilder scannerBuilder; public KuduAsyncReqRow(RowTypeInfo rowTypeInfo, JoinInfo joinInfo, List outFieldInfoList, SideTableInfo sideTableInfo) { super(new KuduAsyncSideInfo(rowTypeInfo, joinInfo, outFieldInfoList, sideTableInfo)); @@ -92,15 +88,14 @@ private void connKuDu() throws KuduException { if (null != defaultOperationTimeoutMs) { asyncKuduClientBuilder.defaultOperationTimeoutMs(defaultOperationTimeoutMs); } - AsyncKuduClient client = asyncKuduClientBuilder.build(); - syncClient = client.syncClient(); - if (!syncClient.tableExists(tableName)) { + asyncClient = asyncKuduClientBuilder.build(); + if (!asyncClient.syncClient().tableExists(tableName)) { throw new IllegalArgumentException("Table Open Failed , please check table exists"); } - table = syncClient.openTable(tableName); + table = asyncClient.syncClient().openTable(tableName); LOG.info("connect kudu is successed!"); } - scannerBuilder = syncClient.newScannerBuilder(table); + scannerBuilder = asyncClient.newScannerBuilder(table); Integer batchSizeBytes = kuduSideTableInfo.getBatchSizeBytes(); Long limitNum = kuduSideTableInfo.getLimitNum(); Boolean isFaultTolerant = kuduSideTableInfo.getFaultTolerant(); @@ -169,46 +164,12 @@ public void asyncInvoke(Row input, ResultFuture resultFuture) throws Except return; } } - String[] sideFieldNames = sideInfo.getSideSelectFields().split(","); List> cacheContent = Lists.newArrayList(); - KuduScanner kuduScanner = scannerBuilder.build(); + AsyncKuduScanner asyncKuduScanner = scannerBuilder.build(); List rowList = Lists.newArrayList(); - //判断是否调用prc获取数据 - while (kuduScanner.hasMoreRows()) { - RowResultIterator results = kuduScanner.nextRows(); - //每次遍历一整条数据 - while (results.hasNext()) { - atomicInteger.incrementAndGet(); - 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); - } - } - Row row = fillData(input, oneRow); - if (openCache()) { - cacheContent.add(oneRow); - } - rowList.add(row); - } - } - if (0 != atomicInteger.get()) { - 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()); - } - } - atomicInteger.set(0); + Deferred data = asyncKuduScanner.nextRows(); + //从之前的同步修改为调用异步的Callback + data.addCallbackDeferring(new GetListRowCB(input, cacheContent, rowList, asyncKuduScanner, resultFuture, key)); } @@ -251,9 +212,9 @@ public String buildCacheKey(JsonArray jsonArray) { @Override public void close() throws Exception { super.close(); - if (null != syncClient) { + if (null != asyncClient) { try { - syncClient.close(); + asyncClient.close(); } catch (Exception e) { LOG.error("Error while closing client.", e); } @@ -296,4 +257,64 @@ private void setMapValue(Type type, Map oneRow, String sideField 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; + } + } + } From 5d2a07f5423026598c570d37f4e5a6311f302d4a Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Mon, 27 May 2019 13:53:54 +0800 Subject: [PATCH 245/250] Update README.md --- README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/README.md b/README.md index 810be11b5..241b90b03 100644 --- a/README.md +++ b/README.md @@ -14,7 +14,6 @@ * 结果表:mysql,SQlServer,oracle,hbase,elasticsearch5.x,mongo,redis,cassandra,console # 后续开发计划 - * 增加SQL支持CEP * 维表快照 * sql优化(谓词下移等) * kafka avro格式 From 6eb3258384abf87a5e94c87ca6dd373ecc36aafa Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Tue, 28 May 2019 14:06:58 +0800 Subject: [PATCH 246/250] fund --- .github/FUNDING.yml | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 .github/FUNDING.yml diff --git a/.github/FUNDING.yml b/.github/FUNDING.yml new file mode 100644 index 000000000..e69de29bb From e57ca7ed4f03023586649e201c94083f3ca7bf71 Mon Sep 17 00:00:00 2001 From: "zhengfeng.yao" Date: Mon, 10 Jun 2019 11:36:28 +0800 Subject: [PATCH 247/250] =?UTF-8?q?=E6=94=AF=E6=8C=81=E5=B5=8C=E5=A5=97JSO?= =?UTF-8?q?N=E9=9D=9E=E5=8F=B6=E5=AD=90=E8=8A=82=E7=82=B9=E6=8F=90?= =?UTF-8?q?=E5=8F=96=EF=BC=8C=E7=9B=B4=E6=8E=A5=E8=BD=AC=E4=B8=BAJSON?= =?UTF-8?q?=E5=AD=97=E7=AC=A6=E4=B8=B2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../kafka/CustomerJsonDeserialization.java | 67 ++++++++++++------ .../kafka/CustomerJsonDeserialization.java | 59 +++++++++++----- .../kafka/CustomerJsonDeserialization.java | 68 +++++++++++++------ 3 files changed, 132 insertions(+), 62 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 98aefdc99..960842c53 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 @@ -127,7 +127,12 @@ public Row deserialize(byte[] message) throws IOException { } } else { // Read the value as specified type - Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + Object value; + if (node.isValueNode()) { + value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + } else { + value = node.toString(); + } row.setField(i, value); } } @@ -143,41 +148,51 @@ 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 node; + return nodeAndJsonNodeMapping.get(nodeMappingKey); } + 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(); JsonNode child = jsonNode.get(next); String nodeKey = getNodeKey(prefix, next); - if (child.isValueNode()){ - nodeAndJsonNodeMapping.put(nodeKey, child); - }else { - parseTree(child, nodeKey); - } + if (child.isValueNode()) { + nodeAndJsonNodeMapping.put(nodeKey, child); + } else { + if (rowAndFieldMapping.containsValue(nodeKey)) { + nodeAndJsonNodeMapping.put(nodeKey, child); + } + parseTree(child, nodeKey); } + } } private String getNodeKey(String prefix, String nodeName){ @@ -188,6 +203,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/CustomerJsonDeserialization.java b/kafka10/kafka10-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index 968828c49..f6099a278 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 @@ -126,7 +126,12 @@ public Row deserialize(byte[] message) throws IOException { } } else { // Read the value as specified type - Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + Object value; + if (node.isValueNode()) { + value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + } else { + value = node.toString(); + } row.setField(i, value); } } @@ -144,19 +149,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); } @@ -166,18 +160,39 @@ 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()){ - nodeAndJsonNodeMapping.put(nodeKey, child); - }else { - parseTree(child, nodeKey); - } + if (child.isValueNode()) { + nodeAndJsonNodeMapping.put(nodeKey, child); + } else { + if (rowAndFieldMapping.containsValue(nodeKey)) { + nodeAndJsonNodeMapping.put(nodeKey, child); + } + parseTree(child, nodeKey); } + } } private String getNodeKey(String prefix, String nodeName){ @@ -188,6 +203,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/CustomerJsonDeserialization.java b/kafka11/kafka11-source/src/main/java/com/dtstack/flink/sql/source/kafka/CustomerJsonDeserialization.java index ac5aad6f2..cc3d57a80 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 @@ -129,7 +129,12 @@ public Row deserialize(byte[] message) throws IOException { } } else { // Read the value as specified type - Object value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + Object value; + if (node.isValueNode()) { + value = objectMapper.treeToValue(node, fieldTypes[i].getTypeClass()); + } else { + value = node.toString(); + } row.setField(i, value); } } @@ -147,18 +152,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); } public void setFailOnMissingField(boolean failOnMissingField) { @@ -167,18 +162,39 @@ public void setFailOnMissingField(boolean failOnMissingField) { 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 { - parseTree(child, nodeKey); + 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()) { + nodeAndJsonNodeMapping.put(nodeKey, child); + } else { + if (rowAndFieldMapping.containsValue(nodeKey)) { + nodeAndJsonNodeMapping.put(nodeKey, child); + } + parseTree(child, nodeKey); } + } } private String getNodeKey(String prefix, String nodeName){ @@ -189,6 +205,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; } From 33fdf482a0cc3f64236839fed0436459872662f6 Mon Sep 17 00:00:00 2001 From: "zhengfeng.yao" Date: Mon, 10 Jun 2019 16:09:02 +0800 Subject: [PATCH 248/250] =?UTF-8?q?=E6=94=AF=E6=8C=81=E5=B5=8C=E5=A5=97JSO?= =?UTF-8?q?N=E9=9D=9E=E5=8F=B6=E5=AD=90=E8=8A=82=E7=82=B9=E6=8F=90?= =?UTF-8?q?=E5=8F=96=EF=BC=8C=E7=9B=B4=E6=8E=A5=E8=BD=AC=E4=B8=BAJSON?= =?UTF-8?q?=E5=AD=97=E7=AC=A6=E4=B8=B2?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../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..a0d71c1ac 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)((\S+\.)*\S+)\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..9604b618d 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)((\S+\.)*\S+)\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..408981e53 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)((\S+\.)*\S+)\s+(\w+)\s+AS\s+(\w+)$"); static { keyPatternMap.put(KAFKA_NEST_FIELD_KEY, kafkaNestFieldKeyPattern); From 6427e6495f46e156ee0db3fe29d8e1783e91ef94 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Mon, 1 Jul 2019 16:52:31 +0800 Subject: [PATCH 249/250] Update README.md --- README.md | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/README.md b/README.md index a6b515099..60f76f621 100644 --- a/README.md +++ b/README.md @@ -8,13 +8,18 @@ > > * 扩展了输入和输出的性能指标到promethus -# 内容更新 +## BUG修复: + + 1. oracle维表获取索引问题。 + 2. Perjob模式下UDF类加载异常问题。 + 3. queue 参数设置无效的问题。 ## 新特性: 1. 支持从kafka读取嵌套JSON格式数据,暂不支持数组类型字段。例如: info.name varchar as info_name。 2. 支持kafka结果表数据写入。 3. 支持为ROWTIME绑定时区,默认为本地时区。例如:timezone="America/New_York" 4. 支持从kafka自定义偏移量中消费数据。 + # 已支持 * 源表:kafka 0.9,1.x版本,serverSocket * 维表:mysql,SQlServer,oracle,hbase,mongo,redis,cassandra @@ -26,16 +31,6 @@ * kafka avro格式 * topN -## BUG修复: - - 1. oracle维表获取索引问题。 - 2. Perjob模式下UDF类加载异常问题。 - 3. queue 参数设置无效的问题。 - -* 单机模式:对应Flink集群的单机模式 -* standalone模式:对应Flink集群的分布式模式 -* yarn模式:对应Flink集群的yarn模式 - ### 1.2 执行环境 * Java: JDK8及以上 @@ -62,6 +57,10 @@ sh submit.sh -sql D:\sideSql.txt -name xctest -remoteSqlPluginPath /opt/dtstack #### 1.4.2 命令行参数选项 +* 单机模式:对应Flink集群的单机模式 +* standalone模式:对应Flink集群的分布式模式 +* yarn模式:对应Flink集群的yarn模式 + * **mode** * 描述:执行模式,也就是flink集群的工作模式 * local: 本地模式 From dc391e03eec5ce4e601b42049e997da5a7665242 Mon Sep 17 00:00:00 2001 From: chaiyq Date: Thu, 11 Jul 2019 09:00:07 +0800 Subject: [PATCH 250/250] Handing No matching record in redis table leads to no output on left-join mode with RedisAsyncReqRow --- .../sql/side/redis/RedisAsyncReqRow.java | 48 ++++++++++--------- 1 file changed, 26 insertions(+), 22 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 251107287..412fca5cc 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 @@ -156,30 +156,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 keyData) {