19
19
20
20
package com .dtstack .flink .sql ;
21
21
22
- import com .dtstack .flink .sql .config .CalciteConfig ;
23
22
import com .dtstack .flink .sql .classloader .ClassLoaderManager ;
24
23
import com .dtstack .flink .sql .enums .ClusterMode ;
25
24
import com .dtstack .flink .sql .enums .ECacheType ;
29
28
import com .dtstack .flink .sql .option .OptionParser ;
30
29
import com .dtstack .flink .sql .parser .CreateFuncParser ;
31
30
import com .dtstack .flink .sql .parser .CreateTmpTableParser ;
31
+ import com .dtstack .flink .sql .parser .FlinkPlanner ;
32
32
import com .dtstack .flink .sql .parser .InsertSqlParser ;
33
33
import com .dtstack .flink .sql .parser .SqlParser ;
34
34
import com .dtstack .flink .sql .parser .SqlTree ;
60
60
import org .apache .flink .table .api .Table ;
61
61
import org .apache .flink .table .api .TableEnvironment ;
62
62
import org .apache .flink .table .api .java .StreamTableEnvironment ;
63
+ import org .apache .flink .table .calcite .FlinkPlannerImpl ;
63
64
import org .apache .flink .table .sinks .TableSink ;
64
65
import org .apache .flink .types .Row ;
65
66
import org .slf4j .Logger ;
72
73
import java .net .URLDecoder ;
73
74
import java .util .List ;
74
75
import java .util .Map ;
75
- import java .util .Optional ;
76
76
import java .util .Properties ;
77
77
import java .util .Set ;
78
78
@@ -120,6 +120,8 @@ public static void main(String[] args) throws Exception {
120
120
StreamExecutionEnvironment env = getStreamExeEnv (confProperties , deployMode );
121
121
StreamTableEnvironment tableEnv = StreamTableEnvironment .getTableEnvironment (env );
122
122
StreamQueryConfig streamQueryConfig = StreamEnvConfigManager .getStreamQueryConfig (tableEnv , confProperties );
123
+ // init global flinkPlanner
124
+ FlinkPlanner .createFlinkPlanner (tableEnv .getFrameworkConfig (), tableEnv .getPlanner (), tableEnv .getTypeFactory ());
123
125
124
126
List <URL > jarURList = Lists .newArrayList ();
125
127
SqlTree sqlTree = SqlParser .parseSql (sql );
@@ -149,7 +151,13 @@ public static void main(String[] args) throws Exception {
149
151
env .execute (name );
150
152
}
151
153
152
- private static void sqlTranslation (String localSqlPluginPath , StreamTableEnvironment tableEnv ,SqlTree sqlTree ,Map <String , SideTableInfo > sideTableMap ,Map <String , Table > registerTableCache , StreamQueryConfig queryConfig ) throws Exception {
154
+ private static void sqlTranslation (String localSqlPluginPath ,
155
+ StreamTableEnvironment tableEnv ,
156
+ SqlTree sqlTree ,
157
+ Map <String , SideTableInfo > sideTableMap ,
158
+ Map <String , Table > registerTableCache ,
159
+ StreamQueryConfig queryConfig ) throws Exception {
160
+
153
161
SideSqlExec sideSqlExec = new SideSqlExec ();
154
162
sideSqlExec .setLocalSqlPluginPath (localSqlPluginPath );
155
163
for (CreateTmpTableParser .SqlParserResult result : sqlTree .getTmpSqlList ()) {
@@ -165,8 +173,9 @@ private static void sqlTranslation(String localSqlPluginPath, StreamTableEnviron
165
173
if (sqlTree .getTmpTableMap ().containsKey (tableName )) {
166
174
CreateTmpTableParser .SqlParserResult tmp = sqlTree .getTmpTableMap ().get (tableName );
167
175
String realSql = DtStringUtil .replaceIgnoreQuota (result .getExecSql (), "`" , "" );
176
+ FlinkPlannerImpl flinkPlanner = FlinkPlanner .getFlinkPlanner ();
168
177
169
- SqlNode sqlNode = org . apache . calcite . sql . parser . SqlParser . create (realSql , CalciteConfig . MYSQL_LEX_CONFIG ). parseStmt ( );
178
+ SqlNode sqlNode = flinkPlanner . parse (realSql );
170
179
String tmpSql = ((SqlInsert ) sqlNode ).getSource ().toString ();
171
180
tmp .setExecSql (tmpSql );
172
181
sideSqlExec .exec (tmp .getExecSql (), sideTableMap , tableEnv , registerTableCache , queryConfig , tmp );
0 commit comments