├── .gitignore
├── LICENSE
├── README.md
├── doc
├── picture
│ └── gzh.png
└── shell
│ ├── start_job.sh
│ ├── start_pre_job.sh
│ └── start_session.sh
├── git.sh
├── pom.xml
└── src
├── main
├── flink
│ └── org
│ │ └── apache
│ │ └── flink
│ │ ├── connector
│ │ ├── hbase
│ │ │ └── util
│ │ │ │ └── HBaseSerde.java
│ │ └── jdbc
│ │ │ ├── catalog
│ │ │ ├── MyMySqlCatalog.java
│ │ │ └── MysqlCatalogUtils.java
│ │ │ ├── internal
│ │ │ └── executor
│ │ │ │ └── SimpleBatchStatementExecutor.java
│ │ │ └── table
│ │ │ └── JdbcDynamicTableFactory.java
│ │ ├── streaming
│ │ ├── api
│ │ │ └── operators
│ │ │ │ ├── KeyedProcessOperator.java
│ │ │ │ └── collect
│ │ │ │ └── CollectStreamSink.java
│ │ └── connectors
│ │ │ ├── elasticsearch
│ │ │ └── ElasticsearchUpsertTableSinkBase.java
│ │ │ └── kafka
│ │ │ └── table
│ │ │ ├── KafkaConnectorOptions.java
│ │ │ ├── KafkaDynamicSource.java
│ │ │ └── KafkaDynamicTableFactory.java
│ │ └── table
│ │ ├── api
│ │ ├── StatementSet.java
│ │ ├── TableEnvironment.java
│ │ ├── config
│ │ │ └── ExecutionConfigOptions.java
│ │ └── internal
│ │ │ ├── StatementSetImpl.java
│ │ │ ├── TableEnvironmentImpl.java
│ │ │ └── TableEnvironmentInternal.java
│ │ ├── planner
│ │ └── plan
│ │ │ └── nodes
│ │ │ └── exec
│ │ │ └── common
│ │ │ ├── CommonExecCalc.java
│ │ │ └── CommonExecLookupJoin.java
│ │ └── runtime
│ │ └── operators
│ │ └── rank
│ │ └── AppendOnlyTopNFunction.java
├── java
│ ├── Median.java
│ └── com
│ │ └── rookie
│ │ └── submit
│ │ ├── cust
│ │ ├── base
│ │ │ └── RowDataConverterBase.java
│ │ ├── connector
│ │ │ ├── bigjdbc
│ │ │ │ ├── BigJdbcSource.java
│ │ │ │ ├── enumerator
│ │ │ │ │ ├── BigJdbcSourceEnumerator.java
│ │ │ │ │ ├── BigJdbcSourceEnumeratorSerializer.java
│ │ │ │ │ └── BigJdbcSourceEnumeratorState.java
│ │ │ │ ├── reader
│ │ │ │ │ ├── BigJdbcSourceEmitter.java
│ │ │ │ │ ├── BigJdbcSourceReader.java
│ │ │ │ │ ├── BigJdbcSourceSplitReader.java
│ │ │ │ │ └── fetch
│ │ │ │ │ │ └── BigJdbcSourceFetcherManager.java
│ │ │ │ └── split
│ │ │ │ │ ├── BigJdbcSplit.java
│ │ │ │ │ ├── BigJdbcSplitSerializer.java
│ │ │ │ │ └── BigJdbcSplitState.java
│ │ │ ├── hbase
│ │ │ │ ├── HbaseDynamicTableFactory.java
│ │ │ │ ├── HbaseDynamicTableSource.java
│ │ │ │ ├── HbaseOption.java
│ │ │ │ └── HbaseRowDataLookUpFunction.java
│ │ │ ├── http
│ │ │ │ ├── HttpClientUtil.java
│ │ │ │ ├── HttpDynamicTableFactory.java
│ │ │ │ ├── HttpDynamicTableSource.java
│ │ │ │ └── HttpSource.java
│ │ │ ├── mysql
│ │ │ │ ├── MysqlDynamicTableFactory.java
│ │ │ │ ├── MysqlDynamicTableSource.java
│ │ │ │ ├── MysqlOption.java
│ │ │ │ ├── MysqlRowDataLookUpFunction.java
│ │ │ │ └── MysqlSource.java
│ │ │ ├── redis
│ │ │ │ ├── RedisDynamicTableFactory.java
│ │ │ │ ├── RedisDynamicTableSource.java
│ │ │ │ ├── RedisOption.java
│ │ │ │ └── RedisRowDataLookUpFunction.java
│ │ │ ├── socket
│ │ │ │ ├── SocketDynamicTableFactory.java
│ │ │ │ ├── SocketDynamicTableSink.java
│ │ │ │ ├── SocketDynamicTableSource.java
│ │ │ │ ├── SocketSinkFunction.java
│ │ │ │ └── SocketSourceFunction.java
│ │ │ └── starrocks
│ │ │ │ ├── StarrocksDynamicTableFactory.java
│ │ │ │ ├── StarrocksDynamicTableSource.java
│ │ │ │ ├── StarrocksOption.java
│ │ │ │ ├── StarrocksRowDataLookUpFunction.java
│ │ │ │ └── StarrocksSource.java
│ │ └── format
│ │ │ └── changelog
│ │ │ └── csv
│ │ │ ├── ChangelogCsvDeserializer.java
│ │ │ ├── ChangelogCsvFormat.java
│ │ │ └── ChangelogCsvFormatFactory.java
│ │ ├── udaf
│ │ ├── BloomFilter.java
│ │ ├── CountAcc.java
│ │ ├── JedisRedisUv.java
│ │ ├── RedisUv.java
│ │ ├── RedisUv2.java
│ │ └── math
│ │ │ ├── Median.java
│ │ │ └── NumberAcc.java
│ │ ├── udf
│ │ ├── DateAdd.java
│ │ ├── ParseDctJson.java
│ │ └── ParseJson.java
│ │ └── udtf
│ │ ├── JoinMysql.java
│ │ └── UdtfTimer.java
├── resources
│ ├── META-INF
│ │ └── services
│ │ │ ├── org.apache.flink.table.factories.Factory
│ │ │ └── org.apache.flink.table.factories.TableFactory
│ ├── demoJobPropFile.properties
│ ├── hive-site.xml
│ ├── log4j.properties
│ ├── sql
│ │ ├── dev
│ │ │ ├── create_table_datagen.sql
│ │ │ ├── datagen_to_hive.sql
│ │ │ ├── datagen_to_hive_2.sql
│ │ │ ├── datagen_to_kafka.sql
│ │ │ ├── flink_cdc_test.sql
│ │ │ ├── hive_to_hive.sql
│ │ │ ├── hive_to_print.sql
│ │ │ ├── insert.sql
│ │ │ ├── kafka_to_hdfs.sql
│ │ │ ├── kafka_to_kafka.sql
│ │ │ ├── kafka_to_kafka_last_1m_tps.sql
│ │ │ ├── kafka_to_mysql_partial_column_update.sql
│ │ │ ├── kafka_to_print.sql
│ │ │ ├── kafka_to_print_arr.sql
│ │ │ ├── kafka_to_print_fluctuation.sql
│ │ │ └── kafka_window_agg.sql
│ │ ├── hudi
│ │ │ ├── cdc_mysql_to_hudi.sql
│ │ │ └── hudi_demo.sql
│ │ ├── iceberg
│ │ │ ├── README.md
│ │ │ ├── kafka_to_iceberg_demo.sql
│ │ │ ├── kafka_to_iceberg_upsert.sql
│ │ │ └── sql_demo.md
│ │ ├── operator
│ │ │ ├── count_distinct.sql
│ │ │ ├── deduplication.sql
│ │ │ ├── history_pv_uv
│ │ │ │ ├── pu_uv_1.sql
│ │ │ │ ├── pu_uv_2.sql
│ │ │ │ ├── pu_uv_3.sql
│ │ │ │ ├── pu_uv_4.sql
│ │ │ │ └── pu_uv_5.sql
│ │ │ ├── interval_join_out_of_order.sql
│ │ │ ├── kafka_join_agg.sql
│ │ │ ├── kafka_to_print_udtf_timer.sql
│ │ │ ├── kafka_to_window_test.sql
│ │ │ ├── multe_insert.sql
│ │ │ ├── topn.sql
│ │ │ ├── tps
│ │ │ │ ├── kafka_lookup_join_hbase_demo_tps.sql
│ │ │ │ ├── kafka_lookup_join_mysql_demo_tps.sql
│ │ │ │ ├── kafka_lookup_join_redis_tps.sql
│ │ │ │ ├── kafka_lookup_join_starrocks_tps.sql
│ │ │ │ ├── kafka_to_hbase.sql
│ │ │ │ └── kafka_to_mysql.sql
│ │ │ ├── window
│ │ │ │ ├── cumulate_offset.sql
│ │ │ │ ├── cumulate_pv_uv.sql
│ │ │ │ ├── kafka_window_agg.sql
│ │ │ │ ├── kafka_window_demo.sql
│ │ │ │ ├── kafka_window_join_agg.sql
│ │ │ │ └── pv_uv.sql
│ │ │ └── window_demo.sql
│ │ ├── other
│ │ │ ├── flink_cdc_tbls_to_mysql.sql
│ │ │ ├── kafka_to_mysql_group_by.sql
│ │ │ └── ods_sap_afpo.sql
│ │ ├── release
│ │ │ ├── cep
│ │ │ │ ├── cep_count_2000.sql
│ │ │ │ └── cep_event_1.sql
│ │ │ ├── connector
│ │ │ │ ├── cust
│ │ │ │ │ ├── cust_http_source_demo.sql
│ │ │ │ │ ├── cust_mysql_source_demo.sql
│ │ │ │ │ ├── cust_socket_source_demo.sql
│ │ │ │ │ └── jdbc_mysql_source_demo.sql
│ │ │ │ ├── hbase
│ │ │ │ │ └── kafka_to_hbase.sql
│ │ │ │ ├── hive
│ │ │ │ │ ├── hive_to_kafka.sql
│ │ │ │ │ ├── kafka_to_hive.sql
│ │ │ │ │ └── kafka_to_hive_ms.sql
│ │ │ │ ├── iceberg
│ │ │ │ │ └── kafka_to_iceberg.sql
│ │ │ │ ├── jdbc
│ │ │ │ │ └── mysql_to_kafka.sql
│ │ │ │ ├── kafka
│ │ │ │ │ ├── kafka_source_parallelism_demo.sql
│ │ │ │ │ ├── kafka_to_hbase.sql
│ │ │ │ │ ├── kafka_to_kafka.sql
│ │ │ │ │ ├── kafka_to_print.sql
│ │ │ │ │ └── kafka_upsert_demo.sql
│ │ │ │ ├── kafka_to_socket.sql
│ │ │ │ ├── kudu
│ │ │ │ │ └── kafka_to_kudu.sql
│ │ │ │ ├── mysql
│ │ │ │ │ ├── batch_write_mysql_test.sql
│ │ │ │ │ ├── cdc_mysql_to_kafka_demo.sql
│ │ │ │ │ ├── cdc_mysql_to_print.sql
│ │ │ │ │ ├── kafka_to_mysql_demo.sql
│ │ │ │ │ └── mysql_count_test.sql
│ │ │ │ └── socket_to_socket.sql
│ │ │ ├── format
│ │ │ │ ├── complex_json.sql
│ │ │ │ └── kafka_special_json_parse.sql
│ │ │ ├── join
│ │ │ │ ├── interval_join_demo.sql
│ │ │ │ ├── join_demo.sql
│ │ │ │ ├── kafka_batch_join_mysql_demo.sql
│ │ │ │ ├── kafka_join_mysql_demo.sql
│ │ │ │ ├── lookup_join
│ │ │ │ │ ├── cust_hbase_lookup_source_demo.sql
│ │ │ │ │ ├── cust_mysql_lookup_source_demo.sql
│ │ │ │ │ ├── kafka_lookup_join_hbase_demo.sql
│ │ │ │ │ ├── kafka_lookup_join_hbase_no_rowkey_demo.sql
│ │ │ │ │ ├── kafka_lookup_join_multi_mysql.sql
│ │ │ │ │ ├── kafka_lookup_join_mysql_demo.sql
│ │ │ │ │ ├── kafka_lookup_join_redis.sql
│ │ │ │ │ ├── kafka_lookup_multi_mysql_table.sql
│ │ │ │ │ └── kafka_tvf_lookup_join.sql
│ │ │ │ └── stream_table_join.sql
│ │ │ └── udf
│ │ │ │ ├── join_hbase_no_rowkey_demo.sql
│ │ │ │ ├── parse_complex_json.sql
│ │ │ │ └── parse_complex_json_1.sql
│ │ └── starrocks
│ │ │ ├── datagen_to_starrocks.sql
│ │ │ ├── kafka_join_starrocks.sql
│ │ │ ├── kafka_to_starrocks.sql
│ │ │ ├── kafka_to_starrocks_tps_test.sql
│ │ │ ├── mysql_to_starrocks.sql
│ │ │ ├── starrocks_agg_demo.sql
│ │ │ └── starrocks_to_print.sql
│ └── sqlSubmit.properties
└── scala
│ └── com
│ └── rookie
│ └── submit
│ ├── common
│ ├── Common.scala
│ └── Constant.java
│ ├── connector
│ └── kafka
│ │ ├── KafkaUpsertTableSink.java
│ │ ├── KafkaUpsertTableSinkFactory.java
│ │ ├── KafkaUpsertTableSinkFactoryBase.java
│ │ └── UpsertKafkaValidator.java
│ ├── main
│ ├── SqlSubmit.scala
│ └── SqlSubmitHive.scala
│ ├── udf
│ ├── Decode.java
│ ├── JoinHbaseNonRowkeyCache.scala
│ ├── JoinHbaseNonRowkeyNoCache.scala
│ ├── RegisterUdf.scala
│ └── SplitFunction.scala
│ └── util
│ ├── CatalogUtil.scala
│ ├── DateTimeUtil.java
│ ├── PropertiesUtil.java
│ ├── SqlFileUtil.java
│ └── TableConfUtil.scala
└── test
└── scala
└── com
└── rookie
└── submit
├── FlinkTestDemo.scala
└── udf
└── JoinHbaseNonRowkeyNoCacheTest.java
/.gitignore:
--------------------------------------------------------------------------------
1 | sqlSubmit.iml
2 | target/
3 | dependency-reduced-pom.xml
4 | .DS_Store
5 |
--------------------------------------------------------------------------------
/doc/picture/gzh.png:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/springMoon/sqlSubmit/4e7e76dfe3df0af2b0507d6849e3f00ab779c04f/doc/picture/gzh.png
--------------------------------------------------------------------------------
/doc/shell/start_job.sh:
--------------------------------------------------------------------------------
1 | #!/bin/bash
2 | basedir=`cd $(dirname $0); pwd -P`
3 | echo "$basedir"
4 | cd $basedir
5 | export FLINK_HOME=/opt/flink-1.15.1
6 | export PATH=$PATH:$FLINK_HOME/bin
7 | export HADOOP_CLASSPATH=`hadoop classpath`
8 |
9 | current_path=$(cd "$(dirname $0)";pwd)
10 | logFile=$current_path/start.log
11 | write_log(){
12 | date_str=`date -d now +"%F %T"`
13 | if [ -n "$1" ];then
14 | message="$1"
15 | else
16 | message="no input"
17 | fi
18 | echo "[ $date_str ] $@" | tee -a $logFile
19 | }
20 |
21 | #set -x
22 | write_log "start : $current_path"
23 |
24 | write_log "start parameter : $@"
25 | ## start flink session
26 | if [ -z "$2" ];then
27 | echo "please input session name"
28 | exit -1
29 | fi
30 | session_name="$2"
31 |
32 | session_status="`yarn application -list | grep "$session_name" | awk -F " " '{ print $1}' | awk 'NR==1'`"
33 | if [ -z "$session_status" ];then
34 | write_log "$session_name is not running, start it"
35 | start_session.sh $session_name
36 | fi
37 |
38 | ## start job
39 | flink run -yd -yid $session_status -c com.rookie.submit.main.SqlSubmit original-sqlSubmit-0.1.jar $@
40 | #write_log "start session $session_name"
41 |
42 |
--------------------------------------------------------------------------------
/doc/shell/start_pre_job.sh:
--------------------------------------------------------------------------------
1 | #!/bin/bash
2 |
3 | basedir=`cd $(dirname $0); pwd -P`
4 | cd $basedir
5 | export FLINK_HOME=/opt/flink-1.15.1
6 | export PATH=$PATH:$FLINK_HOME/bin
7 | export HADOOP_CLASSPATH=`hadoop classpath`
8 |
9 | current_path=$(cd "$(dirname $0)";pwd)
10 | logFile=$current_path/start.log
11 | write_log(){
12 | date_str=`date -d now +"%F %T"`
13 | if [ -n "$1" ];then
14 | message="$1"
15 | else
16 | message="no input"
17 | fi
18 | echo "[ $date_str ] $@" | tee -a $logFile
19 | }
20 |
21 | #set -x
22 | write_log "start : $current_path"
23 |
24 | write_log "start parameter : $@"
25 |
26 | ## start job
27 | flink run -m yarn-cluster -ynm $2 -c com.rookie.submit.main.SqlSubmit original-sqlSubmit-0.1.jar $@
28 |
29 |
--------------------------------------------------------------------------------
/doc/shell/start_session.sh:
--------------------------------------------------------------------------------
1 | #!/bin/bash
2 |
3 | basedir=`cd $(dirname $0); pwd -P`
4 | cd $basedir
5 | export FLINK_HOME=/opt/flink-1.15.1
6 | export PATH=$PATH:$FLINK_HOME/bin
7 | export HADOOP_CLASSPATH=`hadoop classpath`
8 |
9 | current_path=$(cd "$(dirname $0)";pwd)
10 | logFile=$current_path/start.log
11 |
12 | write_log(){
13 | date_str=`date -d now +"%F %T"`
14 |
15 | if [ -n "$1" ];then
16 | message="$1"
17 | fi
18 | echo "[ $date_str ] $message" | tee -a $logFile
19 | }
20 | echo "$current_path" | tee -a $logFile
21 |
22 | #write_log "current : $current_path"
23 |
24 | ## start flink session
25 | if [ -z "$1" ];then
26 | echo "please input session name"
27 | exit -1
28 | fi
29 | session_name="$1"
30 |
31 | session_status="`yarn application -list | grep "$session_name" | awk -F " " '{ print $1}' | awk 'NR==1'`"
32 | if [ -n "$session_status" ];then
33 | write_log "$session_name is alread running"
34 | read -r -t 30 -p "Are You Sure? [Y/n] " input
35 | case $input in
36 | [yY][eE][sS]|[yY])
37 | echo "Yes"
38 | write_log "stop session $session_status"
39 | yarn application -kill $session_status
40 | ;;
41 |
42 | [nN][oO]|[nN])
43 | echo "No"
44 | exit 1
45 | ;;
46 | esac
47 | fi
48 |
49 | ## start session
50 | yarn-session.sh -d -nm "$session_name" -s 4
51 | write_log "start session $session_name"
52 |
53 |
--------------------------------------------------------------------------------
/git.sh:
--------------------------------------------------------------------------------
1 | #!/bin/bash
2 |
3 | git status
4 |
5 | message="update today"
6 | if [ -n "$1" ]; then
7 | message=$1
8 | fi
9 |
10 | git pull
11 | git add *
12 | git commit -m "$message ` date -d now +"%F %T"`"
13 | git push
14 |
--------------------------------------------------------------------------------
/src/main/flink/org/apache/flink/connector/jdbc/internal/executor/SimpleBatchStatementExecutor.java:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/springMoon/sqlSubmit/4e7e76dfe3df0af2b0507d6849e3f00ab779c04f/src/main/flink/org/apache/flink/connector/jdbc/internal/executor/SimpleBatchStatementExecutor.java
--------------------------------------------------------------------------------
/src/main/flink/org/apache/flink/table/api/StatementSet.java:
--------------------------------------------------------------------------------
1 | ///*
2 | // * Licensed to the Apache Software Foundation (ASF) under one
3 | // * or more contributor license agreements. See the NOTICE file
4 | // * distributed with this work for additional information
5 | // * regarding copyright ownership. The ASF licenses this file
6 | // * to you under the Apache License, Version 2.0 (the
7 | // * "License"); you may not use this file except in compliance
8 | // * with the License. You may obtain a copy of the License at
9 | // *
10 | // * http://www.apache.org/licenses/LICENSE-2.0
11 | // *
12 | // * Unless required by applicable law or agreed to in writing, software
13 | // * distributed under the License is distributed on an "AS IS" BASIS,
14 | // * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | // * See the License for the specific language governing permissions and
16 | // * limitations under the License.
17 | // */
18 | //
19 | //package org.apache.flink.table.api;
20 | //
21 | //import org.apache.flink.annotation.PublicEvolving;
22 | //
23 | ///**
24 | // * A {@link StatementSet} accepts DML statements or {@link Table}s,
25 | // * the planner can optimize all added statements and Tables together
26 | // * and then submit as one job.
27 | // *
28 | // *
The added statements and Tables will be cleared
29 | // * when calling the `execute` method.
30 | // */
31 | //@PublicEvolving
32 | //public interface StatementSet {
33 | //
34 | // /**
35 | // * add insert statement to the set.
36 | // */
37 | // StatementSet addInsertSql(String statement);
38 | //
39 | // /**
40 | // * add Table with the given sink table name to the set.
41 | // */
42 | // StatementSet addInsert(String targetPath, Table table);
43 | //
44 | // /**
45 | // * add {@link Table} with the given sink table name to the set.
46 | // */
47 | // StatementSet addInsert(String targetPath, Table table, boolean overwrite);
48 | //
49 | // /**
50 | // * returns the AST and the execution plan to compute the result of the
51 | // * all statements and Tables.
52 | // *
53 | // * @param extraDetails The extra explain details which the explain result should include,
54 | // * e.g. estimated cost, changelog mode for streaming
55 | // * @return AST and the execution plan.
56 | // */
57 | // String explain(ExplainDetail... extraDetails);
58 | //
59 | // /**
60 | // * execute all statements and Tables as a batch.
61 | // *
62 | // *
The added statements and Tables will be cleared when executing this method.
63 | // */
64 | // TableResult execute();
65 | //
66 | // /**
67 | // * execute all statements and Tables as a batch.
68 | // *
69 | // *
The added statements and Tables will be cleared when executing this method.
70 | // */
71 | // TableResult execute(String jobName);
72 | //}
73 |
--------------------------------------------------------------------------------
/src/main/java/Median.java:
--------------------------------------------------------------------------------
1 | import com.rookie.submit.udaf.math.NumberAcc;
2 | import org.apache.flink.table.functions.AggregateFunction;
3 |
4 | import java.util.List;
5 | import java.util.stream.Collectors;
6 |
7 | /**
8 | * agg function: 计算中位数
9 | */
10 | public class Median extends AggregateFunction {
11 | // 获取 acc 的值
12 | @Override
13 | public Double getValue(NumberAcc acc) {
14 | // sort list
15 | List list = acc.list.stream().sorted().collect(Collectors.toList());
16 | // if list is empty, return null
17 | if (list.size() == 0) {
18 | return null;
19 | } else if (list.size() == 1) {
20 | // if list have one element, return it
21 | return list.get(0);
22 | }
23 | double val;
24 | int size = list.size();
25 | int half = size / 2;
26 | if (size % 2 == 0) {
27 | //even, use (size/2 - 1 + size/2) / 2
28 | val = (list.get(half - 1) + list.get(half)) / 2;
29 | } else {
30 | // odd, use size/2
31 | val = list.get(half);
32 | }
33 | return val;
34 | }
35 | // 累加元素
36 | public void accumulate(NumberAcc acc, Double d) {
37 | acc.list.add(d);
38 | }
39 | // 创建累加器
40 | @Override
41 | public NumberAcc createAccumulator() {
42 | return new NumberAcc();
43 | }
44 |
45 | // 窗口聚合
46 | public void merge(NumberAcc acc, Iterable it) {
47 | for (NumberAcc a : it) {
48 | acc.list.addAll(a.list);
49 | }
50 | }
51 | }
52 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/connector/bigjdbc/enumerator/BigJdbcSourceEnumeratorSerializer.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.cust.connector.bigjdbc.enumerator;
2 |
3 | import com.rookie.submit.cust.connector.bigjdbc.split.BigJdbcSplit;
4 | import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState;
5 | import org.apache.flink.core.io.SimpleVersionedSerializer;
6 | import org.apache.kafka.common.TopicPartition;
7 |
8 | import java.io.*;
9 | import java.util.HashSet;
10 | import java.util.Set;
11 |
12 | public class BigJdbcSourceEnumeratorSerializer implements SimpleVersionedSerializer
13 | {
14 |
15 | @Override
16 | public int getVersion() {
17 | return 0;
18 | }
19 |
20 | @Override
21 | public byte[] serialize(BigJdbcSourceEnumeratorState state) throws IOException {
22 | Set splitSet = state.assignedPartitions();
23 |
24 | try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
25 | DataOutputStream out = new DataOutputStream(baos)) {
26 |
27 | out.writeInt(splitSet.size());
28 | for (BigJdbcSplit split : splitSet) {
29 | out.writeLong(split.getSplitStart());
30 | out.writeLong(split.getSplitEnd());
31 | }
32 | out.flush();
33 | return baos.toByteArray();
34 | }
35 | }
36 |
37 | @Override
38 | public BigJdbcSourceEnumeratorState deserialize(int version, byte[] serialized) throws IOException {
39 | Set splitSet;
40 | try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
41 | DataInputStream in = new DataInputStream(bais)) {
42 |
43 | final int numPartitions = in.readInt();
44 | splitSet = new HashSet<>(numPartitions);
45 | for (int i = 0; i < numPartitions; i++) {
46 | final long splitStart = in.readLong();
47 | final long splitEnd = in.readLong();
48 | splitSet.add(new BigJdbcSplit(splitStart, splitEnd));
49 | }
50 | if (in.available() > 0) {
51 | throw new IOException("Unexpected trailing bytes in serialized topic partitions");
52 | }
53 | }
54 | return new BigJdbcSourceEnumeratorState(splitSet);
55 |
56 | }
57 | }
58 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/connector/bigjdbc/enumerator/BigJdbcSourceEnumeratorState.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.cust.connector.bigjdbc.enumerator;
2 |
3 | import com.rookie.submit.cust.connector.bigjdbc.split.BigJdbcSplit;
4 |
5 | import java.util.Set;
6 |
7 | public class BigJdbcSourceEnumeratorState {
8 |
9 | private final Set assignedPartitions;
10 |
11 | public BigJdbcSourceEnumeratorState(Set assignedPartitions) {
12 | this.assignedPartitions = assignedPartitions;
13 | }
14 |
15 | public Set assignedPartitions() {
16 | return assignedPartitions;
17 | }
18 | }
19 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/connector/bigjdbc/reader/BigJdbcSourceEmitter.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.cust.connector.bigjdbc.reader;
2 |
3 | import com.google.gson.JsonObject;
4 | import com.rookie.submit.cust.connector.bigjdbc.split.BigJdbcSplitState;
5 | import org.apache.flink.api.connector.source.SourceOutput;
6 | import org.apache.flink.connector.base.source.reader.RecordEmitter;
7 |
8 | public class BigJdbcSourceEmitter implements RecordEmitter {
9 |
10 | @Override
11 | public void emitRecord(JsonObject element, SourceOutput output, BigJdbcSplitState splitState) throws Exception {
12 |
13 | output.collect(element);
14 |
15 | }
16 | }
17 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/connector/bigjdbc/reader/BigJdbcSourceReader.java:
--------------------------------------------------------------------------------
1 | //package com.rookie.submit.cust.connector.bigjdbc.reader;
2 | //
3 | //import com.rookie.submit.cust.connector.bigjdbc.split.BigJdbcSplit;
4 | //import com.rookie.submit.cust.connector.bigjdbc.split.BigJdbcSplitState;
5 | //import org.apache.flink.api.connector.source.SourceReaderContext;
6 | //import org.apache.flink.configuration.Configuration;
7 | //import org.apache.flink.connector.base.source.reader.RecordEmitter;
8 | //import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
9 | //import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase;
10 | //import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
11 | //import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
12 | //import org.apache.kafka.clients.consumer.ConsumerRecord;
13 | //
14 | //import java.util.Map;
15 | //import java.util.concurrent.ConcurrentHashMap;
16 | //import java.util.concurrent.ConcurrentMap;
17 | //import java.util.function.Supplier;
18 | //
19 | //public class BigJdbcSourceReader extends SingleThreadMultiplexSourceReaderBase {
20 | //
21 | //
22 | // private final ConcurrentMap offsetsOfFinishedSplits;
23 | //
24 | // public BigJdbcSourceReader(FutureCompletingBlockingQueue>>
25 | // elementsQueue,
26 | // ){
27 | // super(elementsQueue, kafkaSourceFetcherManager, recordEmitter, config, context);
28 | // this.offsetsOfFinishedSplits = new ConcurrentHashMap();
29 | // }
30 | //
31 | // @Override
32 | // protected void onSplitFinished(Map finishedSplitIds) {
33 | // finishedSplitIds.forEach(
34 | // (ignored, splitState) -> {
35 | // offsetsOfFinishedSplits.put(
36 | // splitState.getSplitStart(), splitState.getSplitEnd());
37 | // }
38 | // );
39 | // }
40 | //
41 | // @Override
42 | // protected BigJdbcSplitState initializedState(BigJdbcSplit split) {
43 | // return new BigJdbcSplitState(split);
44 | // }
45 | //
46 | // @Override
47 | // protected BigJdbcSplit toSplitType(String splitId, BigJdbcSplitState splitState) {
48 | // return splitState.toBigJdbcSplit();
49 | // }
50 | //}
51 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/connector/bigjdbc/reader/fetch/BigJdbcSourceFetcherManager.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.cust.connector.bigjdbc.reader.fetch;
2 |
3 | import com.google.gson.JsonElement;
4 | import com.rookie.submit.cust.connector.bigjdbc.split.BigJdbcSplit;
5 | import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
6 | import org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager;
7 | import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
8 | import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
9 |
10 | import java.util.function.Supplier;
11 |
12 | public class BigJdbcSourceFetcherManager extends SingleThreadFetcherManager {
13 | public BigJdbcSourceFetcherManager(FutureCompletingBlockingQueue> elementsQueue,
14 | Supplier> splitReaderSupplier) {
16 | super(elementsQueue, splitReaderSupplier);
17 | }
18 | }
19 |
20 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/connector/bigjdbc/split/BigJdbcSplit.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.cust.connector.bigjdbc.split;
2 |
3 | import org.apache.flink.api.connector.source.SourceSplit;
4 |
5 | import java.io.Serializable;
6 |
7 | public class BigJdbcSplit implements SourceSplit, Serializable {
8 |
9 |
10 | // split start, include
11 | private final Long splitStart;
12 | // split end,
13 | private final Long splitEnd;
14 |
15 | public BigJdbcSplit(Long splitStart, Long splitEnd) {
16 | this.splitStart = splitStart;
17 | this.splitEnd = splitEnd;
18 | }
19 |
20 | public Long getSplitStart() {
21 | return splitStart;
22 | }
23 |
24 | public Long getSplitEnd() {
25 | return splitEnd;
26 | }
27 |
28 | @Override
29 | public String splitId() {
30 | return splitStart + "_" + splitEnd;
31 | }
32 | }
33 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/connector/bigjdbc/split/BigJdbcSplitSerializer.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.cust.connector.bigjdbc.split;
2 |
3 | import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
4 | import org.apache.flink.core.io.SimpleVersionedSerializer;
5 | import org.apache.kafka.common.TopicPartition;
6 |
7 | import java.io.*;
8 |
9 | public class BigJdbcSplitSerializer implements SimpleVersionedSerializer {
10 | @Override
11 | public int getVersion() {
12 | return 0;
13 | }
14 |
15 | @Override
16 | public byte[] serialize(BigJdbcSplit split) throws IOException {
17 | try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
18 | DataOutputStream out = new DataOutputStream(baos)) {
19 | out.writeLong(split.getSplitStart());
20 | out.writeLong(split.getSplitEnd());
21 | out.flush();
22 | return baos.toByteArray();
23 | }
24 | }
25 |
26 | @Override
27 | public BigJdbcSplit deserialize(int version, byte[] serialized) throws IOException {
28 | try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
29 | DataInputStream in = new DataInputStream(bais)) {
30 | long splitStart = in.readLong();
31 | long splitEnd = in.readLong();
32 | return new BigJdbcSplit(splitStart, splitEnd);
33 | }
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/connector/bigjdbc/split/BigJdbcSplitState.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.cust.connector.bigjdbc.split;
2 |
3 | import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit;
4 |
5 | public class BigJdbcSplitState extends BigJdbcSplit{
6 | public BigJdbcSplitState(BigJdbcSplit split) {
7 | super(split.getSplitStart(), split.getSplitEnd());
8 | }
9 |
10 | public BigJdbcSplit toBigJdbcSplit() {
11 | return new BigJdbcSplit(getSplitStart(), getSplitEnd());
12 | }
13 | }
14 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/connector/hbase/HbaseDynamicTableSource.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.cust.connector.hbase;
2 |
3 | import org.apache.flink.connector.hbase.util.HBaseTableSchema;
4 | import org.apache.flink.table.connector.source.DynamicTableSource;
5 | import org.apache.flink.table.connector.source.LookupTableSource;
6 | import org.apache.flink.table.connector.source.TableFunctionProvider;
7 | import org.apache.flink.table.types.DataType;
8 | import org.slf4j.Logger;
9 | import org.slf4j.LoggerFactory;
10 |
11 | import java.io.UnsupportedEncodingException;
12 |
13 | public class HbaseDynamicTableSource implements LookupTableSource {
14 |
15 | private final Logger LOG = LoggerFactory.getLogger(HbaseDynamicTableSource.class);
16 |
17 | private final DataType producedDataType;
18 | private final HbaseOption options;
19 | private final HBaseTableSchema hbaseSchema;
20 |
21 | public HbaseDynamicTableSource(
22 |
23 | DataType producedDataType,
24 | HbaseOption options,
25 | HBaseTableSchema hbaseSchema) {
26 |
27 | this.producedDataType = producedDataType;
28 | this.options = options;
29 | this.hbaseSchema = hbaseSchema;
30 | }
31 |
32 | @Override
33 | public DynamicTableSource copy() {
34 | return new HbaseDynamicTableSource(producedDataType, options, hbaseSchema);
35 | }
36 |
37 | @Override
38 | public String asSummaryString() {
39 | return "Hbase Table Source, support Lookup function";
40 | }
41 |
42 | @Override
43 | public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) {
44 |
45 | HbaseRowDataLookUpFunction lookUpFunction = null;
46 | try {
47 | lookUpFunction = new HbaseRowDataLookUpFunction(hbaseSchema, options);
48 | } catch (UnsupportedEncodingException e) {
49 | LOG.error("table schema encoding must by UTF-8", e);
50 | }
51 |
52 | return TableFunctionProvider.of(lookUpFunction);
53 | }
54 | }
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/connector/http/HttpClientUtil.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.cust.connector.http;
2 |
3 | import java.io.BufferedReader;
4 | import java.io.IOException;
5 | import java.io.InputStream;
6 | import java.io.InputStreamReader;
7 | import java.net.HttpURLConnection;
8 | import java.net.MalformedURLException;
9 | import java.net.URL;
10 |
11 | public class HttpClientUtil {
12 |
13 | public static String doGet(String httpurl) throws IOException {
14 | HttpURLConnection connection = null;
15 | InputStream is = null;
16 | BufferedReader br = null;
17 | // 返回结果字符串
18 | String result = null;
19 | try {
20 | // 创建远程url连接对象
21 | URL url = new URL(httpurl);
22 | // 通过远程url连接对象打开一个连接,强转成httpURLConnection类
23 | connection = (HttpURLConnection) url.openConnection();
24 | // 设置连接方式:get
25 | connection.setRequestMethod("GET");
26 | // 设置连接主机服务器的超时时间:15000毫秒
27 | connection.setConnectTimeout(15000);
28 | // 设置读取远程返回的数据时间:60000毫秒
29 | connection.setReadTimeout(60000);
30 | // 发送请求
31 | connection.connect();
32 | // 通过connection连接,获取输入流
33 | if (connection.getResponseCode() == 200) {
34 | is = connection.getInputStream();
35 | // 封装输入流is,并指定字符集
36 | br = new BufferedReader(new InputStreamReader(is, "UTF-8"));
37 |
38 | // 存放数据
39 | StringBuffer sbf = new StringBuffer();
40 | String temp = null;
41 | while ((temp = br.readLine()) != null) {
42 | sbf.append(temp);
43 | sbf.append("\r\n");
44 | }
45 | result = sbf.toString();
46 | }
47 | } catch (MalformedURLException e) {
48 | e.printStackTrace();
49 | } catch (IOException e) {
50 | e.printStackTrace();
51 | } finally {
52 | // 关闭资源
53 | if (null != br) {
54 | try {
55 | br.close();
56 | } catch (IOException e) {
57 | e.printStackTrace();
58 | }
59 | }
60 | if (null != is) {
61 | try {
62 | is.close();
63 | } catch (IOException e) {
64 | e.printStackTrace();
65 | }
66 | }
67 | connection.disconnect();
68 | }
69 | return result;
70 | }
71 | }
72 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/connector/http/HttpDynamicTableSource.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.cust.connector.http;
2 |
3 | import org.apache.flink.api.common.serialization.DeserializationSchema;
4 | import org.apache.flink.streaming.api.functions.source.SourceFunction;
5 | import org.apache.flink.table.connector.ChangelogMode;
6 | import org.apache.flink.table.connector.format.DecodingFormat;
7 | import org.apache.flink.table.connector.source.DynamicTableSource;
8 | import org.apache.flink.table.connector.source.ScanTableSource;
9 | import org.apache.flink.table.connector.source.SourceFunctionProvider;
10 | import org.apache.flink.table.data.RowData;
11 | import org.apache.flink.table.types.DataType;
12 |
13 | public class HttpDynamicTableSource implements ScanTableSource {
14 |
15 | private final String url;
16 | private final long interval;
17 | private final DecodingFormat> decodingFormat;
18 | private final DataType producedDataType;
19 |
20 | public HttpDynamicTableSource(
21 | String hostname,
22 | long interval,
23 | DecodingFormat> decodingFormat,
24 | DataType producedDataType) {
25 | this.url = hostname;
26 | this.interval = interval;
27 | this.decodingFormat = decodingFormat;
28 | this.producedDataType = producedDataType;
29 | }
30 |
31 | @Override
32 | public ChangelogMode getChangelogMode() {
33 | // in our example the format decides about the changelog mode
34 | // but it could also be the source itself
35 | return decodingFormat.getChangelogMode();
36 | }
37 |
38 | @Override
39 | public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) {
40 |
41 | // create runtime classes that are shipped to the cluster
42 | final DeserializationSchema deserializer = decodingFormat.createRuntimeDecoder(
43 | runtimeProviderContext,
44 | producedDataType);
45 |
46 | final SourceFunction sourceFunction = new HttpSource(url, interval, deserializer);
47 |
48 | return SourceFunctionProvider.of(sourceFunction, false);
49 | }
50 |
51 | @Override
52 | public DynamicTableSource copy() {
53 | return new HttpDynamicTableSource(url, interval, decodingFormat, producedDataType);
54 | }
55 |
56 | @Override
57 | public String asSummaryString() {
58 | return "Http Table Source";
59 | }
60 | }
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/connector/http/HttpSource.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.cust.connector.http;
2 |
3 | import org.apache.flink.api.common.serialization.DeserializationSchema;
4 | import org.apache.flink.configuration.Configuration;
5 | import org.apache.flink.metrics.Counter;
6 | import org.apache.flink.metrics.SimpleCounter;
7 | import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
8 | import org.apache.flink.table.data.RowData;
9 |
10 | /**
11 | * http table source
12 | */
13 | public class HttpSource extends RichSourceFunction {
14 |
15 | private volatile boolean isRunning = true;
16 | private String url;
17 | private long requestInterval;
18 | private DeserializationSchema deserializer;
19 | // count out event
20 | private transient Counter counter;
21 |
22 | public HttpSource(String url, long requestInterval, DeserializationSchema deserializer) {
23 | this.url = url;
24 | this.requestInterval = requestInterval;
25 | this.deserializer = deserializer;
26 | }
27 |
28 | @Override
29 | public void open(Configuration parameters) throws Exception {
30 |
31 | counter = new SimpleCounter();
32 | this.counter = getRuntimeContext()
33 | .getMetricGroup()
34 | .counter("myCounter");
35 | }
36 |
37 | @Override
38 | public void run(SourceContext ctx) throws Exception {
39 | while (isRunning) {
40 | try {
41 | // receive http message, csv format
42 | String message = HttpClientUtil.doGet(url);
43 | // deserializer csv message
44 | ctx.collect(deserializer.deserialize(message.getBytes()));
45 | this.counter.inc();
46 |
47 | Thread.sleep(requestInterval);
48 | } catch (Exception e) {
49 | e.printStackTrace();
50 | }
51 | }
52 |
53 | }
54 |
55 | @Override
56 | public void cancel() {
57 | isRunning = false;
58 | }
59 | }
60 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/connector/redis/RedisDynamicTableSource.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.cust.connector.redis;
2 |
3 | import org.apache.flink.api.java.typeutils.RowTypeInfo;
4 | import org.apache.flink.table.api.TableSchema;
5 | import org.apache.flink.table.connector.ChangelogMode;
6 | import org.apache.flink.table.connector.source.*;
7 | import org.apache.flink.table.types.DataType;
8 |
9 | import static org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo;
10 |
11 | public class RedisDynamicTableSource implements ScanTableSource, LookupTableSource {
12 |
13 |
14 | private final DataType producedDataType;
15 | private final RedisOption options;
16 | private final TableSchema physicalSchema;
17 |
18 | public RedisDynamicTableSource(
19 |
20 | DataType producedDataType,
21 | RedisOption options,
22 | TableSchema physicalSchema) {
23 |
24 | this.producedDataType = producedDataType;
25 | this.options = options;
26 | this.physicalSchema = physicalSchema;
27 | }
28 |
29 | @Override
30 | public ChangelogMode getChangelogMode() {
31 | // in our example the format decides about the changelog mode
32 | // but it could also be the source itself
33 | return ChangelogMode.insertOnly();
34 | }
35 |
36 | @Override
37 | public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) {
38 |
39 |
40 |
41 | return null;
42 | }
43 |
44 | @Override
45 | public DynamicTableSource copy() {
46 | return new RedisDynamicTableSource(producedDataType, options, physicalSchema);
47 | }
48 |
49 | @Override
50 | public String asSummaryString() {
51 | return "Redis Table Source, support Lookup function";
52 | }
53 |
54 | @Override
55 | public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) {
56 |
57 | final RowTypeInfo rowTypeInfo = (RowTypeInfo) fromDataTypeToLegacyInfo(producedDataType);
58 |
59 | String[] fieldNames = rowTypeInfo.getFieldNames();
60 |
61 | int[] lookupKeysIndex = context.getKeys()[0];
62 | int keyCount = lookupKeysIndex.length;
63 | String[] keyNames = new String[keyCount];
64 | for (int i = 0; i < keyCount; i++) {
65 | keyNames[i] = fieldNames[lookupKeysIndex[i]];
66 | }
67 | // new RedisRowDataLookUpFunction
68 | RedisRowDataLookUpFunction lookUpFunction
69 | = new RedisRowDataLookUpFunction(options);
70 |
71 | // return MysqlRowDataLookUpFunction
72 | return TableFunctionProvider.of(lookUpFunction);
73 | }
74 | }
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/connector/socket/SocketDynamicTableSink.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.cust.connector.socket;
2 |
3 | import org.apache.flink.api.common.serialization.SerializationSchema;
4 | import org.apache.flink.table.connector.ChangelogMode;
5 | import org.apache.flink.table.connector.format.EncodingFormat;
6 | import org.apache.flink.table.connector.sink.DynamicTableSink;
7 | import org.apache.flink.table.connector.sink.SinkFunctionProvider;
8 | import org.apache.flink.table.data.RowData;
9 | import org.apache.flink.table.types.DataType;
10 |
11 | /**
12 | * socket table sink
13 | */
14 | public class SocketDynamicTableSink implements DynamicTableSink {
15 |
16 | private final String hostname;
17 | private final int port;
18 | private final int maxRetry;
19 | private final long retryInterval;
20 | private final EncodingFormat> encodingFormat;
21 | private final byte byteDelimiter;
22 | private final DataType producedDataType;
23 |
24 | public SocketDynamicTableSink(
25 | String hostname,
26 | int port,
27 | int maxReTryTime,
28 | long retryInterval,
29 | EncodingFormat> encodingFormat,
30 | byte byteDelimiter,
31 | DataType producedDataType) {
32 | this.hostname = hostname;
33 | this.port = port;
34 | this.maxRetry = maxReTryTime;
35 | this.retryInterval = retryInterval;
36 | this.encodingFormat = encodingFormat;
37 | this.byteDelimiter = byteDelimiter;
38 | this.producedDataType = producedDataType;
39 | }
40 |
41 |
42 | @Override
43 | public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
44 | return ChangelogMode.insertOnly();
45 | }
46 |
47 | @Override
48 | public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
49 |
50 |
51 | final SerializationSchema serializer = encodingFormat.createRuntimeEncoder(context, producedDataType);
52 |
53 | SocketSinkFunction sink = new SocketSinkFunction(hostname, port, serializer, maxRetry, retryInterval);
54 | return SinkFunctionProvider.of(sink);
55 | }
56 |
57 |
58 | @Override
59 | public DynamicTableSink copy() {
60 | return new SocketDynamicTableSink(hostname, port, maxRetry, retryInterval, encodingFormat, byteDelimiter, producedDataType);
61 | }
62 |
63 | @Override
64 | public String asSummaryString() {
65 | return "Socket Table Sink";
66 | }
67 |
68 |
69 | }
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/connector/socket/SocketDynamicTableSource.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.cust.connector.socket;
2 |
3 | import org.apache.flink.api.common.serialization.DeserializationSchema;
4 | import org.apache.flink.streaming.api.functions.source.SourceFunction;
5 | import org.apache.flink.table.connector.ChangelogMode;
6 | import org.apache.flink.table.connector.format.DecodingFormat;
7 | import org.apache.flink.table.connector.source.DynamicTableSource;
8 | import org.apache.flink.table.connector.source.ScanTableSource;
9 | import org.apache.flink.table.connector.source.SourceFunctionProvider;
10 | import org.apache.flink.table.data.RowData;
11 | import org.apache.flink.table.types.DataType;
12 |
13 | public class SocketDynamicTableSource implements ScanTableSource {
14 |
15 | private final String hostname;
16 | private final int port;
17 | private final byte byteDelimiter;
18 | private final DecodingFormat> decodingFormat;
19 | private final DataType producedDataType;
20 |
21 | public SocketDynamicTableSource(
22 | String hostname,
23 | int port,
24 | byte byteDelimiter,
25 | DecodingFormat> decodingFormat,
26 | DataType producedDataType) {
27 | this.hostname = hostname;
28 | this.port = port;
29 | this.byteDelimiter = byteDelimiter;
30 | this.decodingFormat = decodingFormat;
31 | this.producedDataType = producedDataType;
32 | }
33 |
34 | @Override
35 | public ChangelogMode getChangelogMode() {
36 | // in our example the format decides about the changelog mode
37 | // but it could also be the source itself
38 | return decodingFormat.getChangelogMode();
39 | }
40 |
41 | @Override
42 | public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) {
43 |
44 | // create runtime classes that are shipped to the cluster
45 | // create deserializer
46 | final DeserializationSchema deserializer = decodingFormat.createRuntimeDecoder(
47 | runtimeProviderContext,
48 | producedDataType);
49 |
50 | final SourceFunction sourceFunction = new SocketSourceFunction(hostname, port, byteDelimiter, deserializer);
51 |
52 | return SourceFunctionProvider.of(sourceFunction, false);
53 | }
54 |
55 | @Override
56 | public DynamicTableSource copy() {
57 | return new SocketDynamicTableSource(hostname, port, byteDelimiter, decodingFormat, producedDataType);
58 | }
59 |
60 | @Override
61 | public String asSummaryString() {
62 | return "Socket Table Source";
63 | }
64 | }
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/format/changelog/csv/ChangelogCsvFormat.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.cust.format.changelog.csv;
2 |
3 | import org.apache.flink.api.common.serialization.DeserializationSchema;
4 | import org.apache.flink.api.common.typeinfo.TypeInformation;
5 | import org.apache.flink.table.connector.ChangelogMode;
6 | import org.apache.flink.table.connector.format.DecodingFormat;
7 | import org.apache.flink.table.connector.source.DynamicTableSource;
8 | import org.apache.flink.table.connector.source.DynamicTableSource.DataStructureConverter;
9 | import org.apache.flink.table.data.RowData;
10 | import org.apache.flink.table.types.DataType;
11 | import org.apache.flink.table.types.logical.LogicalType;
12 | import org.apache.flink.types.RowKind;
13 |
14 | import java.util.List;
15 |
16 | /**
17 | *
18 | * provide decode object
19 | */
20 | public class ChangelogCsvFormat implements DecodingFormat> {
21 |
22 | private final String columnDelimiter;
23 |
24 | public ChangelogCsvFormat(String columnDelimiter) {
25 | this.columnDelimiter = columnDelimiter;
26 | }
27 |
28 | /**
29 | *
30 | * @param context
31 | * @param producedDataType
32 | * @return
33 | */
34 | @Override
35 | @SuppressWarnings("unchecked")
36 | public DeserializationSchema createRuntimeDecoder(
37 | DynamicTableSource.Context context,
38 | DataType producedDataType) {
39 | // create type information for the DeserializationSchema
40 | // final TypeInformation producedTypeInfo = (TypeInformation) context.createTypeInformation(producedDataType);
41 | final TypeInformation producedTypeInfo = context.createTypeInformation(producedDataType);
42 |
43 | // most of the code in DeserializationSchema will not work on internal data structures
44 | // create a converter for conversion at the end
45 | final DataStructureConverter converter = context.createDataStructureConverter(producedDataType);
46 |
47 | // use logical types during runtime for parsing
48 | final List parsingTypes = producedDataType.getLogicalType().getChildren();
49 |
50 | // create runtime class
51 | return new ChangelogCsvDeserializer(parsingTypes, converter, producedTypeInfo, columnDelimiter);
52 | }
53 |
54 | @Override
55 | public ChangelogMode getChangelogMode() {
56 | // define that this format can produce INSERT and DELETE rows
57 | return ChangelogMode.newBuilder()
58 | .addContainedKind(RowKind.INSERT)
59 | .addContainedKind(RowKind.DELETE)
60 | .build();
61 | }
62 | }
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/cust/format/changelog/csv/ChangelogCsvFormatFactory.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.cust.format.changelog.csv;
2 |
3 | import org.apache.flink.api.common.serialization.DeserializationSchema;
4 | import org.apache.flink.configuration.ConfigOption;
5 | import org.apache.flink.configuration.ConfigOptions;
6 | import org.apache.flink.configuration.ReadableConfig;
7 | import org.apache.flink.table.connector.format.DecodingFormat;
8 | import org.apache.flink.table.data.RowData;
9 | import org.apache.flink.table.factories.FactoryUtil;
10 | import org.apache.flink.table.factories.DeserializationFormatFactory;
11 | import org.apache.flink.table.factories.DynamicTableFactory;
12 |
13 | import java.util.Collections;
14 | import java.util.HashSet;
15 | import java.util.Set;
16 |
17 | /**
18 | * FormatFaction, flink use spi register format factory
19 | * define format option and create Format object
20 | * flink format demo
21 | */
22 | public class ChangelogCsvFormatFactory implements DeserializationFormatFactory {
23 |
24 | // define all options statically
25 | public static final ConfigOption COLUMN_DELIMITER = ConfigOptions.key("column-delimiter")
26 | .stringType()
27 | .defaultValue("|");
28 |
29 | @Override
30 | public String factoryIdentifier() {
31 | return "changelog-csv"; // used for matching to `format = 'changelog-csv'`
32 | }
33 |
34 | /**
35 | * special require properties
36 | *
37 | * @return require properties
38 | */
39 | @Override
40 | public Set> requiredOptions() {
41 | return Collections.emptySet();
42 | }
43 |
44 | /**
45 | * special option properties
46 | *
47 | * @return option properties
48 | */
49 | @Override
50 | public Set> optionalOptions() {
51 | final Set> options = new HashSet<>();
52 | options.add(COLUMN_DELIMITER);
53 | return options;
54 | }
55 |
56 | /**
57 | * create decode format class, for decode recode
58 | *
59 | * @param context runtime context
60 | * @param formatOptions format option
61 | * @return format class
62 | */
63 | @Override
64 | public DecodingFormat> createDecodingFormat(
65 | DynamicTableFactory.Context context,
66 | ReadableConfig formatOptions) {
67 | // either implement your custom validation logic here ...
68 | // or use the provided helper method
69 | FactoryUtil.validateFactoryOptions(this, formatOptions);
70 |
71 | // get the validated options
72 | final String columnDelimiter = formatOptions.get(COLUMN_DELIMITER);
73 |
74 | // create and return the format
75 | return new ChangelogCsvFormat(columnDelimiter);
76 | }
77 | }
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/udaf/BloomFilter.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.udaf;
2 |
3 | import com.google.common.hash.Funnels;
4 | import org.apache.flink.table.functions.AggregateFunction;
5 | import org.apache.flink.table.functions.FunctionContext;
6 | import org.slf4j.Logger;
7 | import org.slf4j.LoggerFactory;
8 |
9 | import java.nio.charset.StandardCharsets;
10 |
11 | /**
12 | * bloom filter
13 | *
14 | * todo acc cannot agg over window
15 | */
16 |
17 | public class BloomFilter extends AggregateFunction {
18 |
19 | private final static Logger LOG = LoggerFactory.getLogger(BloomFilter.class);
20 | private com.google.common.hash.BloomFilter filter;
21 | @Override
22 | public void open(FunctionContext context) throws Exception {
23 | LOG.info("bloom filter open...");
24 | // 创建布隆过滤器对象, 预期数据量,误判率
25 | filter = com.google.common.hash.BloomFilter.create(
26 | Funnels.byteArrayFunnel(),
27 | 1000 * 10000,
28 | 0.01);
29 | }
30 |
31 | public void accumulate(CountAcc acc, String userId) {
32 |
33 | if (userId == null || userId.length() == 0) {
34 | return;
35 | }
36 | // parse userId to byte
37 | byte[] arr = userId.getBytes(StandardCharsets.UTF_8);
38 | // check userId exists bloom filter
39 | if(!filter.mightContain(arr)){
40 | // not exists
41 | filter.put(arr);
42 | // count ++
43 | acc.count += 1;
44 | }
45 |
46 | }
47 |
48 | @Override
49 | public void close() throws Exception {
50 | }
51 |
52 | @Override
53 | public Integer getValue(CountAcc acc) {
54 | LOG.info("filter : " + (filter == null));
55 | // get
56 | return acc.count;
57 | }
58 |
59 | @Override
60 | public CountAcc createAccumulator() {
61 | CountAcc acc = new CountAcc();
62 | return acc;
63 | }
64 |
65 | public void merge(CountAcc acc, Iterable it) {
66 | int last = acc.count;
67 | StringBuilder builder = new StringBuilder();
68 | for (CountAcc a : it) {
69 | // LOG.info("last value : " + a.count);
70 | acc.count += a.count;
71 | }
72 | }
73 |
74 |
75 | }
76 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/udaf/CountAcc.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.udaf;
2 |
3 | public class CountAcc {
4 | public int count = 0;
5 | public String key;
6 |
7 | }
8 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/udaf/JedisRedisUv.java:
--------------------------------------------------------------------------------
1 | //package com.rookie.submit.udaf;
2 | //
3 | //import org.apache.flink.table.functions.AggregateFunction;
4 | //import org.apache.flink.table.functions.FunctionContext;
5 | //import org.slf4j.Logger;
6 | //import org.slf4j.LoggerFactory;
7 | //import redis.clients.jedis.Jedis;
8 | //
9 | ///**
10 | // * use jedis
11 | // */
12 | //public class JedisRedisUv extends AggregateFunction {
13 | //
14 | // private final static Logger LOG = LoggerFactory.getLogger(JedisRedisUv.class);
15 | // private String host;
16 | // private int port;
17 | // private Jedis jedis;
18 | // private String key;
19 | //
20 | // public JedisRedisUv(String host, int port) {
21 | // this.host = host;
22 | // this.port = port;
23 | // }
24 | //
25 | // @Override
26 | // public void open(FunctionContext context) throws Exception {
27 | // // connect redis
28 | // reconnect();
29 | // }
30 | //
31 | // public void reconnect() {
32 | // jedis = new Jedis(this.host, this.port);
33 | // }
34 | //
35 | // public void accumulate(CountAcc acc, String key, String userId) {
36 | //
37 | // this.key = key;
38 | // if (acc.key == null) {
39 | // acc.key = key;
40 | // }
41 | // acc.count += 1;
42 | // int retry = 3;
43 | // while (retry >= 1) {
44 | // try {
45 | // jedis.hset(key, userId, "1");
46 | // break;
47 | // } catch (Exception e) {
48 | // LOG.info("set redis error, retry");
49 | // reconnect();
50 | // retry -= 1;
51 | // }
52 | // }
53 | //
54 | // }
55 | //
56 | // @Override
57 | // public Integer getValue(CountAcc acc) {
58 | // long start = System.currentTimeMillis();
59 | // int size = 0;
60 | // if (acc.key == null) {
61 | // return size;
62 | // }
63 | // // get all userId, count size
64 | // int retry = 3;
65 | // while (retry >= 1) {
66 | // try {
67 | // jedis.flushAll();
68 | // size = jedis.hgetAll(this.key).size();
69 | // break;
70 | // } catch (Exception e) {
71 | // LOG.info("set redis error, retry");
72 | // reconnect();
73 | // retry -= 1;
74 | // }
75 | // }
76 | // long end = System.currentTimeMillis();
77 | // LOG.info("count all cost : " + (end - start));
78 | // return size;
79 | // }
80 | //
81 | // @Override
82 | // public CountAcc createAccumulator() {
83 | //
84 | // CountAcc acc = new CountAcc();
85 | // acc.key = this.key;
86 | // return acc;
87 | // }
88 | //
89 | // public void merge(CountAcc acc, Iterable it) {
90 | // // do nothing
91 | // it.forEach(item -> acc.count += item.count);
92 | //
93 | // }
94 | //}
95 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/udaf/RedisUv2.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.udaf;
2 |
3 | import io.lettuce.core.RedisClient;
4 | import io.lettuce.core.api.StatefulRedisConnection;
5 | import io.lettuce.core.api.sync.RedisCommands;
6 | import org.apache.flink.table.functions.AggregateFunction;
7 | import org.apache.flink.table.functions.FunctionContext;
8 | import org.slf4j.Logger;
9 | import org.slf4j.LoggerFactory;
10 |
11 | /**
12 | * udaf for query redis only get value(window fire)
13 | */
14 | public class RedisUv2 extends AggregateFunction {
15 |
16 | private final static Logger LOG = LoggerFactory.getLogger(RedisUv2.class);
17 | // "redis://localhost"
18 | private String url;
19 | private StatefulRedisConnection connection;
20 | private RedisClient redisClient;
21 | private RedisCommands sync;
22 | private String key;
23 |
24 | public RedisUv2(String url, String key ) {
25 | this.url = url;
26 | this.key = key;
27 | }
28 |
29 | @Override
30 | public void open(FunctionContext context) throws Exception {
31 | // connect redis
32 | reconnect();
33 | }
34 |
35 | public void reconnect() {
36 | redisClient = RedisClient.create(this.url);
37 | connection = redisClient.connect();
38 | sync = connection.sync();
39 | }
40 |
41 | public void accumulate(Integer acc, String key, String userId) {
42 |
43 | // if (this.key == null) {
44 | // this.key = key;
45 | // }
46 | int retry = 3;
47 | while (retry >= 1) {
48 | try {
49 | sync.hset(key, userId, "0");
50 | return;
51 | } catch (Exception e) {
52 | LOG.info("set redis error, retry");
53 | reconnect();
54 | retry -= 1;
55 | }
56 | }
57 |
58 | }
59 |
60 | @Override
61 | public Integer getValue(Integer accumulator) {
62 | long start = System.currentTimeMillis();
63 | int size = 0;
64 | if (this.key == null) {
65 | return size;
66 | }
67 | // get all userId, count size
68 | int retry = 3;
69 | while (retry >= 1) {
70 | try {
71 | size = sync.hgetall(this.key).size();
72 | break;
73 | } catch (Exception e) {
74 | LOG.info("set redis error, retry");
75 | reconnect();
76 | retry -= 1;
77 | }
78 | }
79 | long end = System.currentTimeMillis();
80 | LOG.info("count all cost : " + (end - start));
81 | return size;
82 | }
83 |
84 | @Override
85 | public Integer createAccumulator() {
86 | return 0;
87 | }
88 |
89 | public void merge(Integer acc, Iterable it) {
90 | // do nothing
91 | }
92 | }
93 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/udaf/math/Median.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.udaf.math;
2 |
3 | import org.apache.flink.table.functions.AggregateFunction;
4 | import java.util.List;
5 | import java.util.stream.Collectors;
6 |
7 | /**
8 | * agg function: 计算中位数
9 | */
10 | public class Median extends AggregateFunction {
11 | // 获取 acc 的值
12 | @Override
13 | public Double getValue(NumberAcc acc) {
14 | // sort list
15 | List list = acc.list.stream().sorted().collect(Collectors.toList());
16 | // if list is empty, return null
17 | if (list.size() == 0) {
18 | return null;
19 | } else if (list.size() == 1) {
20 | // if list have one element, return it
21 | return list.get(0);
22 | }
23 | double val;
24 | int size = list.size();
25 | int half = size / 2;
26 | if (size % 2 == 0) {
27 | //even, use (size/2 - 1 + size/2) / 2
28 | val = (list.get(half - 1) + list.get(half)) / 2;
29 | } else {
30 | // odd, use size/2
31 | val = list.get(half);
32 | }
33 | return val;
34 | }
35 | // 累加元素
36 | public void accumulate(NumberAcc acc, Double d) {
37 | acc.list.add(d);
38 | }
39 | // 创建累加器
40 | @Override
41 | public NumberAcc createAccumulator() {
42 | return new NumberAcc();
43 | }
44 |
45 | // 窗口聚合
46 | public void merge(NumberAcc acc, Iterable it) {
47 | for (NumberAcc a : it) {
48 | acc.list.addAll(a.list);
49 | }
50 | }
51 | }
52 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/udaf/math/NumberAcc.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.udaf.math;
2 |
3 | import java.util.ArrayList;
4 | import java.util.List;
5 |
6 | /**
7 | * 累加器 存储了聚合的中间结果
8 | */
9 | public class NumberAcc {
10 | public List list = new ArrayList<>();
11 | }
12 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/udf/DateAdd.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.udf;
2 |
3 | import com.rookie.submit.util.DateTimeUtil;
4 | import org.apache.flink.table.annotation.DataTypeHint;
5 | import org.apache.flink.table.annotation.FunctionHint;
6 | import org.apache.flink.table.functions.FunctionContext;
7 | import org.apache.flink.table.functions.ScalarFunction;
8 | import org.slf4j.Logger;
9 | import org.slf4j.LoggerFactory;
10 |
11 | import java.util.*;
12 |
13 | public class DateAdd extends ScalarFunction {
14 |
15 | private final static Logger LOG = LoggerFactory.getLogger(DateAdd.class);
16 | public static volatile Map map = new HashMap<>();
17 |
18 | @Override
19 | public void open(FunctionContext context) throws Exception {
20 | super.open(context);
21 | map.put("" + System.currentTimeMillis(), "aa-" + System.currentTimeMillis());
22 |
23 | // new Timer
24 | // Timer timer = new Timer(true);
25 | // // schedule is 10 second 定义了一个10秒的定时器,定时执行查询数据库的方法
26 | //
27 | // timer.schedule(new TimerTask() {
28 | // @Override
29 | // public void run() {
30 | // query();
31 | // }
32 | // }, 10000, 10000);
33 | }
34 |
35 | // public void query(){
36 | // map.put("" + System.currentTimeMillis(), "aa-" + System.currentTimeMillis());
37 | // LOG.info("timer run, map element size : " + map.size());
38 | // }
39 |
40 | @FunctionHint(output = @DataTypeHint("STRING"))
41 | public String eval(String tar, int num) {
42 |
43 | if (tar == null || tar.length() == 0)
44 | return null;
45 | Date day = null;
46 | try {
47 | day = DateTimeUtil.parse(tar);
48 | } catch (Exception e) {
49 | return null;
50 | }
51 | day = DateTimeUtil.plusDay(day, num);
52 |
53 | String resultStr = DateTimeUtil.format(day, DateTimeUtil.YYYY_MM_DD);
54 |
55 | return resultStr + "-" + map.size();
56 |
57 | }
58 | }
59 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/udf/ParseDctJson.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.udf;
2 |
3 | import com.google.gson.JsonArray;
4 | import com.google.gson.JsonElement;
5 | import com.google.gson.JsonObject;
6 | import com.google.gson.JsonParser;
7 | import org.apache.flink.table.annotation.DataTypeHint;
8 | import org.apache.flink.table.annotation.FunctionHint;
9 | import org.apache.flink.table.functions.TableFunction;
10 | import org.apache.flink.types.Row;
11 | import org.apache.flink.types.RowKind;
12 | import org.slf4j.Logger;
13 | import org.slf4j.LoggerFactory;
14 |
15 | public class ParseDctJson extends TableFunction {
16 |
17 | private final static Logger LOG = LoggerFactory.getLogger(ParseDctJson.class);
18 |
19 | @FunctionHint(output = @DataTypeHint("ROW>"))
20 | public void eval(String json) {
21 | if (json == null || json.length() == 0) {
22 | return;
23 | }
24 | String[] arr = getString(json);
25 | RowKind rowKind = RowKind.fromByteValue((byte) 0);
26 | Row row = new Row(rowKind, 1);
27 | row.setField(0, arr);
28 | collect(row);
29 | }
30 |
31 | /**
32 | * parse user columns from json and provider column name
33 | */
34 | private String[] getString(String json) {
35 | try {
36 | JsonObject jsonObject = new JsonParser().parse(json).getAsJsonObject();
37 |
38 | JsonObject data = jsonObject.getAsJsonObject("data");
39 | if (data == null) {
40 | return null;
41 | }
42 | JsonElement rows = data.get("rows");
43 | if (rows == null) {
44 | return null;
45 | }
46 | JsonArray array = rows.getAsJsonArray();
47 |
48 | String[] result = new String[array.size()];
49 |
50 | for (int i = 0; i < result.length; i++) {
51 |
52 | JsonElement tmp = array.get(i);
53 | if (tmp == null || tmp.isJsonNull()) {
54 | result[i] = null;
55 | } else {
56 | result[i] = tmp.getAsString();
57 | }
58 | }
59 | return result;
60 |
61 | } catch (Exception e){
62 | LOG.warn("parse input error : " + json);
63 | e.printStackTrace();
64 | }
65 |
66 | return null;
67 | }
68 | }
69 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/udf/ParseJson.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.udf;
2 |
3 | import com.google.gson.JsonElement;
4 | import com.google.gson.JsonObject;
5 | import com.google.gson.JsonParser;
6 | import org.apache.flink.table.annotation.DataTypeHint;
7 | import org.apache.flink.table.annotation.FunctionHint;
8 | import org.apache.flink.table.functions.TableFunction;
9 | import org.apache.flink.types.Row;
10 | import org.apache.flink.types.RowKind;
11 |
12 | // 对应 sql : parse_complex_json_1.sql
13 | public class ParseJson extends TableFunction {
14 |
15 | @FunctionHint(output = @DataTypeHint("ROW>"))
16 | public void eval(String... json) {
17 | if (json == null || json.length == 0 || json[0] == null) {
18 | return;
19 | }
20 | String[] arr = getStrings(json);
21 | RowKind rowKind = RowKind.fromByteValue((byte) 0);
22 | Row row = new Row(rowKind, 1);
23 | row.setField(0, arr);
24 | collect(row);
25 | }
26 |
27 | /**
28 | * parse user columns from json and provider column name
29 | */
30 | private String[] getStrings(String[] json) {
31 | JsonObject jsonObject = new JsonParser().parse(json[0]).getAsJsonObject();
32 | int len = json.length - 1;
33 | String[] arr = new String[len];
34 | for (int i = 0; i < len; ++i) {
35 | JsonElement tm = jsonObject.get(json[i + 1]);
36 | if (tm != null) {
37 | arr[i] = tm.getAsString();
38 | } else {
39 | arr[i] = null;
40 | }
41 | }
42 | return arr;
43 | }
44 | }
45 |
--------------------------------------------------------------------------------
/src/main/java/com/rookie/submit/udtf/UdtfTimer.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.udtf;
2 |
3 | import org.apache.flink.table.annotation.DataTypeHint;
4 | import org.apache.flink.table.annotation.FunctionHint;
5 | import org.apache.flink.table.functions.FunctionContext;
6 | import org.apache.flink.table.functions.TableFunction;
7 | import org.apache.flink.types.Row;
8 | import org.slf4j.Logger;
9 | import org.slf4j.LoggerFactory;
10 |
11 | import java.sql.*;
12 | import java.util.*;
13 |
14 |
15 | /**
16 | * udtf timer
17 | */
18 | @FunctionHint(output = @DataTypeHint("ROW"))
19 | public class UdtfTimer extends TableFunction {
20 | private final static Logger LOG = LoggerFactory.getLogger(UdtfTimer.class);
21 |
22 | private long expireTime;
23 | private boolean closeConnect = true;
24 |
25 | public static volatile Map map = new HashMap<>();
26 |
27 | public UdtfTimer(long expireTime) {
28 | this.expireTime = expireTime;
29 | if (expireTime < 600) {
30 | closeConnect = false;
31 | }
32 | }
33 |
34 | @Override
35 | public void open(FunctionContext context) throws Exception {
36 |
37 | // new Timer
38 | Timer timer = new Timer(true);
39 | // schedule is 10 second 定义了一个10秒的定时器,定时执行查询数据库的方法
40 | timer.schedule(new TimerTask() {
41 | @Override
42 | public void run() {
43 | query();
44 | }
45 | }, 10000, 10000);
46 |
47 | }
48 | public void query(){
49 | map.put("" + System.currentTimeMillis(), "aa-" + System.currentTimeMillis());
50 | LOG.info("timer run, map element size : " + map.size());
51 | }
52 |
53 |
54 | public void eval(String key) throws SQLException {
55 | collect(Row.of(key, map.size()));
56 | }
57 |
58 |
59 | @Override
60 | public void close() throws Exception {
61 | super.close();
62 | }
63 | }
64 |
--------------------------------------------------------------------------------
/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory:
--------------------------------------------------------------------------------
1 | com.rookie.submit.cust.connector.socket.SocketDynamicTableFactory
2 | com.rookie.submit.cust.connector.http.HttpDynamicTableFactory
3 | com.rookie.submit.cust.connector.mysql.MysqlDynamicTableFactory
4 | com.rookie.submit.cust.connector.hbase.HbaseDynamicTableFactory
5 | com.rookie.submit.cust.connector.redis.RedisDynamicTableFactory
6 | com.rookie.submit.cust.format.changelog.csv.ChangelogCsvFormatFactory
7 | com.rookie.submit.cust.connector.starrocks.StarrocksDynamicTableFactory
8 |
--------------------------------------------------------------------------------
/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory:
--------------------------------------------------------------------------------
1 | #com.rookie.submit.connector.kafka.KafkaUpsertTableSinkFactory
2 |
--------------------------------------------------------------------------------
/src/main/resources/demoJobPropFile.properties:
--------------------------------------------------------------------------------
1 | ## filesystem & rocksdb
2 | state.backend=filesystem
--------------------------------------------------------------------------------
/src/main/resources/log4j.properties:
--------------------------------------------------------------------------------
1 | ################################################################################
2 | # Licensed to the Apache Software Foundation (ASF) under one
3 | # or more contributor license agreements. See the NOTICE file
4 | # distributed with this work for additional information
5 | # regarding copyright ownership. The ASF licenses this file
6 | # to you under the Apache License, Version 2.0 (the
7 | # "License"); you may not use this file except in compliance
8 | # with the License. You may obtain a copy of the License at
9 | #
10 | # http://www.apache.org/licenses/LICENSE-2.0
11 | #
12 | # Unless required by applicable law or agreed to in writing, software
13 | # distributed under the License is distributed on an "AS IS" BASIS,
14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | # See the License for the specific language governing permissions and
16 | # limitations under the License.
17 | ################################################################################
18 | log4j.rootLogger=info, console
19 | log4j.appender.console=org.apache.log4j.ConsoleAppender
20 | log4j.appender.console.layout=org.apache.log4j.PatternLayout
21 | log4j.appender.console.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss,SSS} %-5p - %m%n
22 | #log4j.appender.console.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss,SSS} %-5p %-10c %x - %m%n
23 |
24 |
--------------------------------------------------------------------------------
/src/main/resources/sql/dev/create_table_datagen.sql:
--------------------------------------------------------------------------------
1 | -- -- kafka source
2 | -- set execution.runtime-mode=BATCH;
3 | drop table if exists user_log1;
4 | CREATE TABLE user_log1
5 | (
6 | user_id VARCHAR,
7 | item_id VARCHAR,
8 | category_id VARCHAR,
9 | behavior VARCHAR
10 | ) WITH (
11 | 'connector' = 'datagen'
12 | ,'rows-per-second' = '20'
13 | ,'number-of-rows' = '10000'
14 | ,'fields.user_id.kind' = 'random'
15 | ,'fields.item_id.kind' = 'random'
16 | ,'fields.category_id.kind' = 'random'
17 | ,'fields.behavior.kind' = 'random'
18 | ,'fields.user_id.length' = '20'
19 | ,'fields.item_id.length' = '10'
20 | ,'fields.category_id.length' = '10'
21 | ,'fields.behavior.length' = '10'
22 | );
23 |
--------------------------------------------------------------------------------
/src/main/resources/sql/dev/datagen_to_hive.sql:
--------------------------------------------------------------------------------
1 | -- -- kafka source
2 | -- set execution.runtime-mode=BATCH;
3 | drop table if exists user_log;
4 | CREATE TABLE user_log
5 | (
6 | user_id VARCHAR,
7 | item_id VARCHAR,
8 | category_id VARCHAR,
9 | behavior VARCHAR
10 | ) WITH (
11 | 'connector' = 'datagen'
12 | ,'rows-per-second' = '20'
13 | ,'number-of-rows' = '10000'
14 | ,'fields.user_id.kind' = 'random'
15 | ,'fields.item_id.kind' = 'random'
16 | ,'fields.category_id.kind' = 'random'
17 | ,'fields.behavior.kind' = 'random'
18 | ,'fields.user_id.length' = '20'
19 | ,'fields.item_id.length' = '10'
20 | ,'fields.category_id.length' = '10'
21 | ,'fields.behavior.length' = '10'
22 | );
23 |
24 |
25 |
26 | insert into myHive.test.user_log
27 | select user_id, item_id, category_id, behavior, DATE_FORMAT(now(), 'yyyy-MM-dd-HH-mm')
28 | from user_log;
29 |
--------------------------------------------------------------------------------
/src/main/resources/sql/dev/datagen_to_hive_2.sql:
--------------------------------------------------------------------------------
1 | -- -- kafka source
2 | -- set execution.runtime-mode=BATCH;
3 | drop table if exists user_log_2;
4 | CREATE TABLE user_log_2
5 | (
6 | user_id VARCHAR,
7 | item_id VARCHAR,
8 | category_id VARCHAR,
9 | behavior VARCHAR
10 | ) WITH (
11 | 'connector' = 'datagen'
12 | ,'rows-per-second' = '20'
13 | ,'number-of-rows' = '10000'
14 | ,'fields.user_id.kind' = 'random'
15 | ,'fields.item_id.kind' = 'random'
16 | ,'fields.category_id.kind' = 'random'
17 | ,'fields.behavior.kind' = 'random'
18 | ,'fields.user_id.length' = '20'
19 | ,'fields.item_id.length' = '10'
20 | ,'fields.category_id.length' = '10'
21 | ,'fields.behavior.length' = '10'
22 | );
23 | --
24 | --
25 | set table.sql-dialect=hive;
26 | drop table if exists myHive.test.user_log_2;
27 | CREATE TABLE myHive.test.user_log_2 (
28 | user_id STRING
29 | ,item_id STRING
30 | ,category_id STRING
31 | ,behavior STRING
32 | ) PARTITIONED BY (ds STRING) STORED AS parquet TBLPROPERTIES (
33 | 'partition.time-extractor.timestamp-pattern'='$ds:00',
34 | 'sink.partition-commit.trigger'='partition-time',
35 | 'sink.partition-commit.delay'='1 min',
36 | 'sink.partition-commit.policy.kind'='metastore,success-file'
37 | );
38 |
39 |
40 | -- streaming sql, insert into hive table
41 | set table.sql-dialect=default;
42 | insert into myHive.test.user_log_2
43 | SELECT user_id, item_id, category_id, behavior, DATE_FORMAT(now(), 'yyyy-MM-dd HH:mm') --,DATE_FORMAT(now(), 'HH')
44 | FROM user_log;
45 |
46 |
47 |
48 |
--------------------------------------------------------------------------------
/src/main/resources/sql/dev/datagen_to_kafka.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | -- drop table if exists user_log;
3 | CREATE TABLE if not exists user_log
4 | (
5 | user_id VARCHAR,
6 | item_id VARCHAR,
7 | category_id VARCHAR,
8 | behavior VARCHAR
9 | )
10 | COMMENT 'abcdefs'
11 | WITH (
12 | 'connector' = 'datagen'
13 | ,'rows-per-second' = '20'
14 | ,'number-of-rows' = '10000'
15 | ,'fields.user_id.kind' = 'random'
16 | ,'fields.item_id.kind' = 'random'
17 | ,'fields.category_id.kind' = 'random'
18 | ,'fields.behavior.kind' = 'random'
19 | ,'fields.user_id.length' = '20'
20 | ,'fields.item_id.length' = '10'
21 | ,'fields.category_id.length' = '10'
22 | ,'fields.behavior.length' = '10'
23 | );
24 | --
25 | --
26 | -- -- set table.sql-dialect=hive;
27 | -- -- kafka sink
28 | drop table if exists user_log_sink;
29 | CREATE TABLE user_log_sink
30 | (
31 | user_id STRING,
32 | item_id STRING,
33 | category_id STRING,
34 | behavior STRING
35 | ) WITH (
36 | 'connector' = 'kafka'
37 | ,'topic' = 'user_log_test'
38 | -- ,'properties.bootstrap.servers' = 'host.docker.internal:9092'
39 | ,'properties.bootstrap.servers' = 'localhost:9092'
40 | ,'properties.group.id' = 'user_log'
41 | ,'scan.startup.mode' = 'latest-offset'
42 | ,'format' = 'json'
43 | );
44 |
45 |
46 | -- streaming sql, insert into mysql table
47 | insert into user_log_sink
48 | SELECT user_id, item_id, category_id, behavior
49 | FROM user_log;
50 |
--------------------------------------------------------------------------------
/src/main/resources/sql/dev/flink_cdc_test.sql:
--------------------------------------------------------------------------------
1 | -- mysql cdc to print
2 | -- creates a mysql table source
3 | CREATE TABLE t_user_log
4 | (
5 | category_id varchar(20),
6 | behavior varchar(20),
7 | cnt bigint,
8 | ts timestamp(3),
9 | db_name STRING METADATA FROM 'database_name' VIRTUAL,
10 | table_name STRING METADATA FROM 'table_name' VIRTUAL,
11 | operation_ts TIMESTAMP_LTZ(3) METADATA FROM 'op_ts' VIRTUAL,
12 | proc_time as PROCTIME(),
13 | PRIMARY KEY (category_id, behavior) NOT ENFORCED
14 | ) WITH (
15 | 'connector' = 'mysql-cdc'
16 | ,'hostname' = 'localhost'
17 | ,'port' = '3306'
18 | ,'username' = 'root'
19 | ,'password' = '123456'
20 | ,'database-name' = 'venn'
21 | ,'table-name' = 'user_view'
22 | ,'server-id' = '5400-5440'
23 | ,'scan.startup.mode' = 'initial'
24 | -- ,'server-time-zone' = 'UTC'
25 | -- ,'scan.startup.mode' = 'specific-offset'
26 | -- ,'scan.startup.specific-offset.file' = 'mysql-bin.000001'
27 | -- ,'scan.startup.specific-offset.pos' = '1'
28 | );
29 |
30 | -- kafka sink
31 | drop table if exists t_user_log_sink;
32 | CREATE TABLE t_user_log_sink
33 | (
34 | category_id varchar(20),
35 | behavior varchar(20),
36 | cnt bigint,
37 | ts timestamp(3)
38 | ) WITH (
39 | 'connector' = 'print'
40 | -- 'connector' = 'upsert-kafka'
41 | -- ,'topic' = 'user_log_sink'
42 | -- ,'properties.bootstrap.servers' = 'localhost:9092'
43 | -- ,'properties.group.id' = 'user_log'
44 | -- ,'key.format' = 'json'
45 | -- ,'key.json.ignore-parse-errors' = 'true'
46 | -- ,'value.format' = 'json'
47 | -- ,'value.json.fail-on-missing-field' = 'false'
48 | -- ,'value.fields-include' = 'ALL'
49 | );
50 |
51 | insert into t_user_log_sink
52 | select category_id, behavior, cnt, ts
53 | from t_user_log;
--------------------------------------------------------------------------------
/src/main/resources/sql/dev/hive_to_hive.sql:
--------------------------------------------------------------------------------
1 | set execution.runtime-mode=BATCH;
2 | set table.sql-dialect=hive;
3 | drop table if exists myHive.test.user_log_1;
4 | CREATE TABLE myHive.test.user_log_1 (
5 | user_id STRING
6 | ,item_id STRING
7 | ,category_id STRING
8 | ,behavior STRING
9 | ) PARTITIONED BY (ds STRING) STORED AS parquet TBLPROPERTIES (
10 | 'partition.time-extractor.timestamp-pattern'='$dt $hr:00:00',
11 | 'sink.partition-commit.trigger'='partition-time',
12 | 'sink.partition-commit.delay'='1 min',
13 | 'sink.partition-commit.policy.kind'='metastore,success-file'
14 | );
15 |
16 | set table.sql-dialect=default;
17 | insert into myHive.test.user_log_1
18 | select * from myHive.test.user_log;
--------------------------------------------------------------------------------
/src/main/resources/sql/dev/hive_to_print.sql:
--------------------------------------------------------------------------------
1 | -- read hive, write to print -- batch when read complete, job finish
2 | -- sink
3 | drop table if exists read_hiv_sink;
4 | CREATE TABLE read_hiv_sink (
5 | user_id VARCHAR
6 | ,item_id VARCHAR
7 | ,category_id VARCHAR
8 | ,behavior VARCHAR
9 | ,ds VARCHAR
10 | ) WITH (
11 | 'connector' = 'print'
12 | );
13 |
14 | -- set streaming-source.enable = false;
15 | -- set execution.runtime-mode = batch;
16 | insert into read_hiv_sink
17 | select user_id, item_id, category_id, behavior, ds
18 | from myHive.test.user_log;
--------------------------------------------------------------------------------
/src/main/resources/sql/dev/insert.sql:
--------------------------------------------------------------------------------
1 | insert into user_log_sink
2 | SELECT user_id, item_id, category_id, behavior
3 | FROM user_log;
--------------------------------------------------------------------------------
/src/main/resources/sql/dev/kafka_to_hdfs.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | drop table if exists user_log;
3 | CREATE TABLE user_log
4 | (
5 | `event_time` TIMESTAMP(3) METADATA FROM 'timestamp' VIRTUAL, -- from Debezium format
6 | `partition_id` BIGINT METADATA FROM 'partition' VIRTUAL, -- from Kafka connector
7 | `offset` BIGINT METADATA VIRTUAL, -- from Kafka connector
8 | user_id VARCHAR,
9 | item_id VARCHAR,
10 | category_id VARCHAR,
11 | behavior VARCHAR,
12 | ts TIMESTAMP(3),
13 | WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
14 | ) WITH (
15 | 'connector' = 'kafka'
16 | ,'topic' = 'user_log'
17 | ,'properties.bootstrap.servers' = 'localhost:9092'
18 | ,'properties.group.id' = 'user_log'
19 | ,'scan.startup.mode' = 'earliest-offset'
20 | ,'format' = 'json'
21 | );
22 |
23 |
24 | -- set table.sql-dialect=hive;
25 | -- kafka sink
26 | drop table if exists user_log_sink;
27 | CREATE TABLE user_log_sink
28 | (
29 | event_time timestamp(3),
30 | partition_id bigint,
31 | `offset` bigint,
32 | batch STRING,
33 | user_id STRING,
34 | item_id STRING,
35 | category_id STRING,
36 | behavior STRING,
37 | ts timestamp(3),
38 | current_t timestamp(3)
39 | ) WITH (
40 | 'connector' = 'filesystem'
41 | ,'path' = 'hdfs:///tmp/data/user_log'
42 | ,'format' = 'csv'
43 | ,'sink.rolling-policy.file-size' = '10M'
44 | ,'sink.rolling-policy.rollover-interval' = '1 min'
45 | ,'sink.rolling-policy.check-interval' = '1 min'
46 | );
47 |
48 |
49 | -- streaming sql, insert into mysql table
50 | insert into user_log_sink
51 | SELECT event_time,
52 | partition_id,
53 | `offset`,
54 | '1',
55 | user_id,
56 | item_id,
57 | category_id,
58 | udf_date_add('2022-04-24 00:00:00', -1),
59 | ts,
60 | now()
61 | FROM user_log;
62 |
--------------------------------------------------------------------------------
/src/main/resources/sql/dev/kafka_to_kafka.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | drop table if exists user_log;
3 | CREATE TABLE user_log
4 | (
5 | `event_time` TIMESTAMP(3) METADATA FROM 'timestamp' VIRTUAL, -- from Debezium format
6 | `partition_id` BIGINT METADATA FROM 'partition' VIRTUAL, -- from Kafka connector
7 | `offset` BIGINT METADATA VIRTUAL, -- from Kafka connector
8 | user_id VARCHAR,
9 | item_id VARCHAR,
10 | category_id VARCHAR,
11 | behavior VARCHAR,
12 | ts TIMESTAMP(3),
13 | WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
14 | ) WITH (
15 | 'connector' = 'kafka'
16 | ,'topic' = 'user_log'
17 | ,'properties.bootstrap.servers' = 'localhost:9092'
18 | ,'properties.group.id' = 'user_log'
19 | ,'scan.startup.mode' = 'earliest-offset'
20 | ,'format' = 'json'
21 | );
22 |
23 |
24 | -- set table.sql-dialect=hive;
25 | -- kafka sink
26 | drop table if exists user_log_sink;
27 | CREATE TABLE user_log_sink
28 | (
29 | event_time timestamp(3),
30 | partition_id bigint,
31 | `offset` bigint,
32 | batch STRING,
33 | user_id STRING,
34 | item_id STRING,
35 | category_id STRING,
36 | behavior STRING,
37 | ts timestamp(3),
38 | current_t timestamp(3)
39 | ) WITH (
40 | 'connector' = 'kafka'
41 | ,'topic' = 'user_log_sink_6'
42 | ,'properties.bootstrap.servers' = 'localhost:9092'
43 | ,'properties.group.id' = 'user_log'
44 | ,'scan.startup.mode' = 'latest-offset'
45 | ,'format' = 'json'
46 | ,'sink.semantic' = 'exactly-once'
47 | ,'properties.transaction.timeout.ms' = '900000'
48 | );
49 |
50 |
51 | -- streaming sql, insert into mysql table
52 | insert into user_log_sink
53 | SELECT event_time, partition_id, `offset`, '1',user_id, item_id, category_id, udf_date_add('2022-04-24 00:00:00', -1), ts, now()
54 | FROM user_log;
55 |
--------------------------------------------------------------------------------
/src/main/resources/sql/dev/kafka_to_mysql_partial_column_update.sql:
--------------------------------------------------------------------------------
1 | -- calc pv
2 | -- test multi column primary key update mysql
3 |
4 | -- kafka source
5 | drop table if exists user_log_1;
6 | CREATE TABLE user_log_1
7 | (
8 | user_id VARCHAR,
9 | item_id VARCHAR,
10 | category_id VARCHAR,
11 | behavior VARCHAR,
12 | ts TIMESTAMP(3),
13 | WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
14 | ) WITH (
15 | 'connector' = 'kafka'
16 | ,'topic' = 'user_log'
17 | ,'properties.bootstrap.servers' = 'localhost:9092'
18 | ,'properties.group.id' = 'user_log'
19 | ,'scan.startup.mode' = 'latest-offset'
20 | ,'format' = 'json'
21 | );
22 |
23 | -- kafka sink
24 | CREATE TABLE user_log_sink_1
25 | (
26 | user_id VARCHAR,
27 | item_id VARCHAR,
28 | category_id VARCHAR,
29 | primary key(user_id) NOT ENFORCED
30 | ) WITH (
31 | 'connector' = 'jdbc'
32 | ,'url' = 'jdbc:mysql://localhost:3306/venn'
33 | ,'table-name' = 'user_info_sink'
34 | ,'username' = 'root'
35 | ,'password' = '123456'
36 | );
37 |
38 | drop table if exists user_log_2;
39 | CREATE TABLE user_log_2
40 | (
41 | user_id VARCHAR,
42 | item_id VARCHAR,
43 | category_id VARCHAR,
44 | behavior VARCHAR,
45 | ts TIMESTAMP(3),
46 | WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
47 | ) WITH (
48 | 'connector' = 'kafka'
49 | ,'topic' = 'user_log_2'
50 | ,'properties.bootstrap.servers' = 'localhost:9092'
51 | ,'properties.group.id' = 'user_log'
52 | ,'scan.startup.mode' = 'latest-offset'
53 | ,'format' = 'json'
54 | );
55 | -- kafka sink
56 | CREATE TABLE user_log_sink_2
57 | (
58 | user_id VARCHAR,
59 | behavior VARCHAR,
60 | ts TIMESTAMP(3),
61 | primary key(user_id) NOT ENFORCED
62 | ) WITH (
63 | -- 'connector' = 'print'
64 | 'connector' = 'jdbc'
65 | ,'url' = 'jdbc:mysql://localhost:3306/venn?serverTimezone=GMT%2B8'
66 | ,'table-name' = 'user_info_sink'
67 | ,'username' = 'root'
68 | ,'password' = '123456'
69 | );
70 |
71 |
72 | -- streaming sql, insert into mysql table
73 | insert into user_log_sink_1
74 | SELECT user_id, item_id, category_id
75 | FROM user_log_1
76 | ;
77 |
78 | insert into user_log_sink_2
79 | select user_id, behavior, ts
80 | from user_log_2;
--------------------------------------------------------------------------------
/src/main/resources/sql/dev/kafka_to_print.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | drop table if exists user_log;
3 | CREATE TABLE user_log (
4 | user_id VARCHAR
5 | ,item_id VARCHAR
6 | ,category_id VARCHAR
7 | ,behavior VARCHAR
8 | ,price double
9 | ,proc_time as PROCTIME()
10 | ,ts TIMESTAMP(3)
11 | ,WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
12 | ) WITH (
13 | 'connector' = 'kafka'
14 | ,'topic' = 'user_log'
15 | ,'properties.bootstrap.servers' = 'localhost:9092'
16 | ,'properties.group.id' = 'user_log'
17 | ,'scan.startup.mode' = 'latest-offset'
18 | ,'format' = 'json'
19 | );
20 |
21 |
22 | -- set table.sql-dialect=hive;
23 | -- kafka sink
24 | -- drop table if exists user_log_sink;
25 | -- CREATE TABLE user_log_sink (
26 | -- user_id STRING
27 | -- ,item_id STRING
28 | -- ,category_id STRING
29 | -- ,behavior STRING
30 | -- ,proc_time timestamp(3)
31 | -- ,ts timestamp(3)
32 | -- ) WITH (
33 | -- 'connector' = 'print'
34 | -- );
35 | --
36 | --
37 | -- -- streaming sql, insert into mysql table
38 | -- insert into user_log_sink
39 | -- SELECT user_id, item_id, category_id, behavior, proc_time,ts
40 | -- from user_log
41 | -- ;
42 | create table user_log_sink(
43 | item_id string
44 | ,median_price double
45 | )WITH(
46 | 'connector' = 'print'
47 | );
48 |
49 | insert into user_log_sink
50 | select item_id, udaf_median(cast(price as double)) median_price
51 | from user_log
52 | group by item_id;
--------------------------------------------------------------------------------
/src/main/resources/sql/dev/kafka_to_print_arr.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | drop table if exists user_log;
3 | CREATE TABLE user_log (
4 | arr ARRAY
5 | ) WITH (
6 | 'connector' = 'kafka'
7 | ,'topic' = 'user_arr'
8 | ,'properties.bootstrap.servers' = 'localhost:9092'
9 | ,'properties.group.id' = 'user_arr'
10 | ,'scan.startup.mode' = 'latest-offset'
11 | ,'format' = 'json'
12 | );
13 |
14 |
15 | -- set table.sql-dialect=hive;
16 | -- kafka sink
17 | drop table if exists user_log_sink;
18 | CREATE TABLE user_log_sink (
19 | arr ARRAY
20 | , a string
21 | ,b string
22 | ) WITH (
23 | 'connector' = 'print'
24 | );
25 |
26 |
27 | -- streaming sql, insert into mysql table
28 | insert into user_log_sink
29 | SELECT arr, arr[1], arr[2]
30 | from user_log
31 | ;
--------------------------------------------------------------------------------
/src/main/resources/sql/dev/kafka_to_print_fluctuation.sql:
--------------------------------------------------------------------------------
1 | -- 计算窗口内,item 的价格 波动,是否联系
2 | -- kafka source
3 | drop table if exists user_log;
4 | CREATE TABLE user_log (
5 | user_id VARCHAR
6 | ,item_id VARCHAR
7 | ,category_id VARCHAR
8 | ,behavior VARCHAR
9 | ,price double
10 | ,proc_time as PROCTIME()
11 | ,ts TIMESTAMP(3)
12 | ,WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
13 | ) WITH (
14 | 'connector' = 'kafka'
15 | ,'topic' = 'user_log'
16 | ,'properties.bootstrap.servers' = 'localhost:9092'
17 | ,'properties.group.id' = 'user_log'
18 | ,'scan.startup.mode' = 'latest-offset'
19 | ,'format' = 'json'
20 | );
21 |
22 |
23 | -- set table.sql-dialect=hive;
24 | -- kafka sink
25 | drop table if exists user_log_sink;
26 | CREATE TABLE user_log_sink (
27 | user_id STRING
28 | ,item_id STRING
29 | ,category_id STRING
30 | ,behavior STRING
31 | ,proc_time timestamp(3)
32 | ,ts timestamp(3)
33 | ) WITH (
34 | 'connector' = 'print'
35 | );
36 |
37 |
38 | -- streaming sql, insert into mysql table
39 | insert into user_log_sink
40 | SELECT user_id, item_id, category_id, behavior, proc_time,ts
41 | from user_log
42 | ;
43 |
--------------------------------------------------------------------------------
/src/main/resources/sql/hudi/cdc_mysql_to_hudi.sql:
--------------------------------------------------------------------------------
1 | -- mysql cdc to print
2 | -- creates a mysql table source
3 | drop table if exists t_feature;
4 | CREATE TABLE t_feature (
5 | id varchar
6 | ,code VARCHAR
7 | ,send_time VARCHAR
8 | ,rms VARCHAR
9 | ,mean VARCHAR
10 | ,peak varchar
11 | ,kurtosis varchar
12 | ,skewness varchar
13 | ,proc_time as PROCTIME()
14 | ,PRIMARY KEY (id) NOT ENFORCED
15 | ) WITH (
16 | 'connector' = 'mysql-cdc'
17 | ,'hostname' = 'localhost'
18 | ,'port' = '3306'
19 | ,'username' = 'root'
20 | ,'password' = '123456'
21 | ,'database-name' = 'test_db'
22 | ,'table-name' = 't_feature'
23 | ,'server-id' = '1'
24 | );
25 |
26 | -- kafka sink
27 | drop table if exists t_feature_sink;
28 | CREATE TABLE t_feature_sink (
29 | id varchar
30 | ,code VARCHAR
31 | ,send_time VARCHAR
32 | ,rms VARCHAR
33 | ,mean VARCHAR
34 | ,peak varchar
35 | ,kurtosis varchar
36 | ,skewness varchar
37 | ,ts timestamp(3)
38 | ) WITH (
39 | 'connector' = 'hudi'
40 | ,'path' = 'hdfs://thinkpad:8020/tmp/hudi/t_feature'
41 | ,'hoodie.datasource.write.recordkey.field' = 'id'
42 | ,'write.precombine.field' = 'ts'
43 | ,'write.tasks' = '1'
44 | ,'table.type' = 'MERGE_ON_READ'
45 | ,'compaction.tasks' = '1'
46 | ,'compaction.trigger.strategy' = 'num_or_time'
47 | ,'compaction.delta_commits' = '100'
48 | ,'compaction.delta_seconds' = '6000'
49 | -- ,'read.tasks' = '1'
50 | -- ,'read.streaming.enabled' = 'true'
51 | -- ,'hoodie.datasource.query.type' = 'snapshot'
52 | -- ,'read.streaming.start-commit' = '20210101000000'
53 | -- ,'read.streaming.check-interval' = '1'
54 | -- ,'hoodie.datasource.merge.type' = 'payload_combine'
55 | );
56 |
57 | -- sink to kafka
58 | insert into t_feature_sink
59 | select id, code, send_time, rms, mean, peak, kurtosis, skewness, proc_time
60 | from t_feature;
61 |
--------------------------------------------------------------------------------
/src/main/resources/sql/hudi/hudi_demo.sql:
--------------------------------------------------------------------------------
1 | -- test sync hudi metadata to hive metastore
2 | create table if not exists kafka_ods_user_info (
3 | id int
4 | ,name string
5 | ,sex string
6 | ,age int
7 | ,birthday string
8 | ) with (
9 | 'connector' = 'kafka'
10 | ,'topic' = 'datalake_test_topic_1'
11 | ,'properties.bootstrap.servers' = 'localhost:9092'
12 | ,'properties.group.id' = 'testGroup'
13 | ,'scan.startup.mode' = 'latest-offset'
14 | ,'format' = 'csv'
15 | );
16 |
17 | drop table ods_user_info_15;
18 |
19 | create table if not exists ods_user_info_15(
20 | dl_uuid string
21 | ,id int
22 | ,name string
23 | ,sex string
24 | ,age int
25 | ,birthday string
26 | ,`etl_create_time` TIMESTAMP(3) COMMENT 'ETL创建时间'
27 | ,`etl_update_time` TIMESTAMP(3) COMMENT 'ETL更新时间'
28 | ,`partition` string
29 | ) with (
30 | 'connector' = 'hudi'
31 | ,'path' = 'hdfs://thinkpad:8020/user/hive/warehouse/dl_ods.db/ods_user_info_15'
32 | ,'hoodie.datasource.write.recordkey.field' = 'dl_uuid'
33 | ,'hoodie.datasource.write.partitionpath.field' = 'partition'
34 | ,'write.precombine.field' = 'etl_update_time'
35 | ,'write.tasks' = '1'
36 | ,'table.type' = 'MERGE_ON_READ'
37 | ,'compaction.tasks' = '1'
38 | ,'compaction.trigger.strategy' = 'num_or_time'
39 | ,'compaction.delta_commits' = '100'
40 | ,'compaction.delta_seconds' = '6000'
41 | ,'hive_sync.enable' = 'true'
42 | ,'hive_sync.db' = 'dl_ods'
43 | ,'hive_sync.table' = 'ods_user_info_15'
44 | ,'hive_sync.file_format' = 'PARQUET'
45 | ,'hive_sync.support_timestamp' = 'true'
46 | ,'hive_sync.use_jdbc' = 'true'
47 | ,'hive_sync.jdbc_url' = 'jdbc:hive2://thinkpad:10000'
48 | ,'hive_sync.metastore.uris' = 'thrift://thinkpad:9083'
49 | -- ,'hoodie.datasource.hive_style_partition' = 'true' -- already remove
50 | ,'hive_sync.partition_fields' = 'partition'
51 | ,'read.tasks' = '1'
52 | ,'read.streaming.enabled' = 'true'
53 | ,'hoodie.datasource.query.type' = 'snapshot'
54 | ,'read.streaming.start-commit' = '20210101000000'
55 | ,'read.streaming.check-interval' = '1'
56 | ,'hoodie.datasource.merge.type' = 'payload_combine'
57 | ,'read.utc-timezone' = 'false'
58 | -- ,'hoodie.memory.spillable.map.path' = '/tmp/hudi'
59 | );
60 |
61 | -- set table.dynamic-table-options.enabled=true;
62 | -- set 'pipeline.name' = 'insert_ods_user_info';
63 | insert into ods_user_info_15
64 | select /*+ OPTIONS('pipeline.name'='insert_ods_user_info') */ -- work on flink 1.13
65 | cast(id as string) dl_uuid
66 | ,id
67 | ,name
68 | ,sex
69 | ,age
70 | ,birthday
71 | ,now() etl_create_time
72 | ,now() etl_update_time
73 | ,date_format(now(), 'yyyy/MM/dd') -- only support partition format
74 | from kafka_ods_user_info;
75 |
--------------------------------------------------------------------------------
/src/main/resources/sql/iceberg/README.md:
--------------------------------------------------------------------------------
1 | # iceberg sql
2 |
3 | * the sql folder for iceberg 0.12.0
4 | * submit sql by flink sql client
--------------------------------------------------------------------------------
/src/main/resources/sql/iceberg/kafka_to_iceberg_demo.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | drop table if exists user_log;
3 | CREATE TABLE user_log (
4 | user_id VARCHAR
5 | ,item_id VARCHAR
6 | ,category_id VARCHAR
7 | ,behavior VARCHAR
8 | ,ts TIMESTAMP(3)
9 | ,WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
10 | ) WITH (
11 | 'connector' = 'kafka'
12 | ,'topic' = 'user_log'
13 | ,'properties.bootstrap.servers' = 'localhost:9092'
14 | ,'properties.group.id' = 'user_log'
15 | ,'scan.startup.mode' = 'latest-offset'
16 | ,'format' = 'json'
17 | );
18 |
19 | -- set table.sql-dialect=hive;
20 | -- kafka sink
21 | -- create catalog
22 | CREATE CATALOG ice WITH (
23 | 'type'='iceberg',
24 | 'catalog-type'='hive',
25 | 'uri'='thrift://thinkpad:9083',
26 | 'clients'='5',
27 | 'property-version'='2',
28 | 'warehouse'='hdfs://thinkpad:8020/user/hive/datalake/ice'
29 | );
30 | -- use catalog
31 | use catalog ice;
32 | -- create database
33 | create database ice;
34 | -- use database;
35 |
36 | CREATE TABLE ice.ice.user_log_sink (
37 | user_id STRING
38 | ,item_id STRING
39 | ,category_id STRING
40 | ,behavior STRING
41 | ,ts timestamp(3)
42 | );
43 |
44 |
45 | -- streaming sql, insert into mysql table
46 | insert into ice.ice.user_log_sink
47 | SELECT user_id, item_id, category_id, behavior, ts
48 | FROM user_log;
49 |
50 |
51 | -- read
52 | SET table.dynamic-table-options.enabled=true;
53 | SELECT * FROM ice.ice.user_log_sink /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/ ;
54 |
--------------------------------------------------------------------------------
/src/main/resources/sql/iceberg/kafka_to_iceberg_upsert.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | drop table if exists user_log;
3 | CREATE TABLE user_log (
4 | user_id VARCHAR
5 | ,item_id VARCHAR
6 | ,category_id VARCHAR
7 | ,behavior VARCHAR
8 | ,ts TIMESTAMP(3)
9 | ,WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
10 | ) WITH (
11 | 'connector' = 'kafka'
12 | ,'topic' = 'user_log'
13 | ,'properties.bootstrap.servers' = 'localhost:9092'
14 | ,'properties.group.id' = 'user_log'
15 | ,'scan.startup.mode' = 'latest-offset'
16 | ,'format' = 'json'
17 | );
18 |
19 | -- set table.sql-dialect=hive;
20 | -- kafka sink
21 | -- create catalog
22 | CREATE CATALOG ice WITH (
23 | 'type'='iceberg',
24 | 'catalog-type'='hive',
25 | 'uri'='thrift://thinkpad:9083',
26 | 'clients'='5',
27 | 'property-version'='2',
28 | 'warehouse'='hdfs://thinkpad:8020/user/hive/datalake/ice'
29 | );
30 | -- use catalog
31 | use catalog ice;
32 | -- create database
33 | create database ice;
34 | -- use database;
35 |
36 | CREATE TABLE ice.ice.user_log_sink (
37 | user_id STRING
38 | ,item_id STRING
39 | ,category_id STRING
40 | ,behavior STRING
41 | ,ts timestamp(3)
42 | ,PRIMARY KEY (user_id) NOT ENFORCED
43 | )WITH (
44 | 'format-version' = '2'
45 | ,'write.upsert.enabled' = 'true'
46 | );
47 |
48 |
49 | -- streaming sql, insert into mysql table
50 | insert into ice.ice.user_log_sink
51 | SELECT user_id, item_id, category_id, behavior, ts
52 | FROM user_log;
53 |
54 |
55 | -- read
56 | SET table.dynamic-table-options.enabled=true;
57 |
58 | SET execution.runtime-mode = streaming ;
59 | SELECT * FROM ice.ice.user_log_sink /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/ ;
60 |
61 | SET execution.runtime-mode = batch ;
62 | SELECT * FROM ice.ice.user_log_sink;
63 |
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/count_distinct.sql:
--------------------------------------------------------------------------------
1 | -- count 每天输入数据量达到 2000 条,输出一条数据
2 | -- kafka source
3 | CREATE TABLE user_log (
4 | user_id STRING
5 | ,item_id STRING
6 | ,category_id STRING
7 | ,behavior STRING
8 | ,ts TIMESTAMP(3)
9 | ,process_time as proctime()
10 | , WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_log'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log'
16 | ,'scan.startup.mode' = 'latest-offset'
17 | ,'format' = 'json'
18 | );
19 |
20 | CREATE TABLE user_log_sink (
21 | `day` string
22 | ,num bigint
23 | ,min_user_id bigint
24 | ,max_user_id bigint
25 | ,decode_x string
26 | ) WITH (
27 | 'connector' = 'print'
28 | );
29 |
30 | -- count : flink will not cache history data
31 | -- count : distinct user_id, i think flink will cache history
32 | insert into user_log_sink
33 | select `day`
34 | , num
35 | , min_user_id, max_user_id, decode_x
36 | from(
37 | select DATE_FORMAT(ts,'yyyyMMdd') `day`
38 | ,count(distinct user_id) num
39 | ,min(cast(replace(user_id,'xxxxxxxxxxxxx','') as bigint)) min_user_id
40 | ,max(cast(replace(user_id,'xxxxxxxxxxxxx','') as bigint)) max_user_id
41 | ,udf_decode(DATE_FORMAT(ts,'yyyyMMdd'), '1', '101', '102') decode_x
42 | from user_log
43 | -- where DATE_FORMAT(ts,'yyyyMMdd') = date_format(current_timestamp, 'yyyyMMdd')
44 | group by DATE_FORMAT(ts,'yyyyMMdd')
45 | )t1
46 | where num % 2 = 0
47 | ;
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/deduplication.sql:
--------------------------------------------------------------------------------
1 | -- 去重查询
2 | -- kafka source
3 | CREATE TABLE user_log (
4 | user_id VARCHAR
5 | ,item_id VARCHAR
6 | ,category_id VARCHAR
7 | ,behavior INT
8 | ,ts TIMESTAMP(3)
9 | ,process_time as proctime()
10 | , WATERMARK FOR ts AS ts
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_behavior'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log'
16 | ,'scan.startup.mode' = 'group-offsets'
17 | ,'format' = 'json'
18 | );
19 |
20 | ---sink table
21 | CREATE TABLE user_log_sink (
22 | user_id VARCHAR
23 | ,item_id VARCHAR
24 | ,category_id VARCHAR
25 | ,behavior INT
26 | ,ts TIMESTAMP(3)
27 | ,num BIGINT
28 | ,primary key (user_id) not enforced
29 | ) WITH (
30 | 'connector' = 'upsert-kafka'
31 | ,'topic' = 'user_behavior_sink'
32 | ,'properties.bootstrap.servers' = 'localhost:9092'
33 | ,'properties.group.id' = 'user_log'
34 | ,'key.format' = 'json'
35 | ,'key.json.ignore-parse-errors' = 'true'
36 | ,'value.format' = 'json'
37 | ,'value.json.fail-on-missing-field' = 'false'
38 | ,'value.fields-include' = 'ALL'
39 | );
40 |
41 | -- insert
42 | insert into user_log_sink(user_id, item_id, category_id,behavior,ts,num)
43 | SELECT user_id, item_id, category_id,behavior,ts,rownum
44 | FROM (
45 | SELECT user_id, item_id, category_id,behavior,ts,
46 | ROW_NUMBER() OVER (PARTITION BY category_id ORDER BY ts desc) AS rownum -- desc use the latest one,
47 | FROM user_log)
48 | WHERE rownum=1
49 | -- 只能使用 rownum=1,如果写 rownum=2(或<10) 会识别为 top n
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/history_pv_uv/pu_uv_1.sql:
--------------------------------------------------------------------------------
1 | -- flink cumulate window tvf calc pv&uv, only current day data
2 | drop table if exists user_log;
3 | CREATE TABLE user_log
4 | (
5 | user_id VARCHAR
6 | ,item_id VARCHAR
7 | ,category_id VARCHAR
8 | ,behavior VARCHAR
9 | ,ts TIMESTAMP(3)
10 | ,WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_log'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log'
16 | ,'scan.startup.mode' = 'latest-offset'
17 | ,'format' = 'json'
18 | );
19 |
20 | create table if not exists user_log_sink(
21 | cal_day varchar
22 | ,behavior varchar
23 | ,start_time VARCHAR
24 | ,end_time VARCHAR
25 | ,pv bigint
26 | ,uv bigint
27 | ,PRIMARY KEY (cal_day, behavior) NOT ENFORCED
28 | ) with (
29 | 'connector' = 'jdbc'
30 | ,'url' = 'jdbc:mysql://venn:3306/venn'
31 | ,'table-name' = 'pv_uv'
32 | ,'username' = 'root'
33 | ,'password' = '123456'
34 | );
35 |
36 | insert into user_log_sink
37 | select
38 | date_format(window_start, 'yyyy-MM-dd') cal_day
39 | ,behavior
40 | ,date_format(window_start, 'HH:mm:ss') start_time
41 | , date_format(window_end, 'HH:mm:ss') end_time
42 | , count(user_id) pv
43 | , count(distinct user_id) uv
44 | FROM TABLE(
45 | CUMULATE(TABLE user_log, DESCRIPTOR(ts), INTERVAL '10' SECOND, INTERVAL '1' DAY))
46 | GROUP BY window_start, window_end, behavior
47 | ;
48 |
49 | --- 接口查询需要查询对应行为全部时间的结果求和, uv 暂时不可用
50 | -- select behavior,sum(pv)
51 | -- from pv_uv
52 | -- group by behavior
53 |
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/history_pv_uv/pu_uv_2.sql:
--------------------------------------------------------------------------------
1 | -- flink cumulate window tvf calc pv&uv, only current day data + history
2 | -- udf: udf_date_add_new like hive date_add
3 | drop table if exists user_log;
4 | CREATE TABLE user_log
5 | (
6 | user_id VARCHAR
7 | ,item_id VARCHAR
8 | ,category_id VARCHAR
9 | ,behavior VARCHAR
10 | ,ts TIMESTAMP(3)
11 | ,proc_time as PROCTIME()
12 | ,WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
13 | ) WITH (
14 | 'connector' = 'kafka'
15 | ,'topic' = 'user_log'
16 | ,'properties.bootstrap.servers' = 'localhost:9092'
17 | ,'properties.group.id' = 'user_log'
18 | ,'scan.startup.mode' = 'latest-offset'
19 | ,'format' = 'json'
20 | );
21 |
22 | create table if not exists user_log_sink(
23 | cal_day varchar
24 | ,behavior varchar
25 | ,start_time VARCHAR
26 | ,end_time VARCHAR
27 | ,pv bigint
28 | ,uv bigint
29 | ,last_pv bigint
30 | ,last_uv bigint
31 | ,PRIMARY KEY (cal_day, behavior) NOT ENFORCED
32 | ) with (
33 | 'connector' = 'jdbc'
34 | ,'url' = 'jdbc:mysql://venn:3306/venn'
35 | ,'table-name' = 'pv_uv'
36 | ,'username' = 'root'
37 | ,'password' = '123456'
38 | );
39 |
40 | create table if not exists user_log_lookup_join(
41 | cal_day varchar
42 | ,behavior varchar
43 | ,pv bigint
44 | ,uv bigint
45 | ,PRIMARY KEY (cal_day, behavior) NOT ENFORCED
46 | ) with (
47 | 'connector' = 'jdbc'
48 | ,'url' = 'jdbc:mysql://localhost:3306/venn'
49 | ,'table-name' = 'pv_uv'
50 | ,'username' = 'root'
51 | ,'password' = '123456'
52 | ,'scan.partition.column' = 'cal_day'
53 | ,'scan.partition.num' = '1'
54 | ,'scan.partition.lower-bound' = '0'
55 | ,'scan.partition.upper-bound' = '9999'
56 | ,'lookup.cache.max-rows' = '1000'
57 | -- one day, once cache, the value will not update
58 | ,'lookup.cache.ttl' = '86400000' -- ttl time 超过这么长时间无数据才行
59 | );
60 |
61 | insert into user_log_sink
62 | select
63 | a.cal_day
64 | ,a.behavior
65 | ,'' start_time
66 | ,date_format(a.proc_time, 'yyyy-MM-dd HH:mm:ss')
67 | ,a.pv + COALESCE(c.pv,0) -- add last
68 | ,a.uv
69 | ,c.pv last_uv
70 | ,c.uv last_uv
71 | from(
72 | select
73 | date_format(window_start, 'yyyy-MM-dd') cal_day
74 | ,behavior
75 | ,max(proc_time) proc_time
76 | ,count(user_id) pv
77 | ,count(distinct user_id) uv
78 | FROM TABLE(
79 | CUMULATE(TABLE user_log, DESCRIPTOR(ts), INTERVAL '10' SECOND, INTERVAL '1' DAY))
80 | GROUP BY window_start, window_end, behavior
81 | )a
82 | left join user_log_lookup_join FOR SYSTEM_TIME AS OF a.proc_time AS c
83 | ON a.behavior = c.behavior
84 | and udf_date_add(date_format(a.proc_time, 'yyyy-MM-dd HH:mm:ss'), -1) = c.cal_day
85 | ;
86 |
87 | -- 直接 lookup join 加上昨天的 pv、uv
88 | -- select behavior,pv
89 | -- from pv_uv
90 | -- group by behavior
91 |
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/history_pv_uv/pu_uv_5.sql:
--------------------------------------------------------------------------------
1 | -- flink cumulate window tvf calc pv&uv, only current day data + history, uv
2 | -- bloom filter cal uv
3 | -- redis udf
4 | drop table if exists user_log;
5 | CREATE TABLE user_log
6 | (
7 | user_id VARCHAR
8 | ,item_id VARCHAR
9 | ,category_id VARCHAR
10 | ,behavior VARCHAR
11 | ,ts TIMESTAMP(3)
12 | ,proc_time as PROCTIME()
13 | ,WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
14 | ) WITH (
15 | 'connector' = 'kafka'
16 | ,'topic' = 'user_log'
17 | ,'properties.bootstrap.servers' = 'localhost:9092'
18 | ,'properties.group.id' = 'user_log'
19 | ,'scan.startup.mode' = 'latest-offset'
20 | ,'format' = 'json'
21 | );
22 |
23 | create table if not exists user_log_sink(
24 | cal_day varchar
25 | ,behavior varchar
26 | ,start_time VARCHAR
27 | ,end_time VARCHAR
28 | ,pv bigint
29 | ,uv bigint
30 | ,last_pv bigint
31 | ,last_uv bigint
32 | ,PRIMARY KEY (cal_day, behavior) NOT ENFORCED
33 | ) with (
34 | 'connector' = 'print'
35 | -- 'connector' = 'jdbc'
36 | -- ,'url' = 'jdbc:mysql://venn:3306/venn'
37 | -- ,'table-name' = 'pv_uv'
38 | -- ,'username' = 'root'
39 | -- ,'password' = '123456'
40 | );
41 |
42 | create table if not exists user_log_lookup_join(
43 | cal_day varchar
44 | ,behavior varchar
45 | ,pv bigint
46 | ,uv bigint
47 | ,PRIMARY KEY (cal_day, behavior) NOT ENFORCED
48 | ) with (
49 | 'connector' = 'jdbc'
50 | ,'url' = 'jdbc:mysql://localhost:3306/venn'
51 | ,'table-name' = 'pv_uv'
52 | ,'username' = 'root'
53 | ,'password' = '123456'
54 | ,'scan.partition.column' = 'cal_day'
55 | ,'scan.partition.num' = '1'
56 | ,'scan.partition.lower-bound' = '0'
57 | ,'scan.partition.upper-bound' = '9999'
58 | ,'lookup.cache.max-rows' = '1000'
59 | -- one day, once cache, the value will not update
60 | ,'lookup.cache.ttl' = '86400000' -- ttl time 超过这么长时间无数据才行
61 | );
62 |
63 | insert into user_log_sink
64 | select
65 | a.cal_day
66 | ,a.behavior
67 | ,'' start_time
68 | ,date_format(a.ts, 'yyyy-MM-dd HH:mm:ss')
69 | ,a.pv + COALESCE(c.pv,0) -- add last
70 | ,a.uv + COALESCE(c.uv,0)
71 | ,c.pv last_uv
72 | ,c.uv last_uv
73 | from(
74 | select
75 | date_format(window_start, 'yyyy-MM-dd') cal_day
76 | ,behavior
77 | ,max(ts) ts
78 | ,max(proc_time) proc_time
79 | ,count(user_id) pv
80 | ,udaf_redis_uv_count('user_log_uv', user_id) uv
81 | FROM TABLE(
82 | CUMULATE(TABLE user_log, DESCRIPTOR(ts), INTERVAL '10' minute, INTERVAL '1' day))
83 | GROUP BY window_start, window_end, behavior
84 | )a
85 | left join user_log_lookup_join FOR SYSTEM_TIME AS OF a.proc_time AS c
86 | ON a.behavior = c.behavior
87 | and udf_date_add(date_format(a.proc_time, 'yyyy-MM-dd HH:mm:ss'), -1) = c.cal_day
88 | ;
89 |
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/kafka_join_agg.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | drop table if exists user_log;
3 | CREATE TABLE user_log (
4 | user_id VARCHAR
5 | ,item_id VARCHAR
6 | ,category_id VARCHAR
7 | ,behavior VARCHAR
8 | ,ts TIMESTAMP(3)
9 | ,WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
10 | ) WITH (
11 | 'connector' = 'kafka'
12 | ,'topic' = 'user_log'
13 | ,'properties.bootstrap.servers' = 'localhost:9092'
14 | ,'properties.group.id' = 'user_log'
15 | ,'scan.startup.mode' = 'latest-offset'
16 | ,'format' = 'json'
17 | );
18 |
19 |
20 | drop table if exists user_log_2;
21 | CREATE TABLE user_log_2 (
22 | user_id VARCHAR
23 | ,page as uuid()
24 | ,ts TIMESTAMP(3)
25 | ,WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
26 | ) WITH (
27 | 'connector' = 'kafka'
28 | ,'topic' = 'user_log'
29 | ,'properties.bootstrap.servers' = 'localhost:9092'
30 | ,'properties.group.id' = 'user_log'
31 | ,'scan.startup.mode' = 'latest-offset'
32 | ,'format' = 'json'
33 | );
34 |
35 |
36 |
37 | -- set table.sql-dialect=hive;
38 | -- kafka sink
39 | drop table if exists user_log_sink;
40 | CREATE TABLE user_log_sink (
41 | user_id STRING
42 | ,item_id STRING
43 | ,category_id STRING
44 | ,behavior STRING
45 | ,page STRING
46 | ,ts timestamp(3)
47 | ) WITH (
48 | 'connector' = 'print'
49 | );
50 |
51 |
52 | -- 5 s, interval join
53 | -- insert into user_log_sink
54 | -- SELECT a.user_id, a.item_id, a.category_id, a.behavior,b.page, a.ts
55 | -- FROM user_log a
56 | -- INNER JOIN user_log_2 b ON a.user_id = b.user_id
57 | -- and a.ts BETWEEN b.ts - INTERVAL '5' SECOND AND b.ts + INTERVAL '5' SECOND
58 | -- where a.user_id is not null
59 | -- ;
60 |
61 | -- join an agg
62 | CREATE TABLE user_log_sink_2 (
63 | behavior STRING
64 | ,pv bigint
65 | ,uv bigint
66 | ,uv_cate bigint
67 | ,max_page STRING
68 | ,max_ts timestamp(3)
69 | ) WITH (
70 | 'connector' = 'print'
71 | );
72 |
73 | insert into user_log_sink_2
74 | SELECT a.behavior, count(a.user_id) pv, count(distinct a.user_id) uv, count(distinct category_id) uv_cate, uuid() max_page
75 | , max(a.ts) max_ts
76 | FROM user_log a
77 | INNER JOIN user_log_2 b ON a.user_id = b.user_id
78 | and a.ts BETWEEN b.ts - INTERVAL '5' SECOND AND b.ts + INTERVAL '5' SECOND
79 | where a.user_id is not null
80 | group by a.behavior
81 | ;
82 |
83 |
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/kafka_to_print_udtf_timer.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | drop table if exists user_log;
3 | CREATE TABLE user_log (
4 | user_id VARCHAR
5 | ,item_id VARCHAR
6 | ,category_id VARCHAR
7 | ,behavior VARCHAR
8 | ,ts TIMESTAMP(3)
9 | ,WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
10 | ) WITH (
11 | 'connector' = 'kafka'
12 | ,'topic' = 'user_log'
13 | ,'properties.bootstrap.servers' = 'localhost:9092'
14 | ,'properties.group.id' = 'user_log'
15 | ,'scan.startup.mode' = 'latest-offset'
16 | ,'format' = 'json'
17 | );
18 |
19 |
20 | -- set table.sql-dialect=hive;
21 | -- kafka sink
22 | drop table if exists user_log_sink;
23 | CREATE TABLE user_log_sink (
24 | user_id STRING
25 | ,item_id STRING
26 | ,category_id STRING
27 | ,behavior STRING
28 | ,ts timestamp(3)
29 | ) WITH (
30 | 'connector' = 'print'
31 | );
32 |
33 |
34 | -- streaming sql, insert into mysql table
35 | insert into user_log_sink
36 | SELECT a.user_id, item_id, category_id, cast( t.`size` as string) sizee, ts
37 | FROM user_log a
38 | LEFT JOIN LATERAL TABLE(udf_timer(a.user_id)) AS t(user_id, `size`) ON TRUE ;
39 |
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/kafka_to_window_test.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | CREATE TABLE user_log
3 | (
4 | user_id VARCHAR,
5 | item_id VARCHAR,
6 | category_id VARCHAR,
7 | behavior VARCHAR,
8 | ts TIMESTAMP(3),
9 | WATERMARK FOR ts AS ts - INTERVAL '1' MINUTES
10 | ) WITH (
11 | 'connector' = 'kafka'
12 | ,'topic' = 'user_log'
13 | ,'properties.bootstrap.servers' = 'localhost:9092'
14 | ,'properties.group.id' = 'user_log'
15 | ,'scan.startup.mode' = 'latest-offset'
16 | ,'format' = 'json'
17 | );
18 |
19 | -- set table.sql-dialect=hive;
20 | -- kafka sink
21 | CREATE TABLE user_log_sink
22 | (
23 | start_time timestamp(3),
24 | end_time timestamp(3),
25 | coun bigint
26 | -- coun MULTISET
27 | ) WITH (
28 | 'connector' = 'print'
29 | );
30 |
31 |
32 | -- streaming sql, insert into mysql table
33 | -- insert into user_log_sink
34 | -- select window_start, window_end, count(user_id)
35 | -- from TABLE(
36 | -- TUMBLE(TABLE user_log, DESCRIPTOR(ts), INTERVAL '10' MINUTES))
37 | -- group by window_start, window_end;
38 |
39 | -- insert into user_log_sink
40 | -- select window_start, window_end
41 | -- ,count(user_id)
42 | -- -- , COLLECT(ts)
43 | -- from TABLE(
44 | -- HOP(TABLE user_log, DESCRIPTOR(ts), INTERVAL '3' MINUTES ,INTERVAL '10' MINUTES ))
45 | -- group by window_start, window_end;
46 |
47 | insert into user_log_sink
48 | select window_start, window_end, count(user_id)
49 | from TABLE(
50 | CUMULATE(TABLE user_log, DESCRIPTOR(ts), INTERVAL '2' MINUTES ,INTERVAL '10' MINUTES ))
51 | group by window_start, window_end;
52 |
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/multe_insert.sql:
--------------------------------------------------------------------------------
1 | -- parse special json, then process window
2 | CREATE TABLE t_feature (
3 | data string
4 | ) WITH (
5 | 'connector' = 'kafka'
6 | ,'topic' = 'test_dd'
7 | ,'properties.bootstrap.servers' = 'localhost:9092'
8 | ,'properties.group.id' = 'user_log'
9 | ,'scan.startup.mode' = 'group-offsets'
10 | ,'format' = 'csv'
11 | );
12 |
13 | CREATE TABLE t_sink_1 (
14 | data STRING
15 | ) WITH (
16 | 'connector' = 'print'
17 | );
18 |
19 | CREATE TABLE t_sink_2 (
20 | data STRING
21 | ) WITH (
22 | 'connector' = 'print'
23 | );
24 | insert into t_sink_1
25 | select concat(data, '_1') from t_feature;
26 |
27 | insert into t_sink_2
28 | select concat(data, '_2') from t_feature;
29 |
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/topn.sql:
--------------------------------------------------------------------------------
1 | -- Top N 查询
2 | -- kafka source
3 | CREATE TABLE user_log (
4 | user_id VARCHAR
5 | ,item_id VARCHAR
6 | ,category_id VARCHAR
7 | ,behavior INT
8 | ,sales DOUBLE
9 | ,sort_col int
10 | ,ts TIMESTAMP(3)
11 | ,process_time as proctime()
12 | , WATERMARK FOR ts AS ts
13 | ) WITH (
14 | 'connector' = 'kafka'
15 | ,'topic' = 'user_behavior'
16 | ,'properties.bootstrap.servers' = 'localhost:9092'
17 | ,'properties.group.id' = 'user_log'
18 | ,'scan.startup.mode' = 'group-offsets'
19 | ,'format' = 'json'
20 | );
21 |
22 | ---sink table
23 | CREATE TABLE user_log_sink (
24 | user_id VARCHAR
25 | ,item_id VARCHAR
26 | ,category_id VARCHAR
27 | ,behavior INT
28 | ,sales DOUBLE
29 | ,sort_col INT
30 | ,ts TIMESTAMP(3)
31 | ,num bigint
32 | ,primary key (user_id) not enforced
33 | ) WITH (
34 | 'connector' = 'upsert-kafka'
35 | ,'topic' = 'user_behavior_sink'
36 | ,'properties.bootstrap.servers' = 'localhost:9092'
37 | ,'properties.group.id' = 'user_log'
38 | ,'key.format' = 'json'
39 | ,'key.json.ignore-parse-errors' = 'true'
40 | ,'value.format' = 'json'
41 | ,'value.json.fail-on-missing-field' = 'false'
42 | ,'value.fields-include' = 'ALL'
43 | );
44 |
45 | -- insert
46 | insert into user_log_sink(user_id, item_id, category_id,behavior,sales,ts,sort_col,rownum)
47 | SELECT user_id, item_id, category_id,behavior,sales,sort_col,ts,rownum
48 | FROM (
49 | SELECT user_id, item_id, category_id,behavior,sales, ts, sort_col,
50 | ROW_NUMBER() OVER (PARTITION BY category_id ORDER BY ts desc) AS rownum
51 | FROM user_log)
52 | -- WHERE rownum < sort_col
53 | WHERE rownum < 6
54 | -- 只支持两种 top n:
55 | -- rownum < 10 or rownum > 3 and rownum < 10
56 | -- rownum < source_table.column
57 | -- rownum > 3 是不支持的 Rank end is not specified. Currently rank only support TopN, which means the rank end must be specified.
58 | -- 不输出 rownum 可以启动无排名优化,仅输出当前数据,对历史数据的排名更新,不再输出
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/tps/kafka_lookup_join_redis_tps.sql:
--------------------------------------------------------------------------------
1 | -- Lookup Source: Sync Mode
2 | -- kafka source
3 | CREATE TABLE user_log (
4 | user_id STRING
5 | ,item_id STRING
6 | ,category_id STRING
7 | ,behavior STRING
8 | ,ts TIMESTAMP(3)
9 | ,process_time as proctime()
10 | , WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_log'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log'
16 | ,'scan.startup.mode' = 'latest-offset'
17 | ,'format' = 'json'
18 | );
19 |
20 | CREATE TEMPORARY TABLE redis_table (
21 | `key` STRING
22 | ,filed STRING
23 | ,`value` STRING
24 | ) WITH (
25 | 'connector' = 'cust-redis'
26 | ,'redis.url' = 'redis://localhost:6379?timeout=3000'
27 | ,'lookup.cache.max.size' = '28'
28 | ,'lookup.cache.expire.ms' = '3600000' -- ttl time 超过这么长时间无数据才行
29 | -- ,'pass' = '11' -- todo test
30 | );
31 |
32 | ---sinkTable
33 | CREATE TABLE kakfa_join_redis_sink (
34 | user_id STRING
35 | ,item_id STRING
36 | ,category_id STRING
37 | ,behavior STRING
38 | ,behavior_map STRING
39 | ,ts TIMESTAMP(3)
40 | ,primary key (user_id) not enforced
41 | ) WITH (
42 | 'connector' = 'print'
43 | );
44 | -- sting/list/set/zset test sql
45 | -- INSERT INTO kakfa_join_redis_sink(user_id, item_id, category_id, behavior, behavior_map, ts)
46 | -- SELECT a.user_id, a.item_id, a.category_id, a.behavior, b.`value`, a.ts
47 | -- FROM user_log a
48 | -- left join redis_table FOR SYSTEM_TIME AS OF a.process_time AS b
49 | -- ON a.behavior = b.`key`
50 | -- where a.behavior is not null;
51 |
52 | CREATE TABLE kakfa_join_redis_sink_1 (
53 | user_id STRING
54 | ,item_id STRING
55 | ,category_id STRING
56 | ,behavior STRING
57 | ,behavior_key STRING
58 | ,behavior_map STRING
59 | ,ts TIMESTAMP(3)
60 | ,primary key (user_id) not enforced
61 | ) WITH (
62 | 'connector' = 'print'
63 | )
64 | ;
65 |
66 |
67 | -- hash multiple input
68 | INSERT INTO kakfa_join_redis_sink_1(user_id, item_id, category_id, behavior, behavior_key,behavior_map, ts)
69 | SELECT a.user_id, a.item_id, a.category_id, a.behavior,b.filed, b.`value`, a.ts
70 | FROM user_log a
71 | left join redis_table FOR SYSTEM_TIME AS OF a.process_time AS b
72 | ON a.behavior = b.key
73 | where a.behavior is not null;
74 |
75 | -- INSERT INTO kakfa_join_redis_sink_1(user_id, item_id, category_id, behavior, behavior_key,behavior_map, ts)
76 | -- SELECT a.user_id, a.item_id, a.category_id, a.behavior,b.filed, b.`value`, a.ts
77 | -- FROM user_log a
78 | -- left join redis_table FOR SYSTEM_TIME AS OF a.process_time AS b
79 | -- ON a.behavior = b.key and a.item = b.filed
80 | -- where a.behavior is not null;
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/tps/kafka_to_hbase.sql:
--------------------------------------------------------------------------------
1 | -- Lookup Source: Sync Mode
2 | -- kafka source
3 | CREATE TABLE user_info
4 | (
5 | user_id STRING,
6 | sex STRING,
7 | age INTEGER,
8 | degree STRING,
9 | address STRING,
10 | work_address STRING,
11 | income_range STRING,
12 | default_shipping_address STRING,
13 | register_date TIMESTAMP(3),
14 | udpate_date TIMESTAMP(3)
15 | ) WITH (
16 | 'connector' = 'kafka'
17 | ,'topic' = 'user_info'
18 | ,'properties.bootstrap.servers' = 'dcmp10:9092,dcmp11:9092,dcmp12:9092'
19 | ,'properties.group.id' = 'user_info'
20 | ,'scan.startup.mode' = 'latest-offset'
21 | ,'format' = 'json'
22 | );
23 |
24 | drop table if exists hbase_user_info_sink;
25 | CREATE TABLE hbase_user_info_sink
26 | (
27 | user_id STRING,
28 | f ROW(sex STRING,
29 | age INTEGER,
30 | degree STRING,
31 | address STRING,
32 | work_address STRING,
33 | income_range STRING,
34 | default_shipping_address STRING,
35 | register_date TIMESTAMP(3),
36 | udpate_date TIMESTAMP(3))
37 | ) WITH (
38 | 'connector' = 'hbase-2.2'
39 | ,'zookeeper.quorum' = 'dcmp10:2181,dcmp11:2181,dcmp12:2181'
40 | ,'zookeeper.znode.parent' = '/hbase'
41 | ,'table-name' = 'user_info'
42 | ,'sink.buffer-flush.max-size' = '10mb'
43 | ,'sink.buffer-flush.max-rows' = '2000'
44 | );
45 |
46 | insert into hbase_user_info_sink
47 | select user_id, row(sex, age, degree, address, work_address, income_range,default_shipping_address, register_date, udpate_date)
48 | from user_info;
49 |
50 |
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/tps/kafka_to_mysql.sql:
--------------------------------------------------------------------------------
1 | -- Lookup Source: Sync Mode
2 | -- kafka source
3 | CREATE TABLE user_info
4 | (
5 | user_id STRING,
6 | sex STRING,
7 | age INTEGER,
8 | degree STRING,
9 | address STRING,
10 | work_address STRING,
11 | income_range STRING,
12 | default_shipping_address STRING,
13 | register_date TIMESTAMP(3),
14 | udpate_date TIMESTAMP(3)
15 | ) WITH (
16 | 'connector' = 'kafka'
17 | ,'topic' = 'user_info'
18 | ,'properties.bootstrap.servers' = 'dcmp10:9092,dcmp11:9092,dcmp12:9092'
19 | ,'properties.group.id' = 'user_info'
20 | ,'scan.startup.mode' = 'latest-offset'
21 | ,'format' = 'json'
22 | );
23 |
24 | drop table if exists mysql_user_info_sink;
25 | CREATE TABLE mysql_user_info_sink
26 | (
27 | user_id STRING,
28 | sex STRING,
29 | age INTEGER,
30 | degree STRING,
31 | address STRING,
32 | work_address STRING,
33 | income_range STRING,
34 | default_shipping_address STRING,
35 | register_date TIMESTAMP(3),
36 | udpate_date TIMESTAMP(3)
37 | ) WITH (
38 | 'connector' = 'jdbc'
39 | ,'url' = 'jdbc:mysql://10.201.0.166:3306/shell1'
40 | ,'table-name' = 'user_info'
41 | ,'username' = 'root'
42 | ,'password' = 'daas2020'
43 | ,'sink.buffer-flush.max-rows' = '1000' -- default
44 | ,'sink.buffer-flush.interval' = '10s'
45 | ,'sink.max-retries' = '3'
46 | );
47 |
48 | insert into mysql_user_info_sink
49 | select user_id, sex, age, degree, address, work_address, income_range,default_shipping_address, register_date, udpate_date
50 | from user_info;
51 |
52 |
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/window/cumulate_offset.sql:
--------------------------------------------------------------------------------
1 | -- flink 1.14.0 cumulate window offset
2 | -- kafka source
3 | CREATE TABLE user_log (
4 | user_id STRING
5 | ,item_id STRING
6 | ,category_id STRING
7 | ,behavior STRING
8 | ,ts TIMESTAMP(3)
9 | ,process_time as proctime()
10 | , WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_behavior'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log'
16 | --,'scan.startup.mode' = 'group-offsets'
17 | ,'scan.startup.mode' = 'latest-offset'
18 | ,'format' = 'json'
19 | );
20 |
21 | ---sinkTable
22 | CREATE TABLE user_log_sink (
23 | window_start timestamp(3)
24 | ,window_end timestamp(3)
25 | ,window_time timestamp(3)
26 | ,coun bigint
27 | ) WITH (
28 | 'connector' = 'print'
29 | );
30 |
31 | insert into user_log_sink
32 | select window_start,window_end,window_time, count(user_id)
33 | from TABLE(CUMULATE(TABLE user_log, DESCRIPTOR(ts), interval '1' minute, interval '1' day, interval '10' minute))
34 | group by window_start,window_end,window_time
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/window/cumulate_pv_uv.sql:
--------------------------------------------------------------------------------
1 | -- flink cumulate window tvf calc pv&uv
2 | create table if not exists datagen_source (
3 | id int
4 | ,name string
5 | ,sex string
6 | ,age int
7 | ,birthday string
8 | ,proc_time as proctime()
9 | ) with (
10 | 'connector' = 'datagen'
11 | ,'rows-per-second' = '10000'
12 | ,'fields.id.kind' = 'random'
13 | ,'fields.id.min' = '1'
14 | ,'fields.id.max' = '2000000'
15 | );
16 |
17 | create table if not exists print_sink(
18 | start_time string
19 | ,end_time string
20 | ,pv bigint
21 | ,uv bigint
22 | ) with (
23 | 'connector' = 'print'
24 | );
25 |
26 | insert into print_sink
27 | select
28 | date_format(window_start, 'HH:mm:ss')
29 | , date_format(window_end, 'HH:mm:ss')
30 | , count(id)
31 | , count(distinct id)
32 | FROM TABLE(
33 | CUMULATE(TABLE datagen_source, DESCRIPTOR(proc_time), INTERVAL '10' SECOND, INTERVAL '1' DAY))
34 | GROUP BY window_start, window_end
35 |
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/window/kafka_window_demo.sql:
--------------------------------------------------------------------------------
1 | -- parse special json, then process window
2 | CREATE TABLE t_feature (
3 | header ROW(`catalog` STRING, readTime bigint)
4 | ,readModule STRING
5 | ,checkPoint STRING
6 | ,operation STRING
7 | ,location ROW(id BIGINT, code STRING, send_time STRING, rms decimal(12, 8),mean decimal(12, 8),peak decimal(12, 8),kurtosis decimal(12, 8),skewness decimal(12, 8))
8 | ,data ROW(meta STRING, `rows` ARRAY)
9 | ,process_time as proctime()
10 | -- ,watermark for header.readTime as header.readTime - INTERVAL '5' SECOND -- not work
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'test_dd'
14 | ,'properties.bootstrap.servers' = '10.201.1.131:9092'
15 | ,'properties.group.id' = 'user_log'
16 | ,'scan.startup.mode' = 'group-offsets'
17 | ,'format' = 'json'
18 | );
19 |
20 | create view v_t_feature as
21 | SELECT operation
22 | ,cast(data.`rows`[1] as bigint) id
23 | ,cast(data.`rows`[2] as string) code
24 | ,FROM_UNIXTIME(cast(data.`rows`[3] as bigint)) send_time
25 | ,cast(data.`rows`[4] as decimal(12, 8)) rms
26 | ,cast(data.`rows`[5] as decimal(12, 8)) mean
27 | ,cast(data.`rows`[6] as decimal(12, 8)) peak
28 | ,cast(data.`rows`[7] as decimal(12, 8)) kurtosis
29 | ,cast(data.`rows`[8] as decimal(12, 8)) skewness
30 | ,location.code
31 | ,process_time
32 | -- ,read_time
33 | FROM t_feature;
34 |
35 | CREATE TABLE t_sink (
36 | code STRING
37 | ,window_start TIMESTAMP(3)
38 | ,window_end TIMESTAMP(3)
39 | ,coun BIGINT
40 | ,rms DECIMAL(12, 8)
41 | ) WITH (
42 | 'connector' = 'print'
43 | );
44 |
45 | insert into t_sink
46 | SELECT code, window_start, window_end, count(id) coun, min(rms) min_rms FROM
47 | TABLE(TUMBLE(TABLE v_t_feature, DESCRIPTOR(process_time), INTERVAL '1' MINUTES))
48 | GROUP BY code, window_start, window_end
49 | ;
50 |
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/window/pv_uv.sql:
--------------------------------------------------------------------------------
1 | -- flink window tvf calc pv&uv
2 | create table if not exists datagen_source (
3 | id int
4 | ,name string
5 | ,sex string
6 | ,age int
7 | ,birthday string
8 | ,proc_time as proctime()
9 | ) with (
10 | 'connector' = 'datagen'
11 | ,'rows-per-second' = '10000'
12 | ,'fields.id.kind' = 'random'
13 | ,'fields.id.min' = '1'
14 | ,'fields.id.max' = '2000000'
15 | );
16 |
17 | create table if not exists print_sink(
18 | start_time string
19 | ,end_time string
20 | ,pv bigint
21 | ,uv bigint
22 | ) with (
23 | 'connector' = 'print'
24 | );
25 |
26 | insert into print_sink
27 | select
28 | date_format(window_start, 'HH:mm:ss')
29 | , date_format(window_end, 'HH:mm:ss')
30 | , count(id)
31 | , count(distinct id)
32 | FROM TABLE(
33 | TUMBLE(TABLE datagen_source, DESCRIPTOR(proc_time), INTERVAL '10' SECOND ))
34 | GROUP BY window_start, window_end
35 | union all
36 | select
37 | date_format(window_start, 'HH:mm:ss')
38 | , date_format(window_end, 'HH:mm:ss')
39 | , count(id)
40 | , count(distinct id)
41 | FROM TABLE(
42 | TUMBLE(TABLE datagen_source, DESCRIPTOR(proc_time), INTERVAL '20' SECOND ))
43 | GROUP BY window_start, window_end
44 | union all
45 | select
46 | date_format(window_start, 'HH:mm:ss')
47 | , date_format(window_end, 'HH:mm:ss')
48 | , count(id)
49 | , count(distinct id)
50 | FROM TABLE(
51 | TUMBLE(TABLE datagen_source, DESCRIPTOR(proc_time), INTERVAL '30' SECOND ))
52 | GROUP BY window_start, window_end
53 | ;
54 |
--------------------------------------------------------------------------------
/src/main/resources/sql/operator/window_demo.sql:
--------------------------------------------------------------------------------
1 | -- mysql cdc to print
2 | -- creates a mysql table source
3 | drop table if exists t_feature_source;
4 | CREATE TABLE t_feature_source (
5 | id bigint
6 | ,code VARCHAR
7 | ,rms DOUBLE
8 | ,mean DOUBLE
9 | ,peak DOUBLE
10 | ,kurtosis DOUBLE
11 | ,skewness DOUBLE
12 | ,send_time TIMESTAMP(3)
13 | ,WATERMARK FOR send_time AS send_time
14 | ) WITH (
15 | 'connector' = 'kafka'
16 | ,'topic' = 't_feature_1'
17 | ,'properties.bootstrap.servers' = '10.201.0.39:9092'
18 | ,'properties.group.id' = 't_feature_source'
19 | ,'format' = 'json'
20 | );
21 |
22 | -- kafka sink
23 | -- drop table if exists t_feature_sink;
24 | -- CREATE TABLE t_feature_sink (
25 | -- window_start TIMESTAMP(3)
26 | -- ,window_end TIMESTAMP(3)
27 | -- ,min_rms DOUBLE
28 | -- ,max_rms DOUBLE
29 | -- ,avg_rms DOUBLE
30 | -- ) WITH (
31 | -- 'connector' = 'print'
32 | -- );
33 |
34 | -- sink to kafka
35 | -- insert into t_feature_sink
36 | -- select window_start,window_end,min(rms) min_rms, max(rms) max_rms, avg(rms) avg_rms
37 | -- from TABLE(TUMBLE(TABLE t_feature_source, DESCRIPTOR(send_time), INTERVAL '1' MINUTES))
38 | -- where code = 'B416_1'
39 | -- group by window_start,window_end;
40 |
41 | CREATE TABLE t_feature_source_sink (
42 | id bigint
43 | ,code VARCHAR
44 | ,rms DOUBLE
45 | ,mean DOUBLE
46 | ,peak DOUBLE
47 | ,kurtosis DOUBLE
48 | ,skewness DOUBLE
49 | ,send_time TIMESTAMP(3)
50 | ) WITH (
51 | 'connector' = 'print'
52 | );
53 |
54 | insert into t_feature_source_sink
55 | select id, code, rms, mean, peak, kurtosis, skewness, send_time from t_feature_source
--------------------------------------------------------------------------------
/src/main/resources/sql/other/flink_cdc_tbls_to_mysql.sql:
--------------------------------------------------------------------------------
1 | CREATE TABLE mysql_tbls (
2 | TBL_ID BIGINT
3 | ,CREATE_TIME INT
4 | ,DB_ID BIGINT
5 | ,LAST_ACCESS_TIME INT
6 | ,OWNER VARCHAR(767)
7 | ,OWNER_TYPE VARCHAR(10)
8 | ,RETENTION INT
9 | ,SD_ID BIGINT
10 | ,TBL_NAME VARCHAR(256)
11 | ,TBL_TYPE VARCHAR(128)
12 | ,VIEW_EXPANDED_TEXT STRING
13 | ,VIEW_ORIGINAL_TEXT STRING
14 | ,PRIMARY KEY (TBL_ID) NOT ENFORCED
15 | ) WITH (
16 | 'connector' = 'mysql-cdc'
17 | ,'hostname' = 'localhost'
18 | ,'port' = '3306'
19 | ,'username' = 'root'
20 | ,'password' = '123456'
21 | ,'database-name' = 'hive_3'
22 | ,'table-name' = 'tbls'
23 | ,'server-id' = '5400-5440'
24 | ,'scan.startup.mode' = 'initial'
25 | );
26 |
27 | -- kafka sink
28 | drop table if exists mysql_tbls_new;
29 | CREATE TABLE mysql_tbls_new (
30 | TBL_ID BIGINT
31 | ,CREATE_TIME INT
32 | ,DB_ID BIGINT
33 | ,LAST_ACCESS_TIME INT
34 | ,OWNER VARCHAR(767)
35 | ,OWNER_TYPE VARCHAR(10)
36 | ,RETENTION INT
37 | ,SD_ID BIGINT
38 | ,TBL_NAME VARCHAR(256)
39 | ,TBL_TYPE VARCHAR(128)
40 | ,VIEW_EXPANDED_TEXT STRING
41 | ,VIEW_ORIGINAL_TEXT STRING
42 | -- ,PRIMARY KEY (tbl_id) NOT ENFORCED
43 | ) WITH (
44 | 'connector' = 'print'
45 | -- 'connector' = 'jdbc'
46 | -- ,'url' = 'jdbc:mysql://venn:3306/venn'
47 | -- ,'table-name' = 'tbls_new'
48 | -- ,'username' = 'root'
49 | -- ,'password' = '123456'
50 | );
51 |
52 | insert into mysql_tbls_new
53 | select TBL_ID,CREATE_TIME,DB_ID,LAST_ACCESS_TIME,OWNER,OWNER_TYPE,RETENTION,SD_ID,TBL_NAME,TBL_TYPE,VIEW_EXPANDED_TEXT,VIEW_ORIGINAL_TEXT
54 | from mysql_tbls;
55 |
56 | -- create table TBLS
57 | -- (
58 | -- TBL_ID bigint not null
59 | -- primary key,
60 | -- CREATE_TIME int not null,
61 | -- DB_ID bigint null,
62 | -- LAST_ACCESS_TIME int not null,
63 | -- OWNER varchar(767) collate latin1_bin null,
64 | -- OWNER_TYPE varchar(10) collate latin1_bin null,
65 | -- RETENTION int not null,
66 | -- SD_ID bigint null,
67 | -- TBL_NAME varchar(256) collate latin1_bin null,
68 | -- TBL_TYPE varchar(128) collate latin1_bin null,
69 | -- VIEW_EXPANDED_TEXT mediumtext null,
70 | -- VIEW_ORIGINAL_TEXT mediumtext null,
71 | -- IS_REWRITE_ENABLED bit default b'0' not null
72 | -- );
--------------------------------------------------------------------------------
/src/main/resources/sql/other/kafka_to_mysql_group_by.sql:
--------------------------------------------------------------------------------
1 | -- calc pv
2 | -- test multi column primary key update mysql
3 |
4 | -- kafka source
5 | drop table if exists user_log;
6 | CREATE TABLE user_log
7 | (
8 | `event_time` TIMESTAMP(3) METADATA FROM 'timestamp' VIRTUAL, -- from Debezium format
9 | `partition_id` BIGINT METADATA FROM 'partition' VIRTUAL, -- from Kafka connector
10 | `offset` BIGINT METADATA VIRTUAL, -- from Kafka connector
11 | user_id VARCHAR,
12 | item_id VARCHAR,
13 | category_id VARCHAR,
14 | behavior VARCHAR,
15 | ts TIMESTAMP(3),
16 | WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
17 | ) WITH (
18 | 'connector' = 'kafka'
19 | ,'topic' = 'user_log'
20 | ,'properties.bootstrap.servers' = 'localhost:9092'
21 | ,'properties.group.id' = 'user_log'
22 | ,'scan.startup.mode' = 'latest-offset'
23 | ,'format' = 'json'
24 | );
25 |
26 | -- set table.sql-dialect=hive;
27 | -- kafka sink
28 | CREATE TABLE user_log_sink
29 | (
30 | category_id varchar(20),
31 | behavior varchar(20),
32 | cnt bigint,
33 | primary key(category_id, behavior) NOT ENFORCED
34 | ) WITH (
35 | 'connector' = 'jdbc'
36 | ,'url' = 'jdbc:mysql://localhost:3306/venn'
37 | ,'table-name' = 'user_view'
38 | ,'username' = 'root'
39 | ,'password' = '123456'
40 | );
41 |
42 |
43 | -- streaming sql, insert into mysql table
44 | insert into user_log_sink
45 | SELECT category_id, behavior, count(user_id) cnt
46 | FROM user_log
47 | group by category_id, behavior;
48 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/cep/cep_count_2000.sql:
--------------------------------------------------------------------------------
1 | -- cep 失败
2 | -- kafka source
3 | CREATE TABLE user_log (
4 | user_id STRING
5 | ,item_id STRING
6 | ,category_id STRING
7 | ,behavior STRING
8 | ,ts TIMESTAMP(3)
9 | ,process_time as proctime()
10 | , WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_log'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log'
16 | ,'scan.startup.mode' = 'latest-offset'
17 | ,'format' = 'json'
18 | );
19 |
20 | CREATE TABLE user_log_sink (
21 | first_ts timestamp (3)
22 | ,last_ts timestamp (3)
23 | ,cout bigint
24 | ) WITH (
25 | 'connector' = 'print'
26 | );
27 |
28 | -- cep 连续事件: a b c
29 | insert into user_log_sink
30 | select first_ts, last_ts, cout
31 | from user_log
32 | MATCH_RECOGNIZE(
33 | -- partition by item_id
34 | order by process_time
35 | MEASURES
36 | FIRST(a.process_time) as first_ts
37 | ,last(a.process_time) as last_ts
38 | ,count(a.user_id) as cout
39 | ONE ROW PER MATCH
40 | PATTERN (a{2000}) --WITHIN INTERVAL '1' MINUTE
41 | DEFINE
42 | a as a.user_id is not null
43 | )as t
--------------------------------------------------------------------------------
/src/main/resources/sql/release/cep/cep_event_1.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | CREATE TABLE user_log (
3 | user_id STRING
4 | ,item_id STRING
5 | ,category_id STRING
6 | ,behavior STRING
7 | ,ts TIMESTAMP(3)
8 | ,process_time as proctime()
9 | , WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
10 | ) WITH (
11 | 'connector' = 'kafka'
12 | ,'topic' = 'user_log'
13 | ,'properties.bootstrap.servers' = 'localhost:9092'
14 | ,'properties.group.id' = 'user_log'
15 | ,'scan.startup.mode' = 'latest-offset'
16 | ,'format' = 'json'
17 | );
18 |
19 | CREATE TABLE user_log_sink (
20 | aid STRING
21 | ,bid STRING
22 | ,cid STRING
23 | ) WITH (
24 | 'connector' = 'print'
25 | );
26 |
27 | -- cep 连续事件: a b c
28 | insert into user_log_sink
29 | select t.aid,t.bid,t.cid
30 | from user_log
31 | MATCH_RECOGNIZE(
32 | partition by item_id
33 | order by process_time
34 | MEASURES
35 | a.user_id as aid
36 | ,b.user_id as bid
37 | ,d.user_id as cid
38 | ONE ROW PER MATCH
39 | PATTERN (a b d) --WITHIN INTERVAL '1' MINUTE
40 | DEFINE
41 | a as a.user_id='user_id_1'
42 | ,b as b.user_id='user_id_2'
43 | ,d as d.user_id='user_id_3'
44 | )as t
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/cust/cust_http_source_demo.sql:
--------------------------------------------------------------------------------
1 | create table cust_http_source(
2 | id string
3 | ,name string
4 | ,sex string
5 | )WITH(
6 | 'connector' = 'http'
7 | ,'http.url' = 'http://localhost:8888'
8 | ,'http.interval' = '1000'
9 | ,'format' = 'csv'
10 | )
11 | ;
12 |
13 | create table cust_http_sink(
14 | id string
15 | ,name string
16 | ,sex string
17 | )WITH(
18 | 'connector' = 'print'
19 | )
20 | ;
21 |
22 | insert into cust_http_sink
23 | select id,name,sex
24 | from cust_http_source;
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/cust/cust_mysql_source_demo.sql:
--------------------------------------------------------------------------------
1 | create table cust_mysql_user_log
2 | (
3 | user_id STRING,
4 | sex STRING,
5 | age INTEGER,
6 | degree STRING,
7 | address STRING,
8 | work_address STRING,
9 | income_range STRING,
10 | default_shipping_address STRING,
11 | register_date TIMESTAMP(3)
12 | ) WITH (
13 | 'connector' = 'cust-mysql'
14 | ,'url' = 'jdbc:mysql://10.201.0.166:3306/shell1?useUnicode=true&characterEncoding=utf8&useSSL=false&allowPublicKeyRetrieval=true'
15 | ,'username' = 'root'
16 | ,'password' = 'daas2020'
17 | ,'database' = 'shell1'
18 | ,'table' = 'user_info'
19 | ,'key' = 'id'
20 | ,'batch.size' = '10000'
21 | )
22 | ;
23 |
24 | create table cust_mysql_user_log_sink
25 | (
26 | user_id STRING,
27 | sex STRING,
28 | age INTEGER,
29 | degree STRING,
30 | address STRING,
31 | work_address STRING,
32 | income_range STRING,
33 | default_shipping_address STRING,
34 | register_date TIMESTAMP(3)
35 | ) WITH (
36 | 'connector' = 'kafka'
37 | ,'topic' = 'user_log_sink'
38 | ,'properties.bootstrap.servers' = 'localhost:9092'
39 | ,'properties.group.id' = 'user_log'
40 | ,'scan.startup.mode' = 'latest-offset'
41 | ,'format' = 'json'
42 | )
43 | ;
44 |
45 | insert into cust_mysql_user_log_sink
46 | select user_id, sex, age, degree, address, work_address, income_range,default_shipping_address, register_date
47 | from cust_mysql_user_log;
48 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/cust/cust_socket_source_demo.sql:
--------------------------------------------------------------------------------
1 | create table cust_socket_source(
2 | id string
3 | ,name string
4 | ,sex string
5 | )WITH(
6 | 'connector' = 'socket'
7 | ,'hostname' = 'localhost'
8 | ,'port' = '8888'
9 | -- ,'byte-delimiter' = '10'
10 | ,'format' = 'changelog-csv'
11 | ,'changelog-csv.column-delimiter' = ','
12 | )
13 | ;
14 |
15 | create table cust_socket_sink(
16 | id string
17 | ,name string
18 | ,sex string
19 | )WITH(
20 | 'connector' = 'print'
21 | )
22 | ;
23 |
24 | insert into cust_socket_sink
25 | select id,name,sex
26 | from cust_socket_source;
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/cust/jdbc_mysql_source_demo.sql:
--------------------------------------------------------------------------------
1 | create table user_log
2 | (
3 | user_id STRING,
4 | sex STRING,
5 | age INTEGER,
6 | degree STRING,
7 | address STRING,
8 | work_address STRING,
9 | income_range STRING,
10 | default_shipping_address STRING,
11 | register_date TIMESTAMP(3)
12 | ) WITH (
13 | 'connector' = 'jdbc'
14 | ,'url' = 'jdbc:mysql://10.201.0.166:3306/shell1?useUnicode=true&characterEncoding=utf8&useSSL=false&allowPublicKeyRetrieval=true'
15 | ,'username' = 'root'
16 | ,'password' = 'daas2020'
17 | ,'table-name' = 'user_info'
18 | ,'scan.partition.column' = 'id'
19 | ,'scan.partition.num' = '10000'
20 | ,'scan.partition.lower-bound' = '0'
21 | ,'scan.partition.upper-bound' = '9999999999'
22 | )
23 | ;
24 |
25 | create table cust_mysql_user_log_sink
26 | (
27 | user_id STRING,
28 | sex STRING,
29 | age INTEGER,
30 | degree STRING,
31 | address STRING,
32 | work_address STRING,
33 | income_range STRING,
34 | default_shipping_address STRING,
35 | register_date TIMESTAMP(3)
36 | ) WITH (
37 | 'connector' = 'kafka'
38 | ,'topic' = 'user_log_sink'
39 | ,'properties.bootstrap.servers' = 'localhost:9092'
40 | ,'properties.group.id' = 'user_log'
41 | ,'scan.startup.mode' = 'latest-offset'
42 | ,'format' = 'json'
43 | )
44 | ;
45 |
46 | insert into cust_mysql_user_log_sink
47 | select user_id, sex, age, degree, address, work_address, income_range,default_shipping_address, register_date
48 | from user_log;
49 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/hbase/kafka_to_hbase.sql:
--------------------------------------------------------------------------------
1 | -- Lookup Source: Sync Mode
2 | -- kafka source
3 | CREATE TABLE user_log (
4 | user_id STRING
5 | ,item_id STRING
6 | ,category_id STRING
7 | ,behavior STRING
8 | ,ts TIMESTAMP(3)
9 | ,process_time as proctime()
10 | , WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_behavior'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log'
16 | ,'scan.startup.mode' = 'group-offsets'
17 | ,'format' = 'json'
18 | );
19 |
20 | drop table if exists hbase_user_log_sink ;
21 | CREATE TABLE hbase_user_log_sink (
22 | user_id STRING
23 | ,cf ROW(item_id STRING
24 | ,category_id STRING
25 | ,behavior STRING
26 | ,ts TIMESTAMP(3))
27 | ) WITH (
28 | 'connector' = 'hbase-2.2'
29 | ,'zookeeper.quorum' = 'localhost:12181'
30 | ,'zookeeper.znode.parent' = '/hbase'
31 | ,'table-name' = 'user_log'
32 | -- ,'lookup.cache.max-rows' = '10000'
33 | -- ,'lookup.cache.ttl' = '10 minute' -- ttl time 超过这么长时间无数据才行
34 | -- ,'lookup.async' = 'true'
35 | );
36 |
37 | insert into hbase_user_log_sink
38 | select user_id, row(item_id, category_id, behavior, ts)
39 | from user_log;
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/hive/hive_to_kafka.sql:
--------------------------------------------------------------------------------
1 | -- read hive, write to kafka -- batch when read complete, job finish
2 | -- sink
3 | drop table if exists read_hiv_sink;
4 | CREATE TABLE read_hiv_sink (
5 | user_id VARCHAR
6 | ,item_id VARCHAR
7 | ,category_id VARCHAR
8 | ,behavior VARCHAR
9 | ,dt VARCHAR
10 | ,hr VARCHAR
11 | ) WITH (
12 | 'connector.type' = 'kafka'
13 | ,'connector.version' = 'universal'
14 | ,'connector.topic' = 'read_hiv_sink'
15 | ,'connector.properties.zookeeper.connect' = 'venn:2181'
16 | ,'connector.properties.bootstrap.servers' = 'venn:9092'
17 | ,'connector.properties.group.id' = 'flink_sql'
18 | ,'connector.startup-mode' = 'group-offsets'
19 | ,'connector.sink-partitioner' = 'fixed'
20 | ,'format.type' = 'json'
21 | );
22 |
23 | insert into read_hiv_sink select user_id, item_id, category_id, behavior, dt, hr from hive_table;
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/hive/kafka_to_hive.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | drop table if exists user_log;
3 | CREATE TABLE user_log (
4 | user_id VARCHAR
5 | ,item_id VARCHAR
6 | ,category_id VARCHAR
7 | ,behavior VARCHAR
8 | ,ts TIMESTAMP(3)
9 | ,WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
10 | ) WITH (
11 | 'connector.type' = 'kafka'
12 | ,'connector.version' = 'universal'
13 | ,'connector.topic' = 'user_behavior'
14 | ,'connector.properties.zookeeper.connect' = 'venn:2181'
15 | ,'connector.properties.bootstrap.servers' = 'venn:9092'
16 | ,'connector.properties.group.id' = 'user_log'
17 | ,'connector.startup-mode' = 'group-offsets'
18 | ,'connector.sink-partitioner' = 'fixed'
19 | ,'format.type' = 'json'
20 | );
21 |
22 | -- set table.sql-dialect=hive;
23 | -- kafka sink
24 | drop table if exists hive_table_user_log_sink;
25 | CREATE TABLE hive_table_user_log_sink (
26 | user_id STRING
27 | ,item_id STRING
28 | ,category_id STRING
29 | ,behavior STRING
30 | ) PARTITIONED BY (dt STRING, hr STRING) STORED AS parquet TBLPROPERTIES (
31 | 'partition.time-extractor.timestamp-pattern'='$dt $hr:00:00',
32 | 'sink.partition-commit.trigger'='partition-time',
33 | 'sink.partition-commit.delay'='1 min',
34 | 'sink.partition-commit.policy.kind'='metastore,success-file'
35 | );
36 |
37 |
38 | -- streaming sql, insert into hive table
39 | insert into table hive_table_user_log_sink
40 | SELECT user_id, item_id, category_id, behavior, DATE_FORMAT(ts, 'yyyy-MM-dd'), DATE_FORMAT(ts, 'HH')
41 | FROM user_log;
42 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/hive/kafka_to_hive_ms.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | drop table if exists user_log_ms;
3 | CREATE TABLE user_log_ms (
4 | user_id VARCHAR
5 | ,item_id VARCHAR
6 | ,category_id VARCHAR
7 | ,behavior VARCHAR
8 | ,ts TIMESTAMP(3)
9 | ,WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
10 | ) WITH (
11 | 'connector.type' = 'kafka'
12 | ,'connector.version' = 'universal'
13 | ,'connector.topic' = 'user_behavior'
14 | ,'connector.properties.zookeeper.connect' = 'venn:2181'
15 | ,'connector.properties.bootstrap.servers' = 'venn:9092'
16 | ,'connector.properties.group.id' = 'user_log'
17 | ,'connector.startup-mode' = 'group-offsets'
18 | ,'connector.sink-partitioner' = 'fixed'
19 | ,'format.type' = 'json'
20 | );
21 |
22 | -- set table.sql-dialect=hive;
23 | -- kafka sink
24 | drop table if exists hive_table_user_log_ms_sink;
25 | CREATE TABLE hive_table_user_log_ms_sink (
26 | user_id STRING
27 | ,item_id STRING
28 | ,category_id STRING
29 | ,behavior STRING
30 | ) PARTITIONED BY (dt STRING, ms STRING) TBLPROPERTIES (
31 | 'partition.time-extractor.timestamp-pattern'='$dt $ms:00',
32 | 'sink.partition-commit.trigger'='partition-time',
33 | 'sink.partition-commit.delay'='1 min',
34 | 'sink.partition-commit.policy.kind'='metastore,success-file'
35 | );
36 |
37 |
38 | -- streaming sql, insert into hive table
39 | INSERT INTO TABLE hive_table_user_log_ms_sink
40 | SELECT user_id, item_id, category_id, behavior, DATE_FORMAT(ts, 'yyyy-MM-dd-HH'), DATE_FORMAT(ts, 'mm')
41 | FROM user_log_ms;
42 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/iceberg/kafka_to_iceberg.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | -- drop table if exists user_log ;
3 | -- CREATE TABLE user_log (
4 | -- user_id VARCHAR
5 | -- ,item_id VARCHAR
6 | -- ,category_id VARCHAR
7 | -- ,behavior VARCHAR
8 | -- ) WITH (
9 | -- 'connector' = 'kafka'
10 | -- ,'topic' = 'user_behavior'
11 | -- ,'properties.bootstrap.servers' = 'localhost:9092'
12 | -- ,'properties.group.id' = 'user_log'
13 | -- ,'format' = 'json'
14 | -- );
15 |
16 | CREATE CATALOG hive_catalog WITH (
17 | 'type'='iceberg',
18 | 'catalog-type'='hive',
19 | 'uri'='thrift://localhost:9083',
20 | 'clients'='5',
21 | 'property-version'='1',
22 | 'warehouse'='hdfs:///tmp/iceberg/hive_catalog'
23 | );
24 | -- kafka sink
25 | -- use catalog hive_catalog;
26 | -- create database hive_catalog_db;
27 | -- use hive_catalog_db;
28 | -- CREATE TABLE hive_catalog.hive_catalog_db.user_log_sink (
29 | -- user_id VARCHAR
30 | -- ,item_id VARCHAR
31 | -- ,category_id VARCHAR
32 | -- ,behavior VARCHAR
33 | -- ) WITH (
34 | -- 'type'='iceberg',
35 | -- 'catalog-type'='hadoop',
36 | -- 'warehouse'='hdfs:////tmp/iceberg/hive_catalog/hive_catalog_db/user_log_sink',
37 | -- 'property-version'='1'
38 | -- );
39 |
40 | -- insert
41 | insert into hive_catalog.hive_catalog_db.user_log_sink
42 | select user_id, item_id, category_id, behavior || '_1'
43 | from user_log;
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/jdbc/mysql_to_kafka.sql:
--------------------------------------------------------------------------------
1 | -- scan source : bounded 一次执行,读完就任务结束
2 | -- mysql source
3 | drop table if exists mysql_user_log ;
4 | CREATE TABLE mysql_user_log (
5 | id int
6 | ,user_id VARCHAR
7 | ,item_id VARCHAR
8 | ,category_id VARCHAR
9 | ,behavior VARCHAR
10 | ,ts TIMESTAMP(3)
11 | ,create_time TIMESTAMP(3)
12 | ,insert_time TIMESTAMP(3)
13 | ,primary key (id) not enforced
14 | ) WITH (
15 | 'connector' = 'jdbc'
16 | ,'url' = 'jdbc:mysql://venn:3306/venn'
17 | ,'table-name' = 'user_log'
18 | ,'username' = 'root'
19 | ,'password' = '123456'
20 | );
21 |
22 | -- kafka sink
23 | drop table if exists user_log_sink ;
24 | CREATE TABLE user_log_sink (
25 | id int
26 | ,user_id VARCHAR
27 | ,item_id VARCHAR
28 | ,category_id VARCHAR
29 | ,behavior VARCHAR
30 | ,ts TIMESTAMP(3)
31 | ,create_time TIMESTAMP(3)
32 | ,insert_time TIMESTAMP(3)
33 | ) WITH (
34 | 'connector' = 'kafka'
35 | ,'topic' = 'user_behavior_sink'
36 | ,'properties.bootstrap.servers' = 'localhost:9092'
37 | ,'properties.group.id' = 'user_log'
38 | ,'scan.startup.mode' = 'group-offsets'
39 | ,'format' = 'json'
40 | );
41 |
42 | -- insert
43 | insert into user_log_sink
44 | select id, user_id, item_id, category_id, behavior, ts, create_time, insert_time
45 | from mysql_user_log;
46 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/kafka/kafka_source_parallelism_demo.sql:
--------------------------------------------------------------------------------
1 | -- execute config: 'table.exec.source.force-break-chain' = 'true'
2 | -- kafka source
3 | CREATE TABLE user_log (
4 | user_id STRING
5 | ,item_id STRING
6 | ,category_id STRING
7 | ,behavior STRING
8 | ,ts TIMESTAMP(3)
9 | ,process_time as proctime()
10 | , WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_log'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log'
16 | ,'scan.startup.mode' = 'latest-offset'
17 | ,'format' = 'json'
18 | --,'source.parallelism' = '-2'
19 | ,'source.parallelism' = '2'
20 | );
21 |
22 | CREATE TABLE user_log_sink (
23 | `day` string
24 | ,num bigint
25 | ,min_user_id bigint
26 | ,max_user_id bigint
27 | ) WITH (
28 | 'connector' = 'print'
29 | );
30 |
31 | insert into user_log_sink
32 | select `day`
33 | , num
34 | , min_user_id, max_user_id
35 | from(
36 | select DATE_FORMAT(ts,'yyyyMMdd') `day`
37 | ,count(distinct user_id) num
38 | ,min(cast(replace(user_id,'xxxxxxxxxxxxx','') as bigint)) min_user_id
39 | ,max(cast(replace(user_id,'xxxxxxxxxxxxx','') as bigint)) max_user_id
40 | from user_log
41 | -- where DATE_FORMAT(ts,'yyyyMMdd') = date_format(current_timestamp, 'yyyyMMdd')
42 | group by DATE_FORMAT(ts,'yyyyMMdd')
43 | )t1
44 | where num % 2 = 0
45 | ;
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/kafka/kafka_to_hbase.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | drop table if exists user_log;
3 | CREATE TABLE user_log (
4 | user_id VARCHAR
5 | ,item_id VARCHAR
6 | ,category_id VARCHAR
7 | ,behavior VARCHAR
8 | ,ts TIMESTAMP(3)
9 | ) WITH (
10 | 'connector' = 'kafka'
11 | ,'topic' = 'user_behavior' -- required: topic name from which the table is read
12 | ,'properties.bootstrap.servers' = 'localhost:9092' -- required: specify the Kafka server connection string
13 | ,'properties.group.id' = 'user_log' -- optional: required in Kafka consumer, specify consumer group
14 | ,'format' = 'json' -- required: 'csv', 'json' and 'avro'.
15 | );
16 |
17 | -- kafka sink
18 | drop table if exists user_log_sink;
19 | CREATE TABLE user_log_sink (
20 | user_id string
21 | ,cf ROW(item_id VARCHAR
22 | ,category_id VARCHAR
23 | ,behavior VARCHAR
24 | ,ts TIMESTAMP(3))
25 | ) WITH (
26 | 'connector.type' = 'hbase'
27 | ,'connector.version' = '1.4.3'
28 | ,'connector.table-name' = 'venn'
29 | ,'connector.zookeeper.quorum' = 'venn:2181'
30 | ,'connector.zookeeper.znode.parent' = '/hbase'
31 | ,'connector.write.buffer-flush.max-size' = '10mb'
32 | ,'connector.write.buffer-flush.max-rows' = '10'
33 | ,'connector.write.buffer-flush.interval' = '2s'
34 | );
35 |
36 | -- insert
37 | insert into user_log_sink
38 | select user_id, ROW(item_id, category_id, behavior, ts) as cf
39 | from user_log;
40 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/kafka/kafka_to_kafka.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | drop table if exists user_log ;
3 | CREATE TABLE user_log (
4 | user_id VARCHAR
5 | ,item_id VARCHAR
6 | ,category_id VARCHAR
7 | ,behavior VARCHAR
8 | ,ts TIMESTAMP(3)
9 | ) WITH (
10 | 'connector' = 'kafka'
11 | ,'topic' = 'user_behavior' -- required: topic name from which the table is read
12 | ,'properties.bootstrap.servers' = 'localhost:9092' -- required: specify the Kafka server connection string
13 | ,'properties.group.id' = 'user_log' -- optional: required in Kafka consumer, specify consumer group
14 | ,'format' = 'json' -- required: 'csv', 'json' and 'avro'.
15 | );
16 |
17 | -- kafka sink
18 | drop table if exists user_log_sink ;
19 | CREATE TABLE user_log_sink (
20 | user_id VARCHAR
21 | ,item_id VARCHAR
22 | ,category_id VARCHAR
23 | ,behavior VARCHAR
24 | ,ts TIMESTAMP(3)
25 | ) WITH (
26 | 'connector' = 'kafka'
27 | -- ,'pipeline.name' = 'kafka_to_Kafka'
28 | ,'topic' = 'user_behavior_sink' -- required: topic name from which the table is read
29 | ,'properties.bootstrap.servers' = 'localhost:9092' -- required: specify the Kafka server connection string
30 | ,'properties.group.id' = 'user_log' -- optional: required in Kafka consumer, specify consumer group
31 | ,'sink.partitioner' = 'fixed' --optional fixed 每个 flink 分区数据只发到 一个 kafka 分区
32 | -- round-robin flink 分区轮询分配到 kafka 分区
33 | -- custom 自定义分区策略
34 | --,'connector.sink-partitioner-class' = 'org.mycompany.MyPartitioner' -- 自定义分区类
35 | ,'format' = 'json' -- required: 'csv', 'json' and 'avro'.
36 | );
37 |
38 | -- insert
39 | insert into user_log_sink
40 | select user_id, item_id, category_id, behavior || '_1', ts
41 | from user_log;
42 |
43 | -- insert 2
44 | insert into user_log_sink
45 | select user_id, item_id, category_id, behavior || '_2', ts
46 | from user_log;
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/kafka/kafka_to_print.sql:
--------------------------------------------------------------------------------
1 | -- 输入数据不按字段分割,当前一个String 类型的字段,交由后续的sql 处理
2 | -- kafka source
3 | drop table if exists user_log ;
4 | CREATE TABLE user_log (
5 | str varchar
6 | ) WITH (
7 | 'connector.type' = 'kafka'
8 | ,'connector.version' = 'universal'
9 | ,'connector.topic' = 'user_behavior' -- required: topic name from which the table is read
10 | ,'connector.properties.zookeeper.connect' = 'venn:2181' -- required: specify the ZooKeeper connection string
11 | ,'connector.properties.bootstrap.servers' = 'venn:9092' -- required: specify the Kafka server connection string
12 | ,'connector.properties.group.id' = 'user_log' -- optional: required in Kafka consumer, specify consumer group
13 | ,'connector.startup-mode' = 'group-offsets' -- optional: valid modes are "earliest-offset", "latest-offset", "group-offsets", "specific-offsets"
14 | ,'connector.sink-partitioner' = 'fixed' --optional fixed 每个 flink 分区数据只发到 一个 kafka 分区
15 | -- round-robin flink 分区轮询分配到 kafka 分区
16 | -- custom 自定义分区策略
17 | --,'connector.sink-partitioner-class' = 'org.mycompany.MyPartitioner' -- 自定义分区类
18 | ,'format.type' = 'csv' -- required: 'csv', 'json' and 'avro'.
19 | ,'format.field-delimiter' = '|'
20 | );
21 |
22 | -- kafka sink
23 | drop table if exists user_log_sink ;
24 | CREATE TABLE user_log_sink (
25 | str varchar
26 | ) WITH (
27 | 'connector' = 'print'
28 | );
29 |
30 | -- insert
31 | insert into user_log_sink
32 | select str
33 | from user_log;
34 |
35 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/kafka/kafka_upsert_demo.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | CREATE TABLE user_log (
3 | user_id VARCHAR
4 | ,item_id VARCHAR
5 | ,category_id VARCHAR
6 | ,behavior VARCHAR
7 | ,ts TIMESTAMP(3)
8 | ) WITH (
9 | 'connector' = 'kafka'
10 | ,'topic' = 'user_behavior'
11 | ,'properties.bootstrap.servers' = 'localhost:9092'
12 | ,'properties.group.id' = 'user_log'
13 | ,'scan.startup.mode' = 'group-offsets'
14 | ,'format' = 'json'
15 | );
16 |
17 | -- kafka sink
18 | CREATE TABLE user_log_sink (
19 | user_id varchar
20 | ,max_tx bigint
21 | ,primary key (user_id) not enforced
22 | ) WITH (
23 | 'connector' = 'upsert-kafka'
24 | ,'topic' = 'user_behavior_sink'
25 | ,'properties.bootstrap.servers' = 'localhost:9092'
26 | ,'properties.group.id' = 'user_log'
27 | ,'key.format' = 'json'
28 | ,'key.json.ignore-parse-errors' = 'true'
29 | ,'value.format' = 'json'
30 | ,'value.json.fail-on-missing-field' = 'false'
31 | ,'value.fields-include' = 'ALL'
32 | -- ,'format' = 'json'
33 | );
34 |
35 | -- insert
36 | insert into user_log_sink
37 | select user_id, count(user_id)
38 | from user_log
39 | group by user_id ;
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/kafka_to_socket.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | drop table if exists user_log;
3 | CREATE TABLE user_log (
4 | user_id VARCHAR
5 | ,item_id VARCHAR
6 | ,category_id VARCHAR
7 | ,behavior VARCHAR
8 | ,ts TIMESTAMP(3)
9 | ,WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
10 | ) WITH (
11 | 'connector' = 'kafka'
12 | ,'topic' = 'user_log'
13 | ,'properties.bootstrap.servers' = 'localhost:9092'
14 | ,'properties.group.id' = 'user_log'
15 | ,'scan.startup.mode' = 'latest-offset'
16 | ,'format' = 'json'
17 | );
18 |
19 | -- set table.sql-dialect=hive;
20 | -- kafka sink
21 | drop table if exists socket_sink;
22 | CREATE TABLE socket_sink (
23 | user_id STRING
24 | ,item_id STRING
25 | ,category_id STRING
26 | ,behavior STRING
27 | ,ts timestamp(3)
28 | ) WITH (
29 | 'connector' = 'socket'
30 | ,'hostname' = 'localhost'
31 | ,'max.retry' = '2'
32 | -- ,'retry.interval' = '2'
33 | ,'port' = '19870'
34 | ,'format' = 'json'
35 | );
36 |
37 |
38 | -- streaming sql, insert into mysql table
39 | insert into socket_sink
40 | SELECT user_id, item_id, category_id, behavior, ts
41 | FROM user_log;
42 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/kudu/kafka_to_kudu.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | drop table if exists user_log;
3 | CREATE TABLE user_log (
4 | user_id VARCHAR
5 | ,item_id VARCHAR
6 | ,category_id VARCHAR
7 | ,behavior INT
8 | ,ts TIMESTAMP(3)
9 | ,process_time as proctime()
10 | , WATERMARK FOR ts AS ts
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_behavior'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log_x'
16 | ,'scan.startup.mode' = 'group-offsets'
17 | ,'format' = 'json'
18 | );
19 |
20 | -- kafka sink
21 | drop table if exists user_log_sink;
22 | CREATE TABLE user_log_sink (
23 | user_id STRING
24 | ,item_id STRING
25 | ,category_id STRING
26 | ,ts TIMESTAMP(3)
27 | ) WITH (
28 | 'connector.type' = 'kudu'
29 | ,'kudu.masters' = 'localhost:7051,localhost:7151,localhost:7251'
30 | ,'kudu.table' = 'user_log'
31 | ,'kudu.hash-columns' = 'user_id'
32 | ,'kudu.primary-key-columns' = 'user_id'
33 | ,'kudu.max-buffer-size' = '5000'
34 | ,'kudu.flush-interval' = '1000'
35 | );
36 |
37 | -- insert
38 | insert into user_log_sink
39 | select user_id, item_id, category_id,ts
40 | from user_log;
41 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/mysql/batch_write_mysql_test.sql:
--------------------------------------------------------------------------------
1 | -- docker mysql, local, tps max : 466
2 | -- kafka source
3 | drop table if exists user_log;
4 | CREATE TABLE user_log
5 | (
6 | user_id VARCHAR,
7 | item_id VARCHAR,
8 | category_id VARCHAR,
9 | behavior VARCHAR,
10 | ts TIMESTAMP(3),
11 | proc_time as proctime(),
12 | WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
13 | ) WITH (
14 | 'connector' = 'kafka'
15 | ,'topic' = 'user_log'
16 | ,'properties.bootstrap.servers' = 'localhost:9092'
17 | ,'properties.group.id' = 'user_log'
18 | ,'scan.startup.mode' = 'latest-offset'
19 | ,'format' = 'json'
20 | );
21 |
22 | -- set table.sql-dialect=hive;
23 | -- kafka sink
24 | drop table if exists mysql_table_venn_user_log_sink;
25 | CREATE TABLE mysql_table_venn_user_log_sink
26 | (
27 | user_id STRING,
28 | item_id STRING,
29 | category_id STRING,
30 | behavior STRING,
31 | ts timestamp(3),
32 | create_time timestamp(3)
33 | ) WITH (
34 | 'connector' = 'jdbc'
35 | ,'url' = 'jdbc:mysql://localhost:3306/venn'
36 | ,'table-name' = 'user_log'
37 | ,'username' = 'root'
38 | ,'password' = '123456'
39 | ,'sink.buffer-flush.max-rows' = '1000' -- default
40 | ,'sink.buffer-flush.interval' = '10s'
41 | ,'sink.max-retries' = '3'
42 | );
43 |
44 |
45 | -- streaming sql, insert into mysql table
46 | insert into mysql_table_venn_user_log_sink
47 | SELECT user_id, item_id, category_id, behavior, ts, proc_time
48 | FROM user_log;
49 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/mysql/cdc_mysql_to_kafka_demo.sql:
--------------------------------------------------------------------------------
1 | -- creates a mysql mysql table source
2 | drop table if exists cdc_mysql_venn_user_log;
3 | CREATE TABLE cdc_mysql_venn_user_log (
4 | id varchar
5 | ,user_id VARCHAR
6 | ,item_id VARCHAR
7 | ,category_id VARCHAR
8 | ,behavior VARCHAR
9 | ,ts TIMESTAMP(3)
10 | ,proc_time as PROCTIME()
11 | ,PRIMARY KEY (id) NOT ENFORCED
12 | ) WITH (
13 | 'connector' = 'mysql-cdc',
14 | 'hostname' = 'venn',
15 | 'port' = '3306',
16 | 'username' = 'root',
17 | 'password' = '123456',
18 | 'database-name' = 'venn',
19 | 'table-name' = 'user_log'
20 | );
21 |
22 | -- kafka sink
23 | drop table if exists cdc_mysql_user_log_sink;
24 | CREATE TABLE cdc_mysql_user_log_sink (
25 | id varchar
26 | ,user_id VARCHAR
27 | ,item_id VARCHAR
28 | ,category_id VARCHAR
29 | ,behavior VARCHAR
30 | ,ts TIMESTAMP(3)
31 | ) WITH (
32 | 'connector.type' = 'upsertKafka'
33 | ,'connector.version' = 'universal'
34 | ,'connector.topic' = 'cdc_mysql_user_log_sink'
35 | ,'connector.properties.zookeeper.connect' = 'venn:2181'
36 | ,'connector.properties.bootstrap.servers' = 'venn:9092'
37 | ,'format.type' = 'json'
38 | );
39 |
40 | -- sink to kafka
41 | insert into cdc_mysql_user_log_sink
42 | select id, user_id, item_id, category_id, behavior, ts
43 | from cdc_mysql_venn_user_log;
44 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/mysql/cdc_mysql_to_print.sql:
--------------------------------------------------------------------------------
1 | -- mysql cdc to print
2 | -- creates a mysql table source
3 | drop table if exists cdc_mysql_venn_user_log;
4 | CREATE TABLE cdc_mysql_venn_user_log (
5 | id varchar
6 | ,user_id VARCHAR
7 | ,item_id VARCHAR
8 | ,category_id VARCHAR
9 | ,behavior VARCHAR
10 | ,ts TIMESTAMP(3)
11 | ,proc_time as PROCTIME()
12 | ,PRIMARY KEY (id) NOT ENFORCED
13 | ) WITH (
14 | 'connector' = 'mysql-cdc',
15 | 'hostname' = 'localhost',
16 | 'port' = '3306',
17 | 'username' = 'root',
18 | 'password' = '123456',
19 | 'database-name' = 'venn',
20 | 'table-name' = 't_feature'
21 | );
22 |
23 | -- kafka sink
24 | drop table if exists cdc_mysql_user_log_sink;
25 | CREATE TABLE cdc_mysql_user_log_sink (
26 | id varchar
27 | ,user_id VARCHAR
28 | ,item_id VARCHAR
29 | ,category_id VARCHAR
30 | ,behavior VARCHAR
31 | ,ts TIMESTAMP(3)
32 | ) WITH (
33 | 'connector' = 'print'
34 | );
35 |
36 | -- sink to kafka
37 | insert into cdc_mysql_user_log_sink
38 | select id, user_id, item_id, category_id, behavior, ts
39 | from cdc_mysql_venn_user_log;
40 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/mysql/kafka_to_mysql_demo.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | drop table if exists user_log;
3 | CREATE TABLE user_log
4 | (
5 | user_id VARCHAR,
6 | item_id VARCHAR,
7 | category_id VARCHAR,
8 | behavior VARCHAR,
9 | ts TIMESTAMP(3),
10 | WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_log'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log'
16 | ,'scan.startup.mode' = 'latest-offset'
17 | ,'format' = 'json'
18 | );
19 |
20 | -- set table.sql-dialect=hive;
21 | -- kafka sink
22 | drop table if exists mysql_table_venn_user_log_sink;
23 | CREATE TABLE mysql_table_venn_user_log_sink
24 | (
25 | user_id STRING,
26 | item_id STRING,
27 | category_id STRING,
28 | behavior STRING,
29 | ts timestamp(3)
30 | ) WITH (
31 | 'connector' = 'jdbc'
32 | ,'url' = 'jdbc:mysql://venn:3306/venn'
33 | ,'table-name' = 'user_log'
34 | ,'username' = 'root'
35 | ,'password' = '123456'
36 | ,'sink.buffer-flush.max-rows' = '100' -- default
37 | ,'sink.buffer-flush.interval' = '10s'
38 | ,'sink.max-retries' = '3'
39 | );
40 |
41 |
42 | -- streaming sql, insert into mysql table
43 | insert into mysql_table_venn_user_log_sink
44 | SELECT user_id, item_id, category_id, behavior, ts
45 | FROM user_log;
46 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/mysql/mysql_count_test.sql:
--------------------------------------------------------------------------------
1 | drop table if exists mysql_behavior_conf ;
2 | CREATE TEMPORARY TABLE mysql_behavior_conf (
3 | id int
4 | ,code STRING
5 | ,`value` STRING
6 | ,update_time TIMESTAMP(3)
7 | -- ,primary key (id) not enforced
8 | -- ,WATERMARK FOR update_time AS update_time - INTERVAL '5' SECOND
9 | ) WITH (
10 | 'connector' = 'jdbc'
11 | ,'url' = 'jdbc:mysql://localhost:3306/venn'
12 | ,'table-name' = 'lookup_join_config'
13 | ,'username' = 'root'
14 | ,'password' = '123456'
15 | ,'scan.partition.column' = 'id'
16 | ,'scan.partition.num' = '5'
17 | ,'scan.partition.lower-bound' = '5'
18 | ,'scan.partition.upper-bound' = '99999'
19 | ,'lookup.cache.max-rows' = '28'
20 | ,'lookup.cache.ttl' = '5555' -- ttl time 超过这么长时间无数据才行
21 | );
22 |
23 | -- 正常执行
24 | select count(code) from mysql_behavior_conf;
25 | -- 报错:语法错误, flink 不能解析 1 , 构造的 sql是这样的: select from mysql_behavior_config;
26 | -- select count(1) from mysql_behavior_conf;
27 | -- 报错:语法错误, flink 不能解析 * , 构造的 sql是这样的: select from mysql_behavior_config;
28 | -- select count(*) from mysql_behavior_conf;
29 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/connector/socket_to_socket.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | drop table if exists user_log;
3 | CREATE TABLE user_log (
4 | user_id VARCHAR
5 | ,item_id VARCHAR
6 | ,category_id VARCHAR
7 | ,behavior VARCHAR
8 | ,ts TIMESTAMP(3)
9 | ,WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
10 | ) WITH (
11 | 'connector' = 'socket'
12 | ,'hostname' = 'localhost'
13 | ,'port' = '12345'
14 | ,'format' = 'json'
15 | -- ,'format' = 'csv'
16 | );
17 |
18 | -- set table.sql-dialect=hive;
19 | -- kafka sink
20 | drop table if exists socket_sink;
21 | CREATE TABLE socket_sink (
22 | user_id STRING
23 | ,item_id STRING
24 | ,category_id STRING
25 | ,behavior STRING
26 | ,ts timestamp(3)
27 | ) WITH (
28 | 'connector' = 'socket'
29 | ,'hostname' = 'localhost'
30 | ,'max.retry' = '2'
31 | -- ,'retry.interval' = '2'
32 | ,'port' = '12346'
33 | ,'format' = 'csv'
34 | );
35 |
36 |
37 | -- streaming sql, insert into mysql table
38 | insert into socket_sink
39 | SELECT user_id, item_id, category_id, behavior, ts
40 | FROM user_log;
41 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/format/complex_json.sql:
--------------------------------------------------------------------------------
1 | -- flink json format, parse complex json
2 | drop table if exists user_log;
3 | CREATE TABLE user_log (
4 | user_id STRING
5 | ,item_id STRING
6 | ,category_id STRING
7 | ,sub_json ROW(sub_name STRING, password STRING, sub_json ROW(sub_name STRING, sub_pass STRING))
8 | ) WITH (
9 | 'connector' = 'kafka'
10 | ,'topic' = 'user_b'
11 | ,'properties.bootstrap.servers' = '10.201.1.132:9092'
12 | ,'properties.group.id' = 'user_log_1'
13 | ,'scan.startup.mode' = 'latest-offset'
14 | ,'format' = 'json'
15 | ,'json.ignore-parse-errors' = 'false'
16 | );
17 |
18 | -- set table.sql-dialect=hive;
19 | -- kafka sink
20 | drop table if exists mysql_table_venn_user_log_sink;
21 | CREATE TABLE mysql_table_venn_user_log_sink (
22 | user_id STRING
23 | ,item_id STRING
24 | ,category_id STRING
25 | ,sub_name STRING
26 | ,password STRING
27 | ,inner_sub_name STRING
28 | ,inner_sub_pass STRING
29 | ) WITH (
30 | 'connector' = 'print'
31 | );
32 |
33 | -- streaming sql, insert into mysql table
34 | insert into mysql_table_venn_user_log_sink
35 | SELECT user_id, item_id, category_id, sub_json.sub_name, sub_json.password, sub_json.sub_json.sub_name, sub_json.sub_json.sub_pass
36 | FROM user_log;
37 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/format/kafka_special_json_parse.sql:
--------------------------------------------------------------------------------
1 | -- parse special json
2 | CREATE TABLE t_feature (
3 | header STRING
4 | ,readModule STRING
5 | ,checkPoint STRING
6 | ,operation STRING
7 | --,location ROW(id BIGINT, code STRING, send_time STRING, rms decimal(12, 8),mean decimal(12, 8),peak decimal(12, 8),kurtosis decimal(12, 8),skewness decimal(12, 8))
8 | ,location ROW(id BIGINT, code STRING, send_time STRING, rms decimal(12, 8))
9 | ,data ROW(meta STRING, `rows` ARRAY)
10 | ,process_time as proctime()
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'test_dd'
14 | ,'properties.bootstrap.servers' = '10.201.1.131:9092'
15 | ,'properties.group.id' = 'user_log'
16 | ,'scan.startup.mode' = 'group-offsets'
17 | ,'format' = 'json'
18 | );
19 |
20 | CREATE TABLE t_sink (
21 | operation STRING
22 | ,id bigint
23 | ,code STRING
24 | ,send_time BIGINT
25 | ,rms decimal(12, 8)
26 | ,mean decimal(12, 8)
27 | ,peak decimal(12, 8)
28 | ,kurtosis decimal(12, 8)
29 | ,skewness decimal(12, 8)
30 | ,l_code STRING
31 | ) WITH (
32 | 'connector' = 'print'
33 | );
34 |
35 | INSERT INTO t_sink
36 | SELECT operation
37 | ,cast(data.`rows`[1] as bigint) id
38 | ,cast(data.`rows`[2] as string) code
39 | ,cast(data.`rows`[3] as BIGINT) send_time
40 | ,cast(data.`rows`[4] as decimal(12, 8)) rms
41 | ,cast(data.`rows`[5] as decimal(12, 8)) mean
42 | ,cast(data.`rows`[6] as decimal(12, 8)) peak
43 | ,cast(data.`rows`[7] as decimal(12, 8)) kurtosis
44 | ,cast(data.`rows`[8] as decimal(12, 8)) skewness
45 | ,location.code
46 | FROM t_feature
--------------------------------------------------------------------------------
/src/main/resources/sql/release/join/interval_join_demo.sql:
--------------------------------------------------------------------------------
1 | -- time-windowd join
2 | ---sourceTable
3 | -- 订单表
4 | CREATE TABLE t_order(
5 | order_id VARCHAR, -- 订单 id
6 | product_id VARCHAR, -- 产品 id
7 | create_time VARCHAR, -- 订单时间
8 | order_proctime as PROCTIME()
9 | ) WITH (
10 | 'connector.type' = 'kafka',
11 | 'connector.version' = 'universal',
12 | 'connector.topic' = 'order',
13 | 'connector.startup-mode' = 'latest-offset',
14 | 'connector.properties.zookeeper.connect' = 'venn:2181',
15 | 'connector.properties.bootstrap.servers' = 'venn:9092',
16 | 'update-mode' = 'append',
17 | 'format.type' = 'json',
18 | 'format.derive-schema' = 'true'
19 | );
20 | ---sourceTable
21 | --产品表
22 | CREATE TABLE t_product (
23 | product_id VARCHAR, -- 产品 id
24 | price DECIMAL(38,18), -- 价格
25 | create_time VARCHAR, -- 订单时间
26 | product_proctime as PROCTIME()
27 | ) WITH (
28 | 'connector.type' = 'kafka',
29 | 'connector.version' = 'universal',
30 | 'connector.topic' = 'shipments',
31 | 'connector.startup-mode' = 'latest-offset',
32 | 'connector.properties.zookeeper.connect' = 'venn:2181',
33 | 'connector.properties.bootstrap.servers' = 'venn:9092',
34 | 'update-mode' = 'append',
35 | 'format.type' = 'json',
36 | 'format.derive-schema' = 'true'
37 | );
38 |
39 | ---sinkTable
40 | --订单表 关联 产品表 成订购表
41 | CREATE TABLE order_detail (
42 | order_id VARCHAR,
43 | producer_id VARCHAR,
44 | price DECIMAL(38,18),
45 | order_create_time VARCHAR,
46 | product_create_time VARCHAR
47 | ) WITH (
48 | 'connector.type' = 'kafka',
49 | 'connector.version' = 'universal',
50 | 'connector.topic' = 'order_detail',
51 | 'connector.startup-mode' = 'latest-offset',
52 | 'connector.properties.zookeeper.connect' = 'venn:2181',
53 | 'connector.properties.bootstrap.servers' = 'venn:9092',
54 | 'update-mode' = 'append',
55 | 'format.type' = 'json',
56 | 'format.derive-schema' = 'true'
57 | );
58 |
59 | ---order_sink
60 | INSERT INTO order_detail(order_id, product_id, price, create_time)
61 | SELECT a.order_id, a.product_id, b.price, a.create_time, b.create_time
62 | FROM t_order a
63 | INNER JOIN t_product b ON a.product_id = b.product_id and a.order_proctime BETWEEN b.product_proctime - INTERVAL '10' MINUTE AND b.product_proctime + INTERVAL '10' MINUTE
64 | where a.order_id is not null;
65 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/join/join_demo.sql:
--------------------------------------------------------------------------------
1 | -- Regular Joins like Global Join
2 | ---sourceTable
3 | -- 订单表
4 | CREATE TABLE t_order(
5 | order_id VARCHAR, -- 订单 id
6 | product_id VARCHAR, -- 产品 id
7 | create_time VARCHAR -- 订单时间
8 | ) WITH (
9 | 'connector.type' = 'kafka'
10 | ,'connector.version' = 'universal'
11 | ,'connector.topic' = 'order'
12 | ,'connector.properties.zookeeper.connect' = 'venn:2181'
13 | ,'connector.properties.bootstrap.servers' = 'venn:9092'
14 | ,'update-mode' = 'append'
15 | ,'format.type' = 'json'
16 | ,'format.derive-schema' = 'true'
17 | );
18 | ---sourceTable
19 | --产品表
20 | CREATE TABLE t_product (
21 | product_id VARCHAR, -- 产品 id
22 | price DECIMAL(38,18), -- 价格
23 | create_time VARCHAR -- 订单时间
24 | ) WITH (
25 | 'connector.type' = 'kafka'
26 | ,'connector.version' = 'universal'
27 | ,'connector.topic' = 'shipments'
28 | ,'connector.startup-mode' = 'latest-offset'
29 | ,'connector.properties.zookeeper.connect' = 'venn:2181'
30 | ,'connector.properties.bootstrap.servers' = 'venn:9092'
31 | ,'update-mode' = 'append'
32 | ,'format.type' = 'json'
33 | ,'format.derive-schema' = 'true'
34 | );
35 |
36 | ---sinkTable
37 | --订单表 关联 产品表 成订购表
38 | CREATE TABLE order_detail (
39 | order_id VARCHAR,
40 | producer_id VARCHAR ,
41 | price DECIMAL(38,18),
42 | order_create_time VARCHAR,
43 | product_create_time VARCHAR
44 | ) WITH (
45 | 'connector.type' = 'kafka'
46 | ,'connector.version' = 'universal'
47 | ,'connector.topic' = 'order_detail'
48 | ,'connector.startup-mode' = 'latest-offset'
49 | ,'connector.properties.zookeeper.connect' = 'venn:2181'
50 | ,'connector.properties.bootstrap.servers' = 'venn:9092'
51 | ,'update-mode' = 'append'
52 | ,'format.type' = 'json'
53 | ,'format.derive-schema' = 'true'
54 | );
55 |
56 | ---order_sink
57 | INSERT INTO order_detail(order_id, product_id, price, create_time)
58 | SELECT a.order_id, a.product_id, b.price, a.create_time, b.create_time
59 | FROM t_order a
60 | INNER JOIN t_product b ON a.product_id = b.product_id
61 | where a.order_id is not null;
62 |
63 | -- 可以再尝试个 聚合的
64 | --product_total
65 | --INSERT INTO product_total
66 | --select a.product_id, sum(price) total
67 | --from order a INNER join product b where a.product_id = b.product_id
68 | --group by a.product_id
--------------------------------------------------------------------------------
/src/main/resources/sql/release/join/kafka_batch_join_mysql_demo.sql:
--------------------------------------------------------------------------------
1 | -- Scan Source: Bounded 虽然是 join, 但也是 bounded 模式,mysql 表也是一次读完,就结束了
2 | -- kafka source
3 | CREATE TABLE user_log (
4 | user_id VARCHAR
5 | ,item_id VARCHAR
6 | ,category_id VARCHAR
7 | ,behavior VARCHAR
8 | ,ts TIMESTAMP(3)
9 | ) WITH (
10 | 'connector' = 'kafka'
11 | ,'topic' = 'user_behavior'
12 | ,'properties.bootstrap.servers' = 'localhost:9092'
13 | ,'properties.group.id' = 'user_log'
14 | ,'scan.startup.mode' = 'group-offsets'
15 | ,'format' = 'json'
16 | );
17 |
18 | -- mysql source
19 | drop table if exists item ;
20 | CREATE TABLE mysql_behavior_conf (
21 | id int
22 | ,behavior VARCHAR
23 | ,behavior_map VARCHAR
24 | ,update_time TIMESTAMP(3)
25 | ,primary key (id) not enforced
26 | ) WITH (
27 | 'connector' = 'jdbc'
28 | ,'url' = 'jdbc:mysql://venn:3306/venn'
29 | ,'table-name' = 'behavior_conf'
30 | ,'username' = 'root'
31 | ,'password' = '123456'
32 | -- ,'lookup.cache.max-rows' = '1000'
33 | -- ,'lookup.cache.ttl' = '2 minute'
34 | );
35 |
36 | ---sinkTable
37 | CREATE TABLE kakfa_join_mysql_demo (
38 | user_id VARCHAR
39 | ,item_id VARCHAR
40 | ,category_id VARCHAR
41 | ,behavior VARCHAR
42 | ,behavior_map VARCHAR
43 | ,ts TIMESTAMP(3)
44 | ,primary key (user_id) not enforced
45 | ) WITH (
46 | 'connector' = 'upsert-kafka'
47 | ,'topic' = 'user_behavior_sink'
48 | ,'properties.bootstrap.servers' = 'localhost:9092'
49 | ,'properties.group.id' = 'user_log'
50 | ,'key.format' = 'json'
51 | ,'key.json.ignore-parse-errors' = 'true'
52 | ,'value.format' = 'json'
53 | ,'value.json.fail-on-missing-field' = 'false'
54 | ,'value.fields-include' = 'ALL'
55 | );
56 |
57 | ---order_sink
58 | INSERT INTO kakfa_join_mysql_demo(user_id, item_id, category_id, behavior, behavior_map, ts)
59 | SELECT a.user_id, a.item_id, a.category_id, a.behavior, b.behavior_map, a.ts
60 | FROM user_log a
61 | left join mysql_behavior_conf b ON a.behavior = b.behavior
62 | where a.behavior is not null;
63 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/join/kafka_join_mysql_demo.sql:
--------------------------------------------------------------------------------
1 | -- Lookup Source: Sync Mode
2 | -- kafka source
3 | CREATE TABLE user_log (
4 | user_id VARCHAR
5 | ,item_id VARCHAR
6 | ,category_id VARCHAR
7 | ,behavior INT
8 | ,ts TIMESTAMP(3)
9 | ,process_time as proctime()
10 | , WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_behavior'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log'
16 | ,'scan.startup.mode' = 'group-offsets'
17 | ,'format' = 'json'
18 | );
19 |
20 | -- mysql source
21 | drop table if exists mysql_behavior_conf ;
22 | CREATE TABLE mysql_behavior_conf (
23 | id int
24 | ,code VARCHAR
25 | ,map_val VARCHAR
26 | ,update_time TIMESTAMP(3)
27 | ,process_time as proctime()
28 | ,primary key (id) not enforced
29 | ,WATERMARK FOR update_time AS update_time - INTERVAL '5' SECOND
30 | ) WITH (
31 | 'connector' = 'jdbc'
32 | ,'url' = 'jdbc:mysql://venn:3306/venn'
33 | ,'table-name' = 'behavior_conf'
34 | ,'username' = 'root'
35 | ,'password' = '123456'
36 | ,'scan.partition.column' = 'id'
37 | ,'scan.partition.num' = '10'
38 | ,'scan.partition.lower-bound' = '0'
39 | ,'scan.partition.upper-bound' = '9999'
40 | ,'lookup.cache.max-rows' = '1000'
41 | ,'lookup.cache.ttl' = '2 minute'
42 | );
43 |
44 | ---sinkTable
45 | CREATE TABLE kakfa_join_mysql_demo (
46 | user_id VARCHAR
47 | ,item_id VARCHAR
48 | ,category_id VARCHAR
49 | ,behavior INT
50 | ,behavior_map VARCHAR
51 | ,ts TIMESTAMP(3)
52 | ,primary key (user_id) not enforced
53 | ) WITH (
54 | 'connector' = 'upsert-kafka'
55 | ,'topic' = 'user_behavior_sink'
56 | ,'properties.bootstrap.servers' = 'localhost:9092'
57 | ,'properties.group.id' = 'user_log'
58 | ,'key.format' = 'json'
59 | ,'key.json.ignore-parse-errors' = 'true'
60 | ,'value.format' = 'json'
61 | ,'value.json.fail-on-missing-field' = 'false'
62 | ,'value.fields-include' = 'ALL'
63 | );
64 |
65 | ---sink
66 | -- 时态表 join 和 一般的 join 的效果看起来并没有什么不一样的(user_log 表需要去掉 事件事件属性),维表都是一次性读取,然后 finish
67 | -- INSERT INTO kakfa_join_mysql_demo(user_id, item_id, category_id, behavior, behavior_map, ts)
68 | -- SELECT a.user_id, a.item_id, a.category_id, a.behavior, b.behavior_map, a.ts
69 | -- FROM user_log a
70 | -- left join mysql_behavior_conf b on a.behavior = b.id
71 | -- where a.behavior is not null;
72 |
73 | INSERT INTO kakfa_join_mysql_demo(user_id, item_id, category_id, behavior, behavior_map, ts)
74 | SELECT a.user_id, a.item_id, a.category_id, a.behavior, b.behavior_map, a.ts
75 | FROM user_log a
76 | left join mysql_behavior_conf for system_time as of a.ts as b on a.behavior = b.id
77 | where a.behavior is not null;
78 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/join/lookup_join/cust_hbase_lookup_source_demo.sql:
--------------------------------------------------------------------------------
1 | -- Lookup Source: Sync Mode
2 | -- kafka source
3 | CREATE TABLE user_log (
4 | user_id STRING
5 | ,item_id STRING
6 | ,category_id STRING
7 | ,behavior STRING
8 | ,ts TIMESTAMP(3)
9 | ,process_time as proctime()
10 | , WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_behavior'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log'
16 | --,'scan.startup.mode' = 'group-offsets'
17 | ,'scan.startup.mode' = 'latest-offset'
18 | ,'format' = 'json'
19 | );
20 |
21 | create temporary table hbase_table_config(
22 | rowkey string
23 | ,cf ROW(code string, `value` string, update_time string)
24 | ,cf2 ROW(code string, `value` string, update_time string)
25 | )WITH(
26 | 'connector' = 'cust-hbase'
27 | ,'zookeeper.quorum' = 'thinkpad:12181'
28 | ,'zookeeper.znode.parent' = '/hbase'
29 | ,'tablename' = 'hbase_table_config'
30 | ,'null-string-literal' = 'null'
31 | ,'lookup.key' = 'cf:code'
32 | -- ,'lookup.key' = 'cf:code,cf2:code'
33 | ,'lookup.cache.max.size' = '100'
34 | ,'lookup.cache.expire.ms' = '6'
35 | ,'lookup.max.retry.times' = '3'
36 | ,'timeout' = '10'
37 | )
38 | ;
39 |
40 | ---sinkTable
41 | CREATE TABLE kakfa_join_mysql_demo (
42 | user_id STRING
43 | ,item_id STRING
44 | ,category_id STRING
45 | ,behavior STRING
46 | ,rowkey STRING
47 | ,behavior_map STRING
48 | ,behavior_map2 STRING
49 | ,ts TIMESTAMP(3)
50 | ,primary key (user_id) not enforced
51 | ) WITH (
52 | 'connector' = 'print'
53 | );
54 |
55 | INSERT INTO kakfa_join_mysql_demo(user_id, item_id, category_id, behavior,rowkey, behavior_map, behavior_map2, ts)
56 | SELECT a.user_id, a.item_id, a.category_id, a.behavior,c.rowkey, c.cf.`value`, c.cf2.`value`,a.ts
57 | FROM user_log a
58 | left join hbase_table_config FOR SYSTEM_TIME AS OF a.process_time AS c
59 | -- on a.user_id = c.cf.code and a.item_id = c.cf2.code
60 | -- 必须要一个key 做关联条件,实际上不会用这个做关联条件
61 | -- 流输入端的字段使用 ',' 拼接的方式传入参数
62 | -- hbase 端通过参数 'lookup.key' = 'cf:code,cf2:code' 传入过滤的字段,两边必须的数量必须匹配
63 | -- ON a.behavior = c.rowkey
64 | ON concat(a.behavior,',',a.category_id) = c.rowkey --and a.item_id = c.cf.`code`
65 | where a.behavior is not null;
66 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/join/lookup_join/cust_mysql_lookup_source_demo.sql:
--------------------------------------------------------------------------------
1 | -- Lookup Source: Sync Mode
2 | -- kafka source
3 | CREATE TABLE user_log (
4 | user_id STRING
5 | ,item_id STRING
6 | ,category_id STRING
7 | ,behavior STRING
8 | ,ts TIMESTAMP(3)
9 | ,process_time as proctime()
10 | , WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_behavior'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log'
16 | --,'scan.startup.mode' = 'group-offsets'
17 | ,'scan.startup.mode' = 'latest-offset'
18 | ,'format' = 'json'
19 | );
20 |
21 | create temporary table mysql_behavior_conf(
22 | id int
23 | ,code STRING
24 | ,`value` STRING
25 | ,update_time TIMESTAMP(3)
26 | )WITH(
27 | 'connector' = 'cust-mysql'
28 | ,'url' = 'jdbc:mysql://localhost:3306/venn?useUnicode=true&characterEncoding=utf8&useSSL=false&allowPublicKeyRetrieval=true'
29 | ,'username' = 'root'
30 | ,'password' = '123456'
31 | ,'database' = 'venn'
32 | ,'table' = 'lookup_join_config'
33 | ,'lookup.cache.max.size' = '100'
34 | ,'lookup.cache.expire.ms' = '600000'
35 | ,'lookup.max.retry.times' = '3'
36 | ,'timeout' = '10'
37 | )
38 | ;
39 |
40 | ---sinkTable
41 | CREATE TABLE kakfa_join_mysql_demo (
42 | user_id STRING
43 | ,item_id STRING
44 | ,category_id STRING
45 | ,behavior STRING
46 | ,behavior_map STRING
47 | ,ts TIMESTAMP(3)
48 | ,primary key (user_id) not enforced
49 | ) WITH (
50 | 'connector' = 'print'
51 | );
52 |
53 | INSERT INTO kakfa_join_mysql_demo(user_id, item_id, category_id, behavior, behavior_map, ts)
54 | SELECT a.user_id, a.item_id, a.category_id, a.behavior, c.`value`, a.ts
55 | FROM user_log a
56 | left join mysql_behavior_conf FOR SYSTEM_TIME AS OF a.process_time AS c
57 | ON a.behavior = c.code -- and a.item_id = c.code
58 | where a.behavior is not null;
59 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/join/lookup_join/kafka_lookup_join_hbase_demo.sql:
--------------------------------------------------------------------------------
1 | -- Lookup Source: Sync Mode
2 | -- kafka source
3 | CREATE TABLE user_log (
4 | user_id STRING
5 | ,item_id STRING
6 | ,category_id STRING
7 | ,behavior STRING
8 | ,ts TIMESTAMP(3)
9 | ,process_time as proctime()
10 | , WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_behavior'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log'
16 | ,'scan.startup.mode' = 'latest-offset'
17 | ,'format' = 'json'
18 | );
19 |
20 | drop table if exists hbase_behavior_conf ;
21 | CREATE TEMPORARY TABLE hbase_behavior_conf (
22 | rowkey STRING
23 | ,cf ROW(item_id STRING
24 | ,category_id STRING
25 | ,behavior STRING
26 | ,ts TIMESTAMP(3))
27 | ) WITH (
28 | 'connector' = 'hbase-2.2'
29 | ,'zookeeper.quorum' = 'thinkpad:12181'
30 | ,'table-name' = 'user_log'
31 | ,'lookup.cache.max-rows' = '10000'
32 | ,'lookup.cache.ttl' = '1 minute' -- ttl time 超过这么长时间无数据才行
33 | ,'lookup.async' = 'true'
34 | );
35 |
36 | ---sinkTable
37 | CREATE TABLE kakfa_join_mysql_demo (
38 | user_id STRING
39 | ,item_id STRING
40 | ,category_id STRING
41 | ,behavior STRING
42 | ,behavior_map STRING
43 | ,ts TIMESTAMP(3)
44 | -- ,primary key (user_id) not enforced
45 | ) WITH (
46 | 'connector' = 'kafka'
47 | ,'topic' = 'user_behavior_1'
48 | ,'properties.bootstrap.servers' = 'localhost:9092'
49 | ,'properties.group.id' = 'user_log'
50 | ,'scan.startup.mode' = 'group-offsets'
51 | ,'format' = 'json'
52 | );
53 |
54 | INSERT INTO kakfa_join_mysql_demo(user_id, item_id, category_id, behavior, behavior_map, ts)
55 | SELECT a.user_id, a.item_id, a.category_id, a.behavior, c.cf.item_id, a.ts
56 | FROM user_log a
57 | left join hbase_behavior_conf FOR SYSTEM_TIME AS OF a.process_time AS c
58 | ON a.user_id = rowkey
59 | where a.behavior is not null;
60 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/join/lookup_join/kafka_lookup_join_hbase_no_rowkey_demo.sql:
--------------------------------------------------------------------------------
1 | -- Lookup Source: Sync Mode, hbase source will finish, when load all data to state
2 | -- kafka source
3 | CREATE TABLE user_log (
4 | user_id STRING
5 | ,item_id STRING
6 | ,category_id STRING
7 | ,behavior STRING
8 | ,ts TIMESTAMP(3)
9 | -- ,process_time as proctime()
10 | -- , WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_behavior'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log'
16 | ,'scan.startup.mode' = 'group-offsets'
17 | ,'format' = 'json'
18 | );
19 |
20 | drop table if exists hbase_behavior_conf ;
21 | CREATE TABLE hbase_behavior_conf (
22 | rowkey STRING
23 | ,cf ROW(item_id STRING
24 | ,category_id STRING
25 | ,behavior STRING
26 | ,ts TIMESTAMP(3))
27 | ) WITH (
28 | 'connector' = 'hbase-2.2'
29 | ,'zookeeper.quorum' = 'thinkpad:12181'
30 | ,'table-name' = 'user_log'
31 | );
32 |
33 | ---sinkTable
34 | CREATE TABLE kakfa_join_mysql_demo (
35 | user_id STRING
36 | ,item_id STRING
37 | ,category_id STRING
38 | ,behavior STRING
39 | ,behavior_map STRING
40 | ,ts TIMESTAMP(3)
41 | ,primary key (user_id) not enforced
42 | ) WITH (
43 | 'connector' = 'upsert-kafka'
44 | ,'topic' = 'user_behavior_1'
45 | ,'properties.bootstrap.servers' = 'localhost:9092'
46 | ,'properties.group.id' = 'user_log'
47 | ,'key.format' = 'csv'
48 | ,'value.format' = 'csv'
49 | );
50 |
51 | INSERT INTO kakfa_join_mysql_demo(user_id, item_id, category_id, behavior, behavior_map, ts)
52 | SELECT a.user_id, a.item_id, a.category_id, a.behavior, concat('map_', c.cf.item_id), a.ts
53 | FROM user_log a
54 | left join hbase_behavior_conf c ON a.item_id = cf.item_id
55 | where a.behavior is not null;
56 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/join/lookup_join/kafka_lookup_join_mysql_demo.sql:
--------------------------------------------------------------------------------
1 | -- Lookup Source: Sync Mode
2 | -- kafka source
3 | CREATE TABLE user_log (
4 | user_id STRING
5 | ,item_id STRING
6 | ,category_id STRING
7 | ,behavior STRING
8 | ,ts TIMESTAMP(3)
9 | ,process_time as proctime()
10 | , WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_log'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log'
16 | ,'scan.startup.mode' = 'latest-offset'
17 | ,'format' = 'json'
18 | ,'source.parallelism' = '1'
19 | );
20 |
21 | drop table if exists mysql_behavior_conf ;
22 | CREATE TEMPORARY TABLE mysql_behavior_conf (
23 | id int
24 | ,code STRING
25 | ,`value` STRING
26 | ,update_time TIMESTAMP(3)
27 | -- ,primary key (id) not enforced
28 | -- ,WATERMARK FOR update_time AS update_time - INTERVAL '5' SECOND
29 | ) WITH (
30 | 'connector' = 'jdbc'
31 | ,'url' = 'jdbc:mysql://localhost:3306/venn'
32 | ,'table-name' = 'lookup_join_config'
33 | ,'username' = 'root'
34 | ,'password' = '123456'
35 | ,'scan.partition.column' = 'id'
36 | ,'scan.partition.num' = '5'
37 | ,'scan.partition.lower-bound' = '5'
38 | ,'scan.partition.upper-bound' = '99999'
39 | ,'lookup.cache.max-rows' = '28'
40 | ,'lookup.cache.ttl' = '5555' -- ttl time 超过这么长时间无数据才行
41 | );
42 |
43 | drop table if exists mysql_behavior_conf_1 ;
44 | CREATE TEMPORARY TABLE mysql_behavior_conf_1 (
45 | id int
46 | ,code STRING
47 | ,`value` STRING
48 | ,update_time TIMESTAMP(3)
49 | -- ,primary key (id) not enforced
50 | -- ,WATERMARK FOR update_time AS update_time - INTERVAL '5' SECOND
51 | ) WITH (
52 | 'connector' = 'jdbc'
53 | ,'url' = 'jdbc:mysql://localhost:3306/venn'
54 | ,'table-name' = 'lookup_join_config_1'
55 | ,'username' = 'root'
56 | ,'password' = '123456'
57 | ,'scan.partition.column' = 'id'
58 | ,'scan.partition.num' = '5'
59 | ,'scan.partition.lower-bound' = '5'
60 | ,'scan.partition.upper-bound' = '99999'
61 | ,'lookup.cache.max-rows' = '28'
62 | ,'lookup.cache.ttl' = '5555' -- ttl time 超过这么长时间无数据才行
63 | );
64 |
65 |
66 | ---sinkTable
67 | CREATE TABLE kakfa_join_mysql_demo (
68 | user_id STRING
69 | ,item_id STRING
70 | ,category_id STRING
71 | ,behavior STRING
72 | ,behavior_map STRING
73 | ,ts TIMESTAMP(3)
74 | ,primary key (user_id) not enforced
75 | ) WITH (
76 | 'connector' = 'print'
77 | );
78 |
79 |
80 | INSERT INTO kakfa_join_mysql_demo(user_id, item_id, category_id, behavior, behavior_map, ts)
81 |
82 | ()
83 | SELECT a.user_id, a.item_id, a.category_id, a.behavior, c.`value`, a.ts
84 | FROM user_log a
85 | left join mysql_behavior_conf FOR SYSTEM_TIME AS OF a.process_time AS c
86 | ON a.behavior = c.code -- and a.item_id = c.`value`
87 | where a.behavior is not null
88 | );
89 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/join/lookup_join/kafka_lookup_join_redis.sql:
--------------------------------------------------------------------------------
1 | -- Lookup Source: Sync Mode
2 | -- kafka source
3 | CREATE TABLE user_log (
4 | user_id STRING
5 | ,item_id STRING
6 | ,category_id STRING
7 | ,behavior STRING
8 | ,ts TIMESTAMP(3)
9 | ,process_time as proctime()
10 | , WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
11 | ) WITH (
12 | 'connector' = 'kafka'
13 | ,'topic' = 'user_log'
14 | ,'properties.bootstrap.servers' = 'localhost:9092'
15 | ,'properties.group.id' = 'user_log'
16 | ,'scan.startup.mode' = 'latest-offset'
17 | ,'format' = 'json'
18 | );
19 |
20 | CREATE TEMPORARY TABLE redis_table (
21 | `key` STRING
22 | ,filed STRING
23 | ,`value` STRING
24 | ) WITH (
25 | 'connector' = 'cust-redis'
26 | ,'redis.url' = 'redis://localhost:6379?timeout=3000'
27 | ,'lookup.cache.max.size' = '28'
28 | ,'lookup.cache.expire.ms' = '3600000' -- ttl time 超过这么长时间无数据才行
29 | -- ,'pass' = '11' -- todo test
30 | );
31 |
32 | ---sinkTable
33 | CREATE TABLE kakfa_join_redis_sink (
34 | user_id STRING
35 | ,item_id STRING
36 | ,category_id STRING
37 | ,behavior STRING
38 | ,behavior_map STRING
39 | ,ts TIMESTAMP(3)
40 | ,primary key (user_id) not enforced
41 | ) WITH (
42 | 'connector' = 'print'
43 | );
44 | -- sting/list/set/zset test sql
45 | -- INSERT INTO kakfa_join_redis_sink(user_id, item_id, category_id, behavior, behavior_map, ts)
46 | -- SELECT a.user_id, a.item_id, a.category_id, a.behavior, b.`value`, a.ts
47 | -- FROM user_log a
48 | -- left join redis_table FOR SYSTEM_TIME AS OF a.process_time AS b
49 | -- ON a.behavior = b.`key`
50 | -- where a.behavior is not null;
51 |
52 | CREATE TABLE kakfa_join_redis_sink_1 (
53 | user_id STRING
54 | ,item_id STRING
55 | ,category_id STRING
56 | ,behavior STRING
57 | ,behavior_key STRING
58 | ,behavior_map STRING
59 | ,ts TIMESTAMP(3)
60 | ,primary key (user_id) not enforced
61 | ) WITH (
62 | 'connector' = 'print'
63 | )
64 | ;
65 |
66 |
67 | -- hash multiple input
68 | INSERT INTO kakfa_join_redis_sink_1(user_id, item_id, category_id, behavior, behavior_key,behavior_map, ts)
69 | SELECT a.user_id, a.item_id, a.category_id, a.behavior,b.filed, b.`value`, a.ts
70 | FROM user_log a
71 | left join redis_table FOR SYSTEM_TIME AS OF a.process_time AS b
72 | ON a.behavior = b.key
73 | where a.behavior is not null;
74 |
75 | -- INSERT INTO kakfa_join_redis_sink_1(user_id, item_id, category_id, behavior, behavior_key,behavior_map, ts)
76 | -- SELECT a.user_id, a.item_id, a.category_id, a.behavior,b.filed, b.`value`, a.ts
77 | -- FROM user_log a
78 | -- left join redis_table FOR SYSTEM_TIME AS OF a.process_time AS b
79 | -- ON a.behavior = b.key and a.item = b.filed
80 | -- where a.behavior is not null;
--------------------------------------------------------------------------------
/src/main/resources/sql/release/join/stream_table_join.sql:
--------------------------------------------------------------------------------
1 | ---sourceTable
2 | CREATE TABLE user_log (
3 | user_id VARCHAR
4 | ,item_id VARCHAR
5 | ,category_id VARCHAR
6 | ,behavior VARCHAR
7 | ,ts TIMESTAMP(3)
8 | ) WITH (
9 | 'connector.type' = 'kafka'
10 | ,'connector.version' = 'universal'
11 | ,'connector.topic' = 'user_behavior'
12 | ,'connector.startup-mode' = 'earliest-offset'
13 | ,'connector.properties.zookeeper.connect' = 'venn:2181'
14 | ,'connector.properties.bootstrap.servers' = 'venn:9092'
15 | ,'update-mode' = 'append'
16 | ,'format.type' = 'json'
17 | ,'format.derive-schema' = 'true'
18 | );
19 |
20 | ---sinkTable
21 | CREATE TABLE user_log_sink (
22 | user_id VARCHAR
23 | ,item_id VARCHAR
24 | ,category_id VARCHAR
25 | ,behavior VARCHAR
26 | ,ts TIMESTAMP(3)
27 | ,b_item_id VARCHAR
28 | ,b_category_id VARCHAR
29 | ,b_behavior VARCHAR
30 | ,b_ts VARCHAR
31 | ) WITH (
32 | 'connector.type' = 'kafka'
33 | ,'connector.version' = 'universal'
34 | ,'connector.topic' = 'user_behavior_sink'
35 | ,'connector.startup-mode' = 'earliest-offset'
36 | ,'connector.properties.zookeeper.connect' = 'venn:2181'
37 | ,'connector.properties.bootstrap.servers' = 'venn:9092'
38 | ,'update-mode' = 'append'
39 | ,'format.type' = 'json'
40 | ,'format.derive-schema' = 'true'
41 | );
42 |
43 | ---sql
44 | insert into user_log_sink
45 | select a.user_id ,a.item_id
46 | ,a.category_id
47 | ,a.behavior
48 | ,a.ts
49 | ,cf.item_id b_item_id
50 | ,cf.category_id b_category_id
51 | ,cf.behavior b_behavior
52 | ,cf.ts b_ts
53 | from user_log a -- inner join
54 | ,lateral table(joinHBaseTable(a.user_id))
--------------------------------------------------------------------------------
/src/main/resources/sql/release/udf/join_hbase_no_rowkey_demo.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | CREATE TABLE user_log (
3 | user_id STRING
4 | ,item_id STRING
5 | ,category_id STRING
6 | ,behavior STRING
7 | ,ts TIMESTAMP(3)
8 | ,process_time as proctime()
9 | -- , WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
10 | ) WITH (
11 | 'connector' = 'kafka'
12 | ,'topic' = 'user_behavior'
13 | ,'properties.bootstrap.servers' = 'localhost:9092'
14 | ,'properties.group.id' = 'user_log'
15 | ,'scan.startup.mode' = 'group-offsets'
16 | ,'format' = 'json'
17 | );
18 |
19 | ---sinkTable
20 | CREATE TABLE join_hbbase_sink (
21 | user_id STRING
22 | ,item_id STRING
23 | ,category_id STRING
24 | ,behavior STRING
25 | ,ts TIMESTAMP(3)
26 | ,rowkey STRING
27 | ,c1 STRING
28 | ,c2 STRING
29 | ,c3 STRING
30 | ,c4 STRING
31 | ,primary key (user_id) not enforced
32 | ) WITH (
33 | 'connector' = 'upsert-kafka'
34 | ,'topic' = 'user_behavior_1'
35 | ,'properties.bootstrap.servers' = 'localhost:9092'
36 | ,'properties.group.id' = 'user_log'
37 | ,'key.format' = 'csv'
38 | ,'value.format' = 'csv'
39 | );
40 |
41 | INSERT INTO join_hbbase_sink
42 | SELECT a.user_id, a.item_id, a.category_id, a.behavior, a.ts, t2.col[1], t2.col[2], t2.col[3], t2.col[4], t2.col[5]
43 | FROM user_log a
44 | -- left join lateral table(udf_join_hbase_non_rowkey_no_cache(item_id)) as t2(col) on true
45 | left join lateral table(udf_join_hbase_non_rowkey_cache(item_id)) as t2(col) on true
46 | where a.item_id is not null
47 | -- and t2.col[1] = a.item_id -- 返回多条数据可以在 where 条件后面过滤
48 | ;
49 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/udf/parse_complex_json.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | CREATE TABLE t_json (
3 | json string
4 | ) WITH (
5 | 'connector' = 'kafka'
6 | ,'topic' = 'user_behavior'
7 | ,'properties.bootstrap.servers' = 'localhost:9092'
8 | ,'properties.group.id' = 'user_log'
9 | ,'format' = 'csv'
10 | ,'csv.field-delimiter' = U&'\0001'
11 | ,'csv.ignore-parse-errors' = 'true'
12 | ,'csv.allow-comments' = 'true'
13 | );
14 |
15 | CREATE TABLE t_json_sink (
16 | user_id string
17 | ,sub_name_1 string
18 | ,sub_name_2 string
19 | ,sub_name_3 string
20 | ,sub_name_4 string
21 | ,sub_name_5 string
22 | ) WITH (
23 | 'connector' = 'print'
24 | );
25 | -- sink
26 | insert into t_json_sink
27 | select T.arr[1], T1.arr[1], T2.arr[1], T3.arr[1], T4.arr[1], T5.arr[1]
28 | from t_json a
29 | LEFT JOIN LATERAL TABLE(udf_parse_json(json, 'user_id', 'sub_json')) AS T(arr) ON TRUE
30 | LEFT JOIN LATERAL TABLE(udf_parse_json(T.arr[2], 'sub_name', 'sub_json')) AS T1(arr) ON TRUE
31 | LEFT JOIN LATERAL TABLE(udf_parse_json(T1.arr[2], 'sub_name', 'sub_json')) AS T2(arr) ON TRUE
32 | LEFT JOIN LATERAL TABLE(udf_parse_json(T2.arr[2], 'sub_name', 'sub_json')) AS T3(arr) ON TRUE
33 | LEFT JOIN LATERAL TABLE(udf_parse_json(T3.arr[2], 'sub_name', 'sub_json')) AS T4(arr) ON TRUE
34 | LEFT JOIN LATERAL TABLE(udf_parse_json(T4.arr[2], 'sub_name', 'sub_json')) AS T5(arr) ON TRUE
35 | ;
36 |
37 |
--------------------------------------------------------------------------------
/src/main/resources/sql/release/udf/parse_complex_json_1.sql:
--------------------------------------------------------------------------------
1 | -- kafka source
2 | CREATE TABLE t_json (
3 | json string
4 | ) WITH (
5 | 'connector' = 'kafka'
6 | ,'topic' = 'user_behavior'
7 | ,'properties.bootstrap.servers' = 'localhost:9092'
8 | ,'properties.group.id' = 'user_log'
9 | ,'format' = 'csv'
10 | ,'csv.field-delimiter' = U&'\0001'
11 | ,'csv.ignore-parse-errors' = 'true'
12 | ,'csv.allow-comments' = 'true'
13 | );
14 |
15 | CREATE TABLE t_json_sink (
16 | category_id string
17 | ,user_id string
18 | ,item_id string
19 | ,sort_col int
20 | ,username string
21 | ,password string
22 | ,doub double
23 | ,sub_name string
24 | -- ,sub_pass string
25 | ) WITH (
26 | 'connector' = 'print'
27 | );
28 | -- sink
29 | insert into t_json_sink
30 | select category_id, user_id, item_id, cast(sort_col as int) sort_col, username, password, cast(doub as double) doub,sub_name --,sub_pass
31 | from t_json a
32 | LEFT JOIN LATERAL TABLE(udf_parse_json(json, 'category_id', 'user_id', 'item_id', 'sort_col', 'sub_json')) AS T(category_id, user_id, item_id, sort_col, sub_json) ON TRUE
33 | LEFT JOIN LATERAL TABLE(udf_parse_json(sub_json, 'username', 'password', 'doub', 'sub_json_1')) AS T1(username, password, doub, sub_json_1) ON TRUE
34 | LEFT JOIN LATERAL TABLE(udf_parse_json(sub_json_1, 'sub_name')) AS T2(sub_name) ON TRUE
35 | ;
36 |
37 |
--------------------------------------------------------------------------------
/src/main/resources/sql/starrocks/datagen_to_starrocks.sql:
--------------------------------------------------------------------------------
1 | -- datagen to starrocks
2 | drop table if exists datagen_key_source;
3 | create table if not exists datagen_key_source1 (
4 | `col1` string
5 | ,`col2` string
6 | ,`col3` string
7 | ,`col4` string
8 | ) WITH (
9 | 'connector' = 'datagen'
10 | ,'rows-per-second' = '20000'
11 | ,'number-of-rows' = '100000000'
12 | ,'fields.col1.kind' = 'random'
13 | ,'fields.col2.kind' = 'random'
14 | ,'fields.col3.kind' = 'random'
15 | ,'fields.col4.kind' = 'random'
16 | ,'fields.col1.length' = '20'
17 | ,'fields.col2.length' = '10'
18 | ,'fields.col3.length' = '10'
19 | ,'fields.col4.length' = '10'
20 | );
21 |
22 |
23 | drop table if exists starrocks_sink;
24 | create table if not exists starrocks_sink (
25 | `col1` string
26 | ,`col2` string
27 | ,`col3` string
28 | ,`col4` string
29 | ,PRIMARY key(col1) NOT ENFORCED
30 | ) WITH (
31 | 'connector'='starrocks',
32 | 'load-url'='10.201.0.228:28030;10.201.0.229:28030;10.201.0.230:28030',
33 | 'jdbc-url'='jdbc:mysql://10.201.0.228:29030,10.201.0.229:29030,10.201.0.230:29030',
34 | 'username'='root',
35 | 'password'='123456',
36 | 'database-name'='shell',
37 | 'table-name'='datagen_key'
38 | );
39 |
40 | insert into starrocks_sink
41 | select * from datagen_key_source ;
--------------------------------------------------------------------------------
/src/main/resources/sql/starrocks/kafka_join_starrocks.sql:
--------------------------------------------------------------------------------
1 | --- lookup join starrocks
2 |
3 | -- kafka source
4 | drop table if exists user_log;
5 | CREATE TABLE user_log
6 | (
7 | `event_time` TIMESTAMP(3) METADATA FROM 'timestamp' VIRTUAL, -- from Debezium format
8 | `partition_id` BIGINT METADATA FROM 'partition' VIRTUAL, -- from Kafka connector
9 | `offset` BIGINT METADATA VIRTUAL, -- from Kafka connector
10 | user_id VARCHAR,
11 | item_id VARCHAR,
12 | category_id VARCHAR,
13 | behavior VARCHAR,
14 | proc_time as PROCTIME(),
15 | ts TIMESTAMP(3),
16 | WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
17 | ) WITH (
18 | 'connector' = 'kafka'
19 | ,'topic' = 'user_log'
20 | ,'properties.bootstrap.servers' = 'localhost:9092'
21 | ,'properties.group.id' = 'user_log'
22 | ,'scan.startup.mode' = 'earliest-offset'
23 | ,'format' = 'json'
24 | );
25 |
26 | drop table if exists starrocks_user_log ;
27 | CREATE TEMPORARY TABLE starrocks_user_log (
28 | behavior string
29 | ,coun bigint
30 |
31 | ) WITH (
32 | 'connector' = 'cust-starrocks'
33 | ,'url' = 'jdbc:mysql://10.201.0.230:19030/hive'
34 | ,'sql' = 'select behavior,count(1) coun from hive.user_log where behavior = ? group by behavior'
35 | ,'username' = 'root'
36 | ,'password' = '123456'
37 | ,'lookup.cache.max.size' = '28'
38 | ,'lookup.cache.expire.ms' = '5555' -- ttl time 超过这么长时间无数据才行
39 | );
40 |
41 |
42 | -- set table.sql-dialect=hive;
43 | -- kafka sink
44 | drop table if exists user_log_sink;
45 | CREATE TABLE user_log_sink
46 | (
47 | user_id VARCHAR,
48 | item_id VARCHAR,
49 | category_id VARCHAR,
50 | behavior VARCHAR,
51 | ts TIMESTAMP(3),
52 | coun bigint
53 | ) WITH (
54 | 'connector' = 'print'
55 | -- ,'topic' = 'user_log_sink_6'
56 | -- ,'properties.bootstrap.servers' = 'localhost:9092'
57 | -- ,'properties.group.id' = 'user_log'
58 | -- ,'scan.startup.mode' = 'latest-offset'
59 | -- ,'format' = 'json'
60 | );
61 |
62 |
63 | -- streaming sql, insert into mysql table
64 | insert into user_log_sink
65 | select user_id, item_id, category_id, t1.behavior, ts, coun
66 | from user_log t1
67 | left join starrocks_user_log FOR SYSTEM_TIME AS OF t1.proc_time AS t2 on t1.behavior = t2.behavior
68 |
--------------------------------------------------------------------------------
/src/main/resources/sql/starrocks/kafka_to_starrocks.sql:
--------------------------------------------------------------------------------
1 | drop table if exists user_log;
2 | CREATE TABLE user_log (
3 | user_id VARCHAR
4 | ,item_id VARCHAR
5 | ,category_id VARCHAR
6 | ,behavior VARCHAR
7 | ,proc_time as PROCTIME()
8 | ,ts TIMESTAMP(3)
9 | ,WATERMARK FOR ts AS ts - INTERVAL '5' SECOND
10 | ) WITH (
11 | 'connector' = 'kafka'
12 | ,'topic' = 'user_log'
13 | ,'properties.bootstrap.servers' = 'localhost:9092'
14 | ,'properties.group.id' = 'user_log'
15 | ,'scan.startup.mode' = 'latest-offset'
16 | ,'format' = 'json'
17 | );
18 |
19 |
20 | drop table if exists starrocks_sink;
21 | create table if not exists starrocks_sink (
22 | `col1` string
23 | ,`col2` string
24 | ,`col3` string
25 | ,`col4` string
26 | ,PRIMARY key(col1) NOT ENFORCED
27 | ) WITH (
28 | 'connector'='starrocks',
29 | 'load-url'='10.201.0.230:28030',
30 | 'jdbc-url'='jdbc:mysql://10.201.0.230:29030',
31 | 'username'='root',
32 | 'password'='123456',
33 | 'database-name'='test',
34 | 'table-name'='datagen_key',
35 | 'sink.buffer-flush.max-rows' = '1000000',
36 | 'sink.buffer-flush.max-bytes' = '300000000',
37 | 'sink.buffer-flush.interval-ms' = '5000'
38 | ,'sink.properties.format' = 'json'
39 | ,'sink.properties.strip_outer_array' = 'true'
40 | );
41 |
42 | insert into starrocks_sink
43 | select user_id, item_id, category_id, behavior from user_log ;
44 |
--------------------------------------------------------------------------------
/src/main/resources/sql/starrocks/starrocks_agg_demo.sql:
--------------------------------------------------------------------------------
1 | -- starrocks to print
2 | -- exception: Exception in thread "main" java.lang.NoSuchMethodError:
3 | -- org.apache.flink.table.utils.TableSchemaUtils.projectSchema(Lorg/apache/flink/table/api/TableSchema;[[I)Lorg/apache/flink/table/api/TableSchema;
4 | -- fix: forget save a flink-connector_2.11-1.14.4.jar to project lib
5 | CREATE TABLE user_log
6 | (
7 | `col1` string
8 | ,`col2` string
9 | ,`col3` string
10 | ,`col4` string
11 | ,proc_time as PROCTIME()
12 | ) WITH (
13 | -- 'connector' = 'datagen'
14 | -- ,'rows-per-second' = '20000'
15 | -- ,'number-of-rows' = '100000000'
16 | -- ,'fields.col1.kind' = 'random'
17 | -- ,'fields.col2.kind' = 'random'
18 | -- ,'fields.col3.kind' = 'random'
19 | -- ,'fields.col4.kind' = 'random'
20 | -- ,'fields.col1.length' = '20'
21 | -- ,'fields.col2.length' = '10'
22 | -- ,'fields.col3.length' = '10'
23 | -- ,'fields.col4.length' = '10'
24 | 'connector' = 'jdbc'
25 | ,'url' = 'jdbc:mysql://10.201.0.230:29030/shell'
26 | ,'table-name' = 'datagen_key'
27 | -- ,'url' = 'jdbc:mysql://localhost:3306/venn'
28 | -- ,'table-name' = 'user_log_datagen'
29 | ,'username' = 'root'
30 | ,'password' = '123456'
31 | );
32 | CREATE TABLE user_log_sink
33 | (
34 | `col1` string
35 | ,`col2` string
36 | ,`col3` string
37 | ,`col4` string
38 | ,cnt bigint
39 | ) WITH (
40 | 'connector' = 'print'
41 | -- 'connector'='starrocks',
42 | -- 'load-url'='10.201.0.230:28030',
43 | -- 'jdbc-url'='jdbc:mysql://10.201.0.230:29030',
44 | -- 'username'='root',
45 | -- 'password'='123456',
46 | -- 'database-name'='test',
47 | -- 'table-name'='datagen_key',
48 | -- 'sink.buffer-flush.max-rows' = '1000000',
49 | -- 'sink.buffer-flush.max-bytes' = '300000000',
50 | -- 'sink.buffer-flush.interval-ms' = '5000'
51 | -- ,'sink.properties.format' = 'json'
52 | -- ,'sink.properties.strip_outer_array' = 'true'
53 | );
54 |
55 | -- insert into user_log_sink
56 | -- select substring(col1,1,5),'' col2,'' col3,'' col4, count(1)
57 | -- from user_log
58 | -- group by substring(col1,1,5)
59 |
60 | insert into user_log_sink
61 | select date_format(window_start,'yyyy-MM-dd HH:mm:ss'),date_format(window_end,'yyyy-MM-dd HH:mm:ss'),substring(col1,1,5),'' col4, count(1)
62 | from TABLE(TUMBLE(TABLE user_log, DESCRIPTOR(proc_time), INTERVAL '2' second)
63 | )
64 | group by window_start,window_end,substring(col1,1,5)
--------------------------------------------------------------------------------
/src/main/resources/sql/starrocks/starrocks_to_print.sql:
--------------------------------------------------------------------------------
1 | drop table if exists user_log ;
2 | CREATE TABLE user_log (
3 | coun bigint
4 | ) WITH (
5 | 'connector' = 'cust-starrocks'
6 | ,'url' = 'jdbc:mysql://10.201.0.230:19030/hive'
7 | ,'sql' = 'select count(1) coun from hive.user_log'
8 | ,'username' = 'root'
9 | ,'password' = '123456'
10 | );
11 |
12 | create table user_log_sink(
13 | coun bigint
14 | )with (
15 | 'connector' = 'print'
16 | )
17 | ;
18 |
19 | insert into user_log_sink
20 | select coun from user_log;
21 |
--------------------------------------------------------------------------------
/src/main/resources/sqlSubmit.properties:
--------------------------------------------------------------------------------
1 | ## filesystem & rocksdb
2 | state.backend=filesystem
3 | checkpoint.dir=file:///opt/data/checkpoint
4 | #checkpoint.dir=hdfs:///tmp/checkpoint
5 | ## second
6 | checkpoint.interval=60
7 | checkpoint.timeout=600
8 | ## for top n/deduplication state ttl
9 | table.exec.state.ttl=900s
10 | ## Deprecated state retention: min,max second, interval must greater than 5 minute
11 | #state.retention.min.time=600
12 | #state.retention.max.time=900
13 | # state.ratention.min=state.retention.duration,state.ratention.max=state.retention.duration * 3 / 2
14 | state.retention.duration=600
15 | table.exec.resource.default-parallelism=1
16 | job.name=sqlSubmitJob
17 | hive.catalog.name=myHive
18 | hive.default.database=flink
19 | hive.config.path=file:///Users/venn/git/sqlSubmit/src/main/resources
20 | zookeeper.quorum=venn:2181
21 | zookeeper.znode.parent=/hbase-unsecure
22 | kafka.bootstrap.server=venn:9092
23 | #table.exec.mini-batch.enabled=true
24 | #table.exec.mini-batch.allow-latency=1 min
25 | #table.exec.mini-batch.size=100
26 | ## for disable kafka sql source chain
27 | #table.exec.source.force-break-chain=true
28 | ## redis url
29 | redis.url=redis://localhost
30 | ## lookup join parallelism
31 | cust_lookup_join_parallelism_mysql_behavior_conf=2
32 |
--------------------------------------------------------------------------------
/src/main/scala/com/rookie/submit/common/Common.scala:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.common
2 |
3 | import java.io.File
4 |
5 | import org.apache.flink.api.java.utils.ParameterTool
6 | import com.rookie.submit.common.Constant._
7 | import org.slf4j.LoggerFactory
8 |
9 | object Common {
10 |
11 | val LOG = LoggerFactory.getLogger("Common")
12 | var path: String = DEFAULT_CONFIG_FILE
13 | var jobName: String = _
14 |
15 | /**
16 | * 1. add sqlSubmit.properties to parameterTool
17 | * 2. add job.prop.file content properties to parameterTool (if file exists)
18 | * 3. add input parameter to parameterTool (if exists)
19 | *
20 | * @param args program input param
21 | * @return
22 | */
23 | def init(args: Array[String]): ParameterTool = {
24 |
25 | // input parameter
26 | val inputPara = ParameterTool.fromArgs(args)
27 | if (!inputPara.has(INPUT_SQL_FILE_PARA)) {
28 | LOG.info("please input sql file. like : --sql sql/demo.sql")
29 | System.exit(-1)
30 | }
31 | // load properties
32 | if (!new File(path).exists()) {
33 | LOG.info(DEFAULT_CONFIG_FILE + " not exists, find in class path")
34 | path = Common.getClass.getClassLoader.getResource(DEFAULT_CONFIG_FILE).getPath //.substring(1)
35 | }
36 |
37 | // load default properties
38 | // load default properties : sqlSubmit.properties
39 | val defaultPropFile = ParameterTool.fromPropertiesFile(path)
40 |
41 | // load input job properties
42 | var inputJobPropFile: ParameterTool = null
43 | if (inputPara.has(INPUT_JOB_PROP_FILE_PARA)) {
44 | inputJobPropFile = ParameterTool.fromPropertiesFile(inputPara.get(INPUT_JOB_PROP_FILE_PARA))
45 | }
46 |
47 | var parameterTool: ParameterTool = null
48 |
49 | if (null != inputJobPropFile) {
50 | // if inputJobPropFile exists
51 | // first putAll inputJobPropFile to defaultPropFile, then put inputPara to defaultPropFile, return defaultPropFile
52 | parameterTool = defaultPropFile.mergeWith(inputJobPropFile).mergeWith(inputPara)
53 | } else {
54 | // if no exists inputJobPropFile
55 | // just put inputPara to defaultPropFile, return defaultPropFile
56 | parameterTool = defaultPropFile.mergeWith(inputPara)
57 | }
58 | // parse job nam
59 | jobName = parameterTool.get(INPUT_SQL_FILE_PARA)
60 | // split prefix
61 | if (jobName.contains("/")) {
62 | jobName = jobName.substring(jobName.lastIndexOf("/") + 1)
63 | }
64 | // suffix
65 | if (jobName.contains(".")) {
66 | jobName = jobName.substring(0, jobName.indexOf("."))
67 | }
68 | if (jobName == null || jobName.isEmpty) {
69 | jobName = parameterTool.get(Constant.JOB_NAME)
70 | }
71 |
72 | parameterTool
73 | }
74 |
75 | }
76 |
--------------------------------------------------------------------------------
/src/main/scala/com/rookie/submit/common/Constant.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.common;
2 |
3 | public class Constant {
4 |
5 | public final static String DEFAULT_CONFIG_FILE = "sqlSubmit.properties";
6 | public final static String INPUT_SQL_FILE_PARA = "sql";
7 | public final static String INPUT_JOB_PROP_FILE_PARA = "job.prop.file";
8 | public final static String HIVE_CATALOG_NAME = "hive.catalog.name";
9 | public final static String HIVE_DEFAULT_DATABASE = "hive.default.database";
10 | public final static String HIVE_CONFIG_PATH = "hive.config.path";
11 | public final static String HIVE_VERSION = "hive.version";
12 | public final static String ZOOKEEPER_QUORUM = "zookeeper.quorum";
13 | public final static String ZOOKEEPER_ZNODE_PARENT = "zookeeper.znode.parent";
14 | public final static String KAFKA_BOOTSTRAP_SERVER = "kafka.bootstrap.server";
15 | public final static String STATE_RETENTION_MIN_TIME = "state.retention.min.time";
16 | public final static String STATE_RETENTION_MAX_TIME = "state.retention.max.time";
17 | public final static String STATE_RETENTION_DURATION = "state.retention.duration";
18 |
19 | public final static String TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM = "table.exec.resource.default-parallelism";
20 | public final static String JOB_NAME = "job.name";
21 | public final static String TABLE_EXEC_SOURCE_FORCE_BREAK_CHAIN = "table.exec.source.force-break-chain";
22 |
23 | public final static String STATE_BACKEND = "state.backend";
24 | public final static String CHECKPOINT_DIR = "checkpoint.dir";
25 | public final static String CHECKPOINT_INTERVAL = "checkpoint.interval";
26 | public final static String CHECKPOINT_TIMEOUT = "checkpoint.timeout";
27 | public final static String TABLE_EXEC_MINI_BATCH_ENABLE = "table.exec.mini-batch.enabled";
28 | public final static String TABLE_EXEC_MINI_BATCH_ALLOW_LATENCY = "table.exec.mini-batch.allow-latency";
29 | public final static String TABLE_EXEC_MINI_BATCH_SIZE = "table.exec.mini-batch.size";
30 | public final static String TABLE_EXEC_STATE_TTL = "table.exec.state.ttl";
31 |
32 | public final static String URL = "jdbc:mysql://localhost:3306/venn";
33 | public final static String USERNAME = "root";
34 | public final static String PASSWORD = "123456";
35 |
36 | // redis
37 | public final static String REDIS_URL = "redis.url";
38 |
39 | }
40 |
--------------------------------------------------------------------------------
/src/main/scala/com/rookie/submit/connector/kafka/KafkaUpsertTableSinkFactory.java:
--------------------------------------------------------------------------------
1 | ///*
2 | // * Licensed to the Apache Software Foundation (ASF) under one or more
3 | // * contributor license agreements. See the NOTICE file distributed with
4 | // * this work for additional information regarding copyright ownership.
5 | // * The ASF licenses this file to You under the Apache License, Version 2.0
6 | // * (the "License"); you may not use this file except in compliance with
7 | // * the License. You may obtain a copy of the License at
8 | // *
9 | // * http://www.apache.org/licenses/LICENSE-2.0
10 | // *
11 | // * Unless required by applicable law or agreed to in writing, software
12 | // * distributed under the License is distributed on an "AS IS" BASIS,
13 | // * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14 | // * See the License for the specific language governing permissions and
15 | // * limitations under the License.
16 | // */
17 | //
18 | //package com.rookie.submit.connector.kafka;
19 | //
20 | //import org.apache.flink.api.common.serialization.DeserializationSchema;
21 | //import org.apache.flink.api.common.serialization.SerializationSchema;
22 | //import org.apache.flink.streaming.connectors.kafka.KafkaTableSource;
23 | //import org.apache.flink.streaming.connectors.kafka.KafkaTableSourceBase;
24 | //import org.apache.flink.streaming.connectors.kafka.KafkaTableSourceSinkFactoryBase;
25 | //import org.apache.flink.streaming.connectors.kafka.config.StartupMode;
26 | //import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition;
27 | //import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
28 | //import org.apache.flink.table.api.TableSchema;
29 | //import org.apache.flink.table.descriptors.KafkaValidator;
30 | //import org.apache.flink.table.sources.RowtimeAttributeDescriptor;
31 | //import org.apache.flink.types.Row;
32 | //
33 | //import java.util.List;
34 | //import java.util.Map;
35 | //import java.util.Optional;
36 | //import java.util.Properties;
37 | //
38 | ///**
39 | // * Factory for creating configured instances of {@link KafkaTableSource}.
40 | // */
41 | //public class KafkaUpsertTableSinkFactory extends KafkaUpsertTableSinkFactoryBase {
42 | //
43 | // @Override
44 | // protected String kafkaVersion() {
45 | // return KafkaValidator.CONNECTOR_VERSION_VALUE_UNIVERSAL;
46 | // }
47 | //
48 | // @Override
49 | // protected boolean supportsKafkaTimestamps() {
50 | // return true;
51 | // }
52 | //
53 | //
54 | // @Override
55 | // protected KafkaUpsertTableSink createKafkaTableSink(
56 | // TableSchema schema,
57 | // String topic,
58 | // Properties properties,
59 | // Optional> partitioner,
60 | // SerializationSchema serializationSchema) {
61 | //
62 | // return new KafkaUpsertTableSink(
63 | // schema,
64 | // topic,
65 | // properties,
66 | // partitioner,
67 | // serializationSchema);
68 | // }
69 | //}
70 |
--------------------------------------------------------------------------------
/src/main/scala/com/rookie/submit/udf/Decode.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.udf;
2 |
3 | import org.apache.flink.api.common.typeinfo.TypeInformation;
4 | import org.apache.flink.api.common.typeinfo.Types;
5 | import org.apache.flink.table.annotation.DataTypeHint;
6 | import org.apache.flink.table.annotation.FunctionHint;
7 | import org.apache.flink.table.api.DataTypes;
8 | import org.apache.flink.table.catalog.DataTypeFactory;
9 | import org.apache.flink.table.functions.ScalarFunction;
10 | import org.apache.flink.table.types.inference.TypeInference;
11 | import org.slf4j.Logger;
12 | import org.slf4j.LoggerFactory;
13 |
14 | import static org.apache.flink.table.types.inference.TypeStrategies.explicit;
15 |
16 | public class Decode extends ScalarFunction {
17 |
18 | private static Logger logger = LoggerFactory.getLogger(Decode.class);
19 |
20 | public Decode() {
21 | }
22 |
23 | public Object eval(Object... obj) throws Exception {
24 | int size = obj.length;
25 | if (size % 2 != 0) {
26 | logger.error("decode input parameter must pair.");
27 | throw new Exception("decode input parameter must pair.");
28 | }
29 | for (int i = 1; i < size; i += 2) {
30 | if (String.valueOf(obj[0]).equals(String.valueOf(obj[i]))) {
31 | return obj[i + 1];
32 | }
33 | }
34 | return obj[size - 1];
35 | }
36 |
37 |
38 | @Override
39 | public TypeInference getTypeInference(DataTypeFactory typeFactory) {
40 | return TypeInference
41 | .newBuilder()
42 | .outputTypeStrategy(explicit(DataTypes.STRING()))
43 | .build();
44 | }
45 |
46 | public static void main(String[] args) throws Exception {
47 | Decode decode = new Decode();
48 | Object[] arr = {null, "a", "1", "b", "2", 3};
49 | System.out.println(decode.eval(arr));
50 | }
51 | }
52 |
--------------------------------------------------------------------------------
/src/main/scala/com/rookie/submit/udf/RegisterUdf.scala:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.udf
2 |
3 | import com.rookie.submit.common.Constant
4 | import com.rookie.submit.udaf.math.Median
5 | import com.rookie.submit.udaf.{BloomFilter, RedisUv}
6 | import com.rookie.submit.udtf.UdtfTimer
7 | import org.apache.flink.api.java.utils.ParameterTool
8 | import org.apache.flink.table.api.bridge.scala.StreamTableEnvironment
9 |
10 | object RegisterUdf {
11 |
12 | def registerUdf(tabEnv: StreamTableEnvironment, paraTool: ParameterTool) = {
13 |
14 | // udf
15 | tabEnv.createTemporarySystemFunction("udf_decode", new Decode)
16 | tabEnv.createTemporarySystemFunction("udf_date_add", new DateAdd)
17 |
18 | // udtf
19 | tabEnv.createTemporarySystemFunction("udf_split", new SplitFunction)
20 | tabEnv.createTemporarySystemFunction("udf_parse_json", new ParseJson)
21 | tabEnv.createTemporarySystemFunction("udf_timer", new UdtfTimer(1000))
22 | // 可以基于配置动态生成UDF
23 | // join hbase table, first qualify is join key
24 | tabEnv.createTemporarySystemFunction("udf_join_hbase_non_rowkey_no_cache", new JoinHbaseNonRowkeyNoCache("cf", "c1,c2,c3,c4,c5,c6,c7,c8,c9,c10"))
25 | tabEnv.createTemporarySystemFunction("udf_join_hbase_non_rowkey_cache", new JoinHbaseNonRowkeyCache("cf", "c1,c2,c3,c4,c5,c6,c7,c8,c9,c10", 600, 10000))
26 |
27 | // udaf
28 | tabEnv.createTemporarySystemFunction("udaf_uv_count", classOf[BloomFilter]);
29 | tabEnv.createTemporarySystemFunction("udaf_redis_uv_count", new RedisUv(paraTool.get(Constant.REDIS_URL), "user_log_uv"));
30 | // env.createTemporarySystemFunction("udaf_redis_uv_count", new JedisRedisUv("localhost", 6379));
31 | tabEnv.createTemporarySystemFunction("udaf_median", classOf[Median]);
32 |
33 | }
34 |
35 | }
36 |
--------------------------------------------------------------------------------
/src/main/scala/com/rookie/submit/udf/SplitFunction.scala:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.udf
2 |
3 | import org.apache.flink.table.annotation.DataTypeHint
4 | import org.apache.flink.table.annotation.FunctionHint
5 | import org.apache.flink.table.functions.TableFunction
6 | import org.apache.flink.types.Row
7 |
8 | @FunctionHint(output = new DataTypeHint("ROW"))
9 | class SplitFunction extends TableFunction[Row] {
10 |
11 | def eval(str: String): Unit = {
12 | // use collect(...) to emit a row
13 | str.split(" ").foreach(s => collect(Row.of(s, Int.box(s.length))))
14 | }
15 | }
--------------------------------------------------------------------------------
/src/main/scala/com/rookie/submit/util/CatalogUtil.scala:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.util
2 |
3 | import com.rookie.submit.common.Constant
4 | import org.apache.flink.table.api.bridge.scala.StreamTableEnvironment
5 | import org.apache.flink.table.catalog.hive.HiveCatalog
6 |
7 | /**
8 | * hive catalog util
9 | */
10 | object CatalogUtil {
11 |
12 | def initCatalog(tableEnv: StreamTableEnvironment): Unit = {
13 | // Create a HiveCatalog // Create a HiveCatalog
14 | val catalog = new HiveCatalog(Constant.HIVE_CATALOG_NAME, Constant.HIVE_DEFAULT_DATABASE,
15 | Constant.DEFAULT_CONFIG_FILE, Constant.HIVE_VERSION)
16 | // Register the catalog
17 | tableEnv.registerCatalog(Constant.HIVE_CATALOG_NAME, catalog)
18 | }
19 |
20 | }
21 |
--------------------------------------------------------------------------------
/src/main/scala/com/rookie/submit/util/PropertiesUtil.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.util;
2 |
3 | import java.io.File;
4 | import java.io.FileInputStream;
5 | import java.io.IOException;
6 | import java.util.Properties;
7 |
8 | public class PropertiesUtil {
9 |
10 | public static Properties loadProp(String path, boolean isDefaultProp) throws IOException {
11 | Properties prop = new Properties();
12 |
13 | boolean propExists = new File(path).exists();
14 | // check file exists
15 | if (isDefaultProp && !propExists) {
16 | // is default properties and file not exists, exit -1
17 | System.out.println("properties not exists : " + path);
18 | System.exit(-1);
19 | } else if (!propExists) {
20 | // job properties and file not exists, ignore it, and return empty prop
21 | System.out.println("input properties not exists, ignore it");
22 | return prop;
23 | }
24 | // file exists load it
25 | prop.load(new FileInputStream(path));
26 |
27 | // return prop
28 | return prop;
29 | }
30 | }
31 |
--------------------------------------------------------------------------------
/src/main/scala/com/rookie/submit/util/SqlFileUtil.java:
--------------------------------------------------------------------------------
1 | package com.rookie.submit.util;
2 |
3 | import org.apache.commons.lang3.StringUtils;
4 | import org.slf4j.Logger;
5 | import org.slf4j.LoggerFactory;
6 |
7 | import java.io.BufferedReader;
8 | import java.io.File;
9 | import java.io.FileReader;
10 | import java.io.IOException;
11 | import java.util.ArrayList;
12 | import java.util.List;
13 |
14 | import static com.rookie.submit.common.Constant.DEFAULT_CONFIG_FILE;
15 |
16 | public class SqlFileUtil {
17 | private final static Logger LOG = LoggerFactory.getLogger(SqlFileUtil.class);
18 |
19 |
20 | public static List readFile(String fileName) throws IOException {
21 | // list store return sql
22 | List sqlList = new ArrayList<>();
23 | // if file not exists, find in classpath
24 | File file = new File(fileName);
25 | if(!file.exists()){
26 | // find sql file in classpath
27 | LOG.info(fileName + " not exists, find in class path");
28 | fileName = SqlFileUtil.class.getClassLoader().getResource("").getPath() + fileName;
29 | }
30 | file = new File(fileName);
31 | // check file exists
32 | if (!file.exists()) {
33 | System.out.println("File not found: " + fileName);
34 | System.exit(-1);
35 | }
36 | // read file
37 | BufferedReader br = new BufferedReader(new FileReader(file));
38 | String line;
39 | StringBuilder sqlBuffer = new StringBuilder();
40 | while ((line = br.readLine()) != null) {
41 | // ignore empty line and comment line
42 | if (StringUtils.isEmpty(line) || line.trim().startsWith("--")) {
43 | continue;
44 | }
45 | // remove comment
46 | if (line.contains("--")) {
47 | line = line.substring(0, line.indexOf("--"));
48 | }
49 | // add current line to sqlBuffer
50 | sqlBuffer.append(line);
51 | sqlBuffer.append("\n");
52 | // check sql end
53 | if (line.endsWith(";")) {
54 | // add sql to sqlList
55 | String tmpSql = sqlBuffer.toString();
56 | // remove last ";"
57 | tmpSql = tmpSql.substring(0, tmpSql.lastIndexOf(";"));
58 | sqlList.add(tmpSql);
59 | // remove StringBuilder
60 | sqlBuffer.delete(0, sqlBuffer.length());
61 | }
62 | }
63 | // if last sql sentence not end with ";"
64 | if (sqlBuffer.length() != 0) {
65 | sqlList.add(sqlBuffer.toString());
66 | }
67 | return sqlList;
68 |
69 |
70 | }
71 |
72 | }
73 |
--------------------------------------------------------------------------------
/src/test/scala/com/rookie/submit/FlinkTestDemo.scala:
--------------------------------------------------------------------------------
1 | package com.rookie.submit
2 |
3 | object FlinkTestDemo {
4 |
5 | def test() = {
6 | println("xx")
7 | println("xx")
8 | }
9 |
10 | }
11 |
--------------------------------------------------------------------------------
/src/test/scala/com/rookie/submit/udf/JoinHbaseNonRowkeyNoCacheTest.java:
--------------------------------------------------------------------------------
1 | package scala.com.rookie.submit.udf;
2 |
3 | import com.rookie.submit.udf.JoinHbaseNonRowkeyNoCache;
4 | import org.slf4j.Logger;
5 | import org.slf4j.LoggerFactory;
6 |
7 | public class JoinHbaseNonRowkeyNoCacheTest {
8 |
9 | public static final Logger LOG = LoggerFactory.getLogger("JoinHbaseNonRowkey1Test");
10 |
11 | // @Test
12 | public void testjoin1() {
13 |
14 | // new object
15 | JoinHbaseNonRowkeyNoCache joinHbase = new JoinHbaseNonRowkeyNoCache("cf", "c1,c2,c3,c4,c5,c6,c7,c8,c9,c10");
16 |
17 | // init join Hbase
18 | joinHbase.open(null);
19 |
20 | // query hbase
21 | joinHbase.eval("002");
22 |
23 | }
24 | }
25 |
--------------------------------------------------------------------------------