2222
2323import com .dtstack .flink .sql .classloader .DtClassLoader ;
2424import com .dtstack .flink .sql .enums .ECacheType ;
25- import com .dtstack .flink .sql .parser .*;
25+ import com .dtstack .flink .sql .exec .FlinkSQLExec ;
26+ import com .dtstack .flink .sql .parser .CreateFuncParser ;
27+ import com .dtstack .flink .sql .parser .CreateTmpTableParser ;
28+ import com .dtstack .flink .sql .parser .InsertSqlParser ;
29+ import com .dtstack .flink .sql .parser .SqlParser ;
30+ import com .dtstack .flink .sql .parser .SqlTree ;
2631import com .dtstack .flink .sql .side .SideSqlExec ;
2732import com .dtstack .flink .sql .side .SideTableInfo ;
2833import com .dtstack .flink .sql .table .SourceTableInfo ;
3540import com .dtstack .flink .sql .util .FlinkUtil ;
3641import com .dtstack .flink .sql .util .PluginUtil ;
3742import org .apache .calcite .config .Lex ;
38- import org .apache .calcite .sql .SqlIdentifier ;
3943import org .apache .calcite .sql .SqlInsert ;
4044import org .apache .calcite .sql .SqlNode ;
4145import org .apache .commons .cli .CommandLine ;
6165import org .apache .flink .streaming .api .environment .StreamContextEnvironment ;
6266import org .apache .flink .streaming .api .environment .StreamExecutionEnvironment ;
6367import org .apache .flink .table .api .Table ;
64- import org .apache .flink .table .api .TableEnvironment ;
65- import org .apache .flink .table .api .TableException ;
6668import org .apache .flink .table .api .java .StreamTableEnvironment ;
67- import org .apache .flink .table .calcite .FlinkPlannerImpl ;
68- import org .apache .flink .table .plan .logical .LogicalRelNode ;
69- import org .apache .flink .table .plan .schema .TableSinkTable ;
7069import org .apache .flink .table .sinks .TableSink ;
7170import org .apache .flink .types .Row ;
7271import org .slf4j .Logger ;
@@ -210,7 +209,7 @@ public static void main(String[] args) throws Exception {
210209 //sql-dimensional table contains the dimension table of execution
211210 sideSqlExec .exec (result .getExecSql (), sideTableMap , tableEnv , registerTableCache );
212211 }else {
213- sqlUpdate (tableEnv , result .getExecSql ());
212+ FlinkSQLExec . sqlUpdate (tableEnv , result .getExecSql ());
214213 if (LOG .isInfoEnabled ()){
215214 LOG .info ("exec sql: " + result .getExecSql ());
216215 }
@@ -228,36 +227,6 @@ public static void main(String[] args) throws Exception {
228227 env .execute (name );
229228 }
230229
231- public static void sqlUpdate (StreamTableEnvironment tableEnv , String stmt ) {
232-
233- FlinkPlannerImpl planner = new FlinkPlannerImpl (tableEnv .getFrameworkConfig (), tableEnv .getPlanner (), tableEnv .getTypeFactory ());
234- SqlNode insert = planner .parse (stmt );
235-
236- if (!(insert instanceof SqlInsert )) {
237- throw new TableException (
238- "Unsupported SQL query! sqlUpdate() only accepts SQL statements of type INSERT." );
239- }
240- SqlNode query = ((SqlInsert ) insert ).getSource ();
241-
242- SqlNode validatedQuery = planner .validate (query );
243-
244- Table queryResult = new Table (tableEnv , new LogicalRelNode (planner .rel (validatedQuery ).rel ));
245- String targetTableName = ((SqlIdentifier ) ((SqlInsert ) insert ).getTargetTable ()).names .get (0 );
246-
247- try {
248- Method method = TableEnvironment .class .getDeclaredMethod ("getTable" , String .class );
249- method .setAccessible (true );
250-
251- TableSinkTable targetTable = (TableSinkTable ) method .invoke (tableEnv , targetTableName );
252- String [] fieldNames = targetTable .tableSink ().getFieldNames ();
253- Table newTable = queryResult .select (String .join ("," , fieldNames ));
254- // insert query result into sink table
255- tableEnv .insertInto (newTable , targetTableName , tableEnv .queryConfig ());
256- } catch (Exception e ) {
257- e .printStackTrace ();
258- }
259- }
260-
261230 /**
262231 * This part is just to add classpath for the jar when reading remote execution, and will not submit jar from a local
263232 * @param env
0 commit comments