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 01/65] 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 02/65] 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 03/65] 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 04/65] 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 05/65] =?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 06/65] =?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 07/65] 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 08/65] 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 09/65] =?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 10/65] 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 11/65] 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 12/65] 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 13/65] =?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 14/65] =?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 15/65] =?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 16/65] =?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 17/65] 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 18/65] 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 19/65] 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 20/65] 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 21/65] 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 22/65] 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 23/65] 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 24/65] =?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 25/65] 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 b112db7337765ac0ecafec8c158c692c1adabd76 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 20 Sep 2018 19:49:46 +0800 Subject: [PATCH 26/65] 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 01a4ee3ad1123be7540293b7d98fd13281208777 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Fri, 28 Sep 2018 11:31:17 +0800 Subject: [PATCH 27/65] 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 28/65] 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 29/65] 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 30/65] 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 31/65] 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 32/65] 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 33/65] 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 34/65] 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 35/65] 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 f7acdad9642d6eaf0b5e711f16e5d7270fa003ad Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sun, 30 Sep 2018 10:36:15 +0800 Subject: [PATCH 36/65] 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 37/65] 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 38f83a8c84a7b4109e5108e078e0e8568be42283 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sun, 30 Sep 2018 11:33:16 +0800 Subject: [PATCH 38/65] 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 39/65] 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 40/65] 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 41/65] 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 356f6051166fddceee67e71012b5d3e98184029d Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Wed, 10 Oct 2018 16:04:06 +0800 Subject: [PATCH 42/65] 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 43/65] 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 b5be9c0e277569b33698d1545137409cf0d65bd7 Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Thu, 11 Oct 2018 09:17:37 +0800 Subject: [PATCH 44/65] 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 a64da8637c11bf964116968e5bc8a19912308e9b Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Thu, 11 Oct 2018 17:52:46 +0800 Subject: [PATCH 45/65] 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 74f0cf62ee2ce7a5be3547b30f6056d4334e6e36 Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 18 Oct 2018 13:55:42 +0800 Subject: [PATCH 46/65] 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 b4f424c97eb2a82b16455186a4f909da3a70531e Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 18 Oct 2018 21:43:17 +0800 Subject: [PATCH 47/65] 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 48/65] 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 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 49/65] 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 50/65] 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 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 51/65] 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 52/65] 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 53/65] 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 92b403734815b8bdb75267864ff46b93cf462c3d Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Mon, 22 Oct 2018 17:39:04 +0800 Subject: [PATCH 54/65] 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 55/65] 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 56/65] 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 57/65] 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 58/65] 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 59/65] 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 60/65] 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 480179345d8e1d9bc3657fb64c637e05af07d27f Mon Sep 17 00:00:00 2001 From: whiletrue <670694243@qq.com> Date: Thu, 1 Nov 2018 14:09:41 +0800 Subject: [PATCH 61/65] 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 7e904b512a349d274471eb5e8992ee93961bba39 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Fri, 2 Nov 2018 16:12:04 +0800 Subject: [PATCH 62/65] 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 63/65] 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 aea00f721b9acf656bc7fe84565330f30e713604 Mon Sep 17 00:00:00 2001 From: zoudaokoulife Date: Sat, 3 Nov 2018 11:31:23 +0800 Subject: [PATCH 64/65] 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 94ea2a1bea0b2c2e6ad7ce9abe94bff13853b56c Mon Sep 17 00:00:00 2001 From: "sishu@dtstack.com" Date: Tue, 6 Nov 2018 20:23:40 +0800 Subject: [PATCH 65/65] 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){