2020
2121package com .dtstack .flink .sql ;
2222
23- import com .dtstack .flink .sql .classloader .DtClassLoader ;
2423import com .dtstack .flink .sql .config .CalciteConfig ;
24+ import com .dtstack .flink .sql .classloader .ClassLoaderManager ;
2525import com .dtstack .flink .sql .constrant .ConfigConstrant ;
2626import com .dtstack .flink .sql .enums .ClusterMode ;
2727import com .dtstack .flink .sql .enums .ECacheType ;
7676import java .lang .reflect .InvocationTargetException ;
7777import java .lang .reflect .Method ;
7878import java .net .URL ;
79- import java .net .URLClassLoader ;
8079import java .net .URLDecoder ;
81- import java .util .ArrayList ;
82- import java .util .Arrays ;
8380import java .util .List ;
8481import java .util .Map ;
8582import java .util .Properties ;
@@ -123,10 +120,6 @@ public static void main(String[] args) throws Exception {
123120 addJarFileList = objMapper .readValue (addJarListStr , List .class );
124121 }
125122
126- ClassLoader threadClassLoader = Thread .currentThread ().getContextClassLoader ();
127- DtClassLoader parentClassloader = new DtClassLoader (new URL []{}, threadClassLoader );
128- Thread .currentThread ().setContextClassLoader (parentClassloader );
129-
130123 confProp = URLDecoder .decode (confProp , Charsets .UTF_8 .toString ());
131124 Properties confProperties = PluginUtil .jsonStrToObject (confProp , Properties .class );
132125 StreamExecutionEnvironment env = getStreamExeEnv (confProperties , deployMode );
@@ -145,16 +138,14 @@ public static void main(String[] args) throws Exception {
145138 Map <String , Table > registerTableCache = Maps .newHashMap ();
146139
147140 //register udf
148- registerUDF (sqlTree , jarURList , parentClassloader , tableEnv );
141+ registerUDF (sqlTree , jarURList , tableEnv );
149142 //register table schema
150143 registerTable (sqlTree , env , tableEnv , localSqlPluginPath , remoteSqlPluginPath , sideTableMap , registerTableCache );
151144
152145 sqlTranslation (options ,tableEnv ,sqlTree ,sideTableMap ,registerTableCache );
153146
154147 if (env instanceof MyLocalStreamEnvironment ) {
155- List <URL > urlList = new ArrayList <>();
156- urlList .addAll (Arrays .asList (parentClassloader .getURLs ()));
157- ((MyLocalStreamEnvironment ) env ).setClasspaths (urlList );
148+ ((MyLocalStreamEnvironment ) env ).setClasspaths (ClassLoaderManager .getClassPath ());
158149 }
159150
160151 env .execute (name );
@@ -221,19 +212,12 @@ private static void addEnvClassPath(StreamExecutionEnvironment env, Set<URL> cla
221212 }
222213 }
223214
224- private static void registerUDF (SqlTree sqlTree , List <URL > jarURList , URLClassLoader parentClassloader ,
225- StreamTableEnvironment tableEnv )
215+ private static void registerUDF (SqlTree sqlTree , List <URL > jarURList , StreamTableEnvironment tableEnv )
226216 throws ClassNotFoundException , NoSuchMethodException , IllegalAccessException , InvocationTargetException {
227- List <CreateFuncParser .SqlParserResult > funcList = sqlTree .getFunctionList ();
228- if (funcList .isEmpty ()) {
229- return ;
230- }
231- //load jar
232- URLClassLoader classLoader = FlinkUtil .loadExtraJar (jarURList , parentClassloader );
233217 //register urf
218+ List <CreateFuncParser .SqlParserResult > funcList = sqlTree .getFunctionList ();
234219 for (CreateFuncParser .SqlParserResult funcInfo : funcList ) {
235- FlinkUtil .registerUDF (funcInfo .getType (), funcInfo .getClassName (), funcInfo .getName (),
236- tableEnv , classLoader );
220+ FlinkUtil .registerUDF (funcInfo .getType (), funcInfo .getClassName (), funcInfo .getName (), tableEnv , jarURList );
237221 }
238222 }
239223
0 commit comments