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 .PluginLoadMode ;
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
@@ -109,9 +109,9 @@ public static void main(String[] args) throws Exception {
109109 String name = options .getName ();
110110 String addJarListStr = options .getAddjar ();
111111 String localSqlPluginPath = options .getLocalSqlPluginPath ();
112- String remoteSqlPluginPath = options .getRemoteSqlPluginPath ();
113112 String deployMode = options .getMode ();
114113 String confProp = options .getConfProp ();
114+
115115 sql = URLDecoder .decode (sql , Charsets .UTF_8 .name ());
116116 SqlParser .setLocalSqlPluginRoot (localSqlPluginPath );
117117
@@ -141,7 +141,7 @@ public static void main(String[] args) throws Exception {
141141 //register udf
142142 registerUDF (sqlTree , jarURList , tableEnv );
143143 //register table schema
144- registerTable (sqlTree , env , tableEnv , localSqlPluginPath , remoteSqlPluginPath , sideTableMap , registerTableCache );
144+ registerTable (sqlTree , env , tableEnv , options , sideTableMap , registerTableCache );
145145
146146 sqlTranslation (options ,tableEnv ,sqlTree ,sideTableMap ,registerTableCache );
147147
@@ -230,8 +230,7 @@ private static void registerUDF(SqlTree sqlTree, List<URL> jarURList, StreamTabl
230230 }
231231
232232
233- private static void registerTable (SqlTree sqlTree , StreamExecutionEnvironment env , StreamTableEnvironment tableEnv ,
234- String localSqlPluginPath , String remoteSqlPluginPath ,
233+ private static void registerTable (SqlTree sqlTree , StreamExecutionEnvironment env , StreamTableEnvironment tableEnv , Options options ,
235234 Map <String , SideTableInfo > sideTableMap , Map <String , Table > registerTableCache ) throws Exception {
236235 Set <URL > classPathSet = Sets .newHashSet ();
237236 WaterMarkerAssigner waterMarkerAssigner = new WaterMarkerAssigner ();
@@ -240,7 +239,7 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en
240239 if (tableInfo instanceof SourceTableInfo ) {
241240
242241 SourceTableInfo sourceTableInfo = (SourceTableInfo ) tableInfo ;
243- Table table = StreamSourceFactory .getStreamSource (sourceTableInfo , env , tableEnv , localSqlPluginPath );
242+ Table table = StreamSourceFactory .getStreamSource (sourceTableInfo , env , tableEnv , options . getLocalSqlPluginPath () );
244243 tableEnv .registerTable (sourceTableInfo .getAdaptName (), table );
245244 //Note --- parameter conversion function can not be used inside a function of the type of polymerization
246245 //Create table in which the function is arranged only need adaptation sql
@@ -267,18 +266,18 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en
267266 LOG .info ("registe table {} success." , tableInfo .getName ());
268267 }
269268 registerTableCache .put (tableInfo .getName (), regTable );
270- classPathSet .add (PluginUtil . getRemoteJarFilePath (tableInfo .getType (), SourceTableInfo .SOURCE_SUFFIX , remoteSqlPluginPath , localSqlPluginPath ));
269+ classPathSet .add (buildSourceAndSinkPathByLoadMode (tableInfo .getType (), SourceTableInfo .SOURCE_SUFFIX , options ));
271270 } else if (tableInfo instanceof TargetTableInfo ) {
272271
273- TableSink tableSink = StreamSinkFactory .getTableSink ((TargetTableInfo ) tableInfo , localSqlPluginPath );
272+ TableSink tableSink = StreamSinkFactory .getTableSink ((TargetTableInfo ) tableInfo , options . getLocalSqlPluginPath () );
274273 TypeInformation [] flinkTypes = FlinkUtil .transformTypes (tableInfo .getFieldClasses ());
275274 tableEnv .registerTableSink (tableInfo .getName (), tableInfo .getFields (), flinkTypes , tableSink );
276- classPathSet .add ( PluginUtil . getRemoteJarFilePath (tableInfo .getType (), TargetTableInfo .TARGET_SUFFIX , remoteSqlPluginPath , localSqlPluginPath ));
275+ classPathSet .add (buildSourceAndSinkPathByLoadMode (tableInfo .getType (), TargetTableInfo .TARGET_SUFFIX , options ));
277276 } else if (tableInfo instanceof SideTableInfo ){
278277
279278 String sideOperator = ECacheType .ALL .name ().equals (((SideTableInfo ) tableInfo ).getCacheType ()) ? "all" : "async" ;
280279 sideTableMap .put (tableInfo .getName (), (SideTableInfo ) tableInfo );
281- classPathSet .add (PluginUtil . getRemoteSideJarFilePath (tableInfo .getType (), sideOperator , SideTableInfo .TARGET_SUFFIX , remoteSqlPluginPath , localSqlPluginPath ));
280+ classPathSet .add (buildSidePathByLoadMode (tableInfo .getType (), sideOperator , SideTableInfo .TARGET_SUFFIX , options ));
282281 }else {
283282 throw new RuntimeException ("not support table type:" + tableInfo .getType ());
284283 }
@@ -294,6 +293,22 @@ private static void registerTable(SqlTree sqlTree, StreamExecutionEnvironment en
294293 }
295294 }
296295
296+ private static URL buildSourceAndSinkPathByLoadMode (String type , String suffix , Options options ) throws Exception {
297+ String pluginLoadMode = options .getPluginLoadMode ();
298+ if (StringUtils .equalsIgnoreCase (pluginLoadMode , PluginLoadMode .classpath .name ())) {
299+ return PluginUtil .getRemoteJarFilePath (type , suffix , options .getRemoteSqlPluginPath (), options .getLocalSqlPluginPath ());
300+ }
301+ return PluginUtil .getLocalJarFilePath (type , suffix , options .getLocalSqlPluginPath ());
302+ }
303+
304+ private static URL buildSidePathByLoadMode (String type , String operator , String suffix , Options options ) throws Exception {
305+ String pluginLoadMode = options .getPluginLoadMode ();
306+ if (StringUtils .equalsIgnoreCase (pluginLoadMode , PluginLoadMode .classpath .name ())) {
307+ return PluginUtil .getRemoteSideJarFilePath (type , operator , suffix , options .getRemoteSqlPluginPath (), options .getLocalSqlPluginPath ());
308+ }
309+ return PluginUtil .getLocalSideJarFilePath (type , operator , suffix , options .getLocalSqlPluginPath ());
310+ }
311+
297312 private static StreamExecutionEnvironment getStreamExeEnv (Properties confProperties , String deployMode ) throws Exception {
298313 confProperties = PropertiesUtils .propertiesTrim (confProperties );
299314
0 commit comments