|
26 | 26 | import com.dtstack.flink.sql.enums.ClusterMode; |
27 | 27 | import com.dtstack.flink.sql.enums.ECacheType; |
28 | 28 | import com.dtstack.flink.sql.enums.EPluginLoadMode; |
| 29 | +//import com.dtstack.flink.sql.exec.FlinkSQLExec; |
29 | 30 | import com.dtstack.flink.sql.environment.MyLocalStreamEnvironment; |
30 | | -import com.dtstack.flink.sql.exec.FlinkSQLExec; |
31 | 31 | import com.dtstack.flink.sql.option.OptionParser; |
32 | 32 | import com.dtstack.flink.sql.parser.CreateFuncParser; |
33 | 33 | import com.dtstack.flink.sql.parser.CreateTmpTableParser; |
|
66 | 66 | import org.apache.flink.streaming.api.datastream.DataStream; |
67 | 67 | import org.apache.flink.streaming.api.environment.StreamContextEnvironment; |
68 | 68 | import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; |
69 | | -import org.apache.flink.table.api.StreamQueryConfig; |
| 69 | +import org.apache.flink.table.api.EnvironmentSettings; |
70 | 70 | import org.apache.flink.table.api.Table; |
| 71 | +import org.apache.flink.table.api.TableEnvironment; |
71 | 72 | import org.apache.flink.table.api.java.StreamTableEnvironment; |
72 | 73 | import org.apache.flink.table.sinks.TableSink; |
73 | 74 | import org.apache.flink.types.Row; |
@@ -187,7 +188,8 @@ private static void sqlTranslation(String localSqlPluginPath, StreamTableEnviron |
187 | 188 | //sql-dimensional table contains the dimension table of execution |
188 | 189 | sideSqlExec.exec(result.getExecSql(), sideTableMap, tableEnv, registerTableCache); |
189 | 190 | }else{ |
190 | | - FlinkSQLExec.sqlUpdate(tableEnv, result.getExecSql()); |
| 191 | +// FlinkSQLExec.sqlUpdate(tableEnv, result.getExecSql()); |
| 192 | + tableEnv.sqlUpdate(result.getExecSql()); |
191 | 193 | if(LOG.isInfoEnabled()){ |
192 | 194 | LOG.info("exec sql: " + result.getExecSql()); |
193 | 195 | } |
@@ -216,7 +218,7 @@ private static void addEnvClassPath(StreamExecutionEnvironment env, Set<URL> cla |
216 | 218 | } |
217 | 219 | } |
218 | 220 |
|
219 | | - private static void registerUDF(SqlTree sqlTree, List<URL> jarURList, StreamTableEnvironment tableEnv) |
| 221 | + private static void registerUDF(SqlTree sqlTree, List<URL> jarURList, TableEnvironment tableEnv) |
220 | 222 | throws NoSuchMethodException, IllegalAccessException, InvocationTargetException { |
221 | 223 | //register urf |
222 | 224 | // udf和tableEnv须由同一个类加载器加载 |
@@ -249,7 +251,7 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en |
249 | 251 | String adaptSql = sourceTableInfo.getAdaptSelectSql(); |
250 | 252 | Table adaptTable = adaptSql == null ? table : tableEnv.sqlQuery(adaptSql); |
251 | 253 |
|
252 | | - RowTypeInfo typeInfo = new RowTypeInfo(adaptTable.getSchema().getTypes(), adaptTable.getSchema().getColumnNames()); |
| 254 | + RowTypeInfo typeInfo = new RowTypeInfo(adaptTable.getSchema().getFieldTypes(), adaptTable.getSchema().getFieldNames()); |
253 | 255 | DataStream adaptStream = tableEnv.toRetractStream(adaptTable, typeInfo) |
254 | 256 | .map((Tuple2<Boolean, Row> f0) -> { return f0.f1; }) |
255 | 257 | .returns(typeInfo); |
@@ -353,12 +355,16 @@ private static StreamExecutionEnvironment getStreamExeEnv(Properties confPropert |
353 | 355 | * 获取StreamTableEnvironment并设置相关属性 |
354 | 356 | * |
355 | 357 | * @param confProperties |
356 | | - * @param env |
357 | 358 | * @return |
358 | 359 | */ |
359 | 360 | private static StreamTableEnvironment getStreamTableEnv(Properties confProperties, StreamExecutionEnvironment env) { |
| 361 | + EnvironmentSettings settings = EnvironmentSettings.newInstance() |
| 362 | + .useBlinkPlanner() |
| 363 | + .inStreamingMode() |
| 364 | + .build(); |
| 365 | + StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env, settings); |
| 366 | + |
360 | 367 | confProperties = PropertiesUtils.propertiesTrim(confProperties); |
361 | | - StreamTableEnvironment tableEnv = StreamTableEnvironment.getTableEnvironment(env); |
362 | 368 | FlinkUtil.setTableEnvTTL(confProperties, tableEnv); |
363 | 369 | return tableEnv; |
364 | 370 | } |
|
0 commit comments