2525import com .dtstack .flink .sql .constrant .ConfigConstrant ;
2626import com .dtstack .flink .sql .enums .ClusterMode ;
2727import com .dtstack .flink .sql .enums .ECacheType ;
28+ import com .dtstack .flink .sql .enums .EPluginLoadMode ;
2829import com .dtstack .flink .sql .environment .MyLocalStreamEnvironment ;
2930import com .dtstack .flink .sql .exec .FlinkSQLExec ;
3031import com .dtstack .flink .sql .option .OptionParser ;
4546import com .dtstack .flink .sql .watermarker .WaterMarkerAssigner ;
4647import com .dtstack .flink .sql .util .FlinkUtil ;
4748import com .dtstack .flink .sql .util .PluginUtil ;
48- import org .apache .calcite .config .Lex ;
4949import org .apache .calcite .sql .SqlInsert ;
5050import org .apache .calcite .sql .SqlNode ;
5151import org .apache .commons .io .Charsets ;
52+ import org .apache .commons .lang3 .StringUtils ;
5253import org .apache .flink .api .common .ExecutionConfig ;
5354import org .apache .flink .api .common .restartstrategy .RestartStrategies ;
5455import org .apache .flink .api .common .time .Time ;
8485import java .util .Set ;
8586import java .util .concurrent .TimeUnit ;
8687import com .dtstack .flink .sql .option .Options ;
87- import org .apache .calcite .sql .parser .SqlParser .Config ;
8888
8989/**
9090 * Date: 2018/6/26
@@ -110,8 +110,10 @@ public static void main(String[] args) throws Exception {
110110 String addJarListStr = options .getAddjar ();
111111 String localSqlPluginPath = options .getLocalSqlPluginPath ();
112112 String remoteSqlPluginPath = options .getRemoteSqlPluginPath ();
113+ String pluginLoadMode = options .getPluginLoadMode ();
113114 String deployMode = options .getMode ();
114115 String confProp = options .getConfProp ();
116+
115117 sql = URLDecoder .decode (sql , Charsets .UTF_8 .name ());
116118 SqlParser .setLocalSqlPluginRoot (localSqlPluginPath );
117119
@@ -141,9 +143,9 @@ public static void main(String[] args) throws Exception {
141143 //register udf
142144 registerUDF (sqlTree , jarURList , tableEnv );
143145 //register table schema
144- registerTable (sqlTree , env , tableEnv , localSqlPluginPath , remoteSqlPluginPath , sideTableMap , registerTableCache );
146+ registerTable (sqlTree , env , tableEnv , localSqlPluginPath , remoteSqlPluginPath , pluginLoadMode , sideTableMap , registerTableCache );
145147
146- sqlTranslation (options , tableEnv ,sqlTree ,sideTableMap ,registerTableCache );
148+ sqlTranslation (localSqlPluginPath , tableEnv ,sqlTree ,sideTableMap ,registerTableCache );
147149
148150 if (env instanceof MyLocalStreamEnvironment ) {
149151 ((MyLocalStreamEnvironment ) env ).setClasspaths (ClassLoaderManager .getClassPath ());
@@ -152,9 +154,9 @@ public static void main(String[] args) throws Exception {
152154 env .execute (name );
153155 }
154156
155- private static void sqlTranslation (Options options , StreamTableEnvironment tableEnv ,SqlTree sqlTree ,Map <String , SideTableInfo > sideTableMap ,Map <String , Table > registerTableCache ) throws Exception {
157+ private static void sqlTranslation (String localSqlPluginPath , StreamTableEnvironment tableEnv ,SqlTree sqlTree ,Map <String , SideTableInfo > sideTableMap ,Map <String , Table > registerTableCache ) throws Exception {
156158 SideSqlExec sideSqlExec = new SideSqlExec ();
157- sideSqlExec .setLocalSqlPluginPath (options . getLocalSqlPluginPath () );
159+ sideSqlExec .setLocalSqlPluginPath (localSqlPluginPath );
158160 for (CreateTmpTableParser .SqlParserResult result : sqlTree .getTmpSqlList ()) {
159161 sideSqlExec .registerTmpTable (result , sideTableMap , tableEnv , registerTableCache );
160162 }
@@ -230,9 +232,8 @@ private static void registerUDF(SqlTree sqlTree, List<URL> jarURList, StreamTabl
230232 }
231233
232234
233- private static void registerTable (SqlTree sqlTree , StreamExecutionEnvironment env , StreamTableEnvironment tableEnv ,
234- String localSqlPluginPath , String remoteSqlPluginPath ,
235- Map <String , SideTableInfo > sideTableMap , Map <String , Table > registerTableCache ) throws Exception {
235+ private static void registerTable (SqlTree sqlTree , StreamExecutionEnvironment env , StreamTableEnvironment tableEnv , String localSqlPluginPath ,
236+ String remoteSqlPluginPath , String pluginLoadMode , Map <String , SideTableInfo > sideTableMap , Map <String , Table > registerTableCache ) throws Exception {
236237 Set <URL > classPathSet = Sets .newHashSet ();
237238 WaterMarkerAssigner waterMarkerAssigner = new WaterMarkerAssigner ();
238239 for (TableInfo tableInfo : sqlTree .getTableInfoMap ().values ()) {
@@ -267,18 +268,18 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en
267268 LOG .info ("registe table {} success." , tableInfo .getName ());
268269 }
269270 registerTableCache .put (tableInfo .getName (), regTable );
270- classPathSet .add (PluginUtil . getRemoteJarFilePath (tableInfo .getType (), SourceTableInfo .SOURCE_SUFFIX , remoteSqlPluginPath , localSqlPluginPath ));
271+ classPathSet .add (buildSourceAndSinkPathByLoadMode (tableInfo .getType (), SourceTableInfo .SOURCE_SUFFIX , localSqlPluginPath , remoteSqlPluginPath , pluginLoadMode ));
271272 } else if (tableInfo instanceof TargetTableInfo ) {
272273
273274 TableSink tableSink = StreamSinkFactory .getTableSink ((TargetTableInfo ) tableInfo , localSqlPluginPath );
274275 TypeInformation [] flinkTypes = FlinkUtil .transformTypes (tableInfo .getFieldClasses ());
275276 tableEnv .registerTableSink (tableInfo .getName (), tableInfo .getFields (), flinkTypes , tableSink );
276- classPathSet .add ( PluginUtil . getRemoteJarFilePath (tableInfo .getType (), TargetTableInfo .TARGET_SUFFIX , remoteSqlPluginPath , localSqlPluginPath ));
277+ classPathSet .add (buildSourceAndSinkPathByLoadMode (tableInfo .getType (), TargetTableInfo .TARGET_SUFFIX , localSqlPluginPath , remoteSqlPluginPath , pluginLoadMode ));
277278 } else if (tableInfo instanceof SideTableInfo ){
278279
279280 String sideOperator = ECacheType .ALL .name ().equals (((SideTableInfo ) tableInfo ).getCacheType ()) ? "all" : "async" ;
280281 sideTableMap .put (tableInfo .getName (), (SideTableInfo ) tableInfo );
281- classPathSet .add (PluginUtil . getRemoteSideJarFilePath (tableInfo .getType (), sideOperator , SideTableInfo .TARGET_SUFFIX , remoteSqlPluginPath , localSqlPluginPath ));
282+ classPathSet .add (buildSidePathByLoadMode (tableInfo .getType (), sideOperator , SideTableInfo .TARGET_SUFFIX , localSqlPluginPath , remoteSqlPluginPath , pluginLoadMode ));
282283 }else {
283284 throw new RuntimeException ("not support table type:" + tableInfo .getType ());
284285 }
@@ -294,6 +295,20 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en
294295 }
295296 }
296297
298+ private static URL buildSourceAndSinkPathByLoadMode (String type , String suffix , String localSqlPluginPath , String remoteSqlPluginPath , String pluginLoadMode ) throws Exception {
299+ if (StringUtils .equalsIgnoreCase (pluginLoadMode , EPluginLoadMode .CLASSPATH .name ())) {
300+ return PluginUtil .getRemoteJarFilePath (type , suffix , remoteSqlPluginPath , localSqlPluginPath );
301+ }
302+ return PluginUtil .getLocalJarFilePath (type , suffix , localSqlPluginPath );
303+ }
304+
305+ private static URL buildSidePathByLoadMode (String type , String operator , String suffix , String localSqlPluginPath , String remoteSqlPluginPath , String pluginLoadMode ) throws Exception {
306+ if (StringUtils .equalsIgnoreCase (pluginLoadMode , EPluginLoadMode .CLASSPATH .name ())) {
307+ return PluginUtil .getRemoteSideJarFilePath (type , operator , suffix , remoteSqlPluginPath , localSqlPluginPath );
308+ }
309+ return PluginUtil .getLocalSideJarFilePath (type , operator , suffix , localSqlPluginPath );
310+ }
311+
297312 private static StreamExecutionEnvironment getStreamExeEnv (Properties confProperties , String deployMode ) throws Exception {
298313 confProperties = PropertiesUtils .propertiesTrim (confProperties );
299314
0 commit comments