2020
2121package com .dtstack .flink .sql ;
2222
23- import com .dtstack .flink .sql .classloader .DtClassLoader ;
23+ import com .dtstack .flink .sql .config .CalciteConfig ;
24+ import com .dtstack .flink .sql .classloader .ClassLoaderManager ;
2425import com .dtstack .flink .sql .constrant .ConfigConstrant ;
2526import com .dtstack .flink .sql .enums .ClusterMode ;
2627import com .dtstack .flink .sql .enums .ECacheType ;
7778import java .net .URL ;
7879import java .net .URLClassLoader ;
7980import java .net .URLDecoder ;
80- import java .util .ArrayList ;
81- import java .util .Arrays ;
8281import java .util .List ;
8382import java .util .Map ;
8483import java .util .Properties ;
@@ -101,10 +100,6 @@ public class Main {
101100
102101 private static final Logger LOG = LoggerFactory .getLogger (Main .class );
103102
104- private static Config config = org .apache .calcite .sql .parser .SqlParser
105- .configBuilder ()
106- .setLex (Lex .MYSQL )
107- .build ();
108103
109104 public static void main (String [] args ) throws Exception {
110105
@@ -126,10 +121,6 @@ public static void main(String[] args) throws Exception {
126121 addJarFileList = objMapper .readValue (addJarListStr , List .class );
127122 }
128123
129- ClassLoader threadClassLoader = Thread .currentThread ().getContextClassLoader ();
130- DtClassLoader parentClassloader = new DtClassLoader (new URL []{}, threadClassLoader );
131- Thread .currentThread ().setContextClassLoader (parentClassloader );
132-
133124 confProp = URLDecoder .decode (confProp , Charsets .UTF_8 .toString ());
134125 Properties confProperties = PluginUtil .jsonStrToObject (confProp , Properties .class );
135126 StreamExecutionEnvironment env = getStreamExeEnv (confProperties , deployMode );
@@ -148,16 +139,14 @@ public static void main(String[] args) throws Exception {
148139 Map <String , Table > registerTableCache = Maps .newHashMap ();
149140
150141 //register udf
151- registerUDF (sqlTree , jarURList , parentClassloader , tableEnv );
142+ registerUDF (sqlTree , jarURList , tableEnv );
152143 //register table schema
153144 registerTable (sqlTree , env , tableEnv , localSqlPluginPath , remoteSqlPluginPath , sideTableMap , registerTableCache );
154145
155146 sqlTranslation (options ,tableEnv ,sqlTree ,sideTableMap ,registerTableCache );
156147
157148 if (env instanceof MyLocalStreamEnvironment ) {
158- List <URL > urlList = new ArrayList <>();
159- urlList .addAll (Arrays .asList (parentClassloader .getURLs ()));
160- ((MyLocalStreamEnvironment ) env ).setClasspaths (urlList );
149+ ((MyLocalStreamEnvironment ) env ).setClasspaths (ClassLoaderManager .getClassPath ());
161150 }
162151
163152 env .execute (name );
@@ -180,7 +169,7 @@ private static void sqlTranslation(Options options,StreamTableEnvironment tableE
180169 CreateTmpTableParser .SqlParserResult tmp = sqlTree .getTmpTableMap ().get (tableName );
181170 String realSql = DtStringUtil .replaceIgnoreQuota (result .getExecSql (), "`" , "" );
182171
183- SqlNode sqlNode = org .apache .calcite .sql .parser .SqlParser .create (realSql ,config ).parseStmt ();
172+ SqlNode sqlNode = org .apache .calcite .sql .parser .SqlParser .create (realSql , CalciteConfig . MYSQL_LEX_CONFIG ).parseStmt ();
184173 String tmpSql = ((SqlInsert ) sqlNode ).getSource ().toString ();
185174 tmp .setExecSql (tmpSql );
186175 sideSqlExec .registerTmpTable (tmp , sideTableMap , tableEnv , registerTableCache );
@@ -224,19 +213,19 @@ private static void addEnvClassPath(StreamExecutionEnvironment env, Set<URL> cla
224213 }
225214 }
226215
227- private static void registerUDF (SqlTree sqlTree , List <URL > jarURList , URLClassLoader parentClassloader ,
228- StreamTableEnvironment tableEnv )
229- throws ClassNotFoundException , NoSuchMethodException , IllegalAccessException , InvocationTargetException {
230- List <CreateFuncParser .SqlParserResult > funcList = sqlTree .getFunctionList ();
231- if (funcList .isEmpty ()) {
232- return ;
233- }
234- //load jar
235- URLClassLoader classLoader = FlinkUtil .loadExtraJar (jarURList , parentClassloader );
216+ private static void registerUDF (SqlTree sqlTree , List <URL > jarURList , StreamTableEnvironment tableEnv )
217+ throws NoSuchMethodException , IllegalAccessException , InvocationTargetException {
236218 //register urf
219+ // udf和tableEnv须由同一个类加载器加载
220+ ClassLoader levelClassLoader = tableEnv .getClass ().getClassLoader ();
221+ URLClassLoader classLoader = null ;
222+ List <CreateFuncParser .SqlParserResult > funcList = sqlTree .getFunctionList ();
237223 for (CreateFuncParser .SqlParserResult funcInfo : funcList ) {
238- FlinkUtil .registerUDF (funcInfo .getType (), funcInfo .getClassName (), funcInfo .getName (),
239- tableEnv , classLoader );
224+ //classloader
225+ if (classLoader == null ) {
226+ classLoader = FlinkUtil .loadExtraJar (jarURList , (URLClassLoader )levelClassLoader );
227+ }
228+ FlinkUtil .registerUDF (funcInfo .getType (), funcInfo .getClassName (), funcInfo .getName (), tableEnv , classLoader );
240229 }
241230 }
242231
0 commit comments