|
16 | 16 | * limitations under the License. |
17 | 17 | */ |
18 | 18 |
|
19 | | - |
| 19 | + |
20 | 20 |
|
21 | 21 | package com.dtstack.flink.sql.util; |
22 | 22 |
|
|
33 | 33 | import org.apache.flink.table.api.java.StreamTableEnvironment; |
34 | 34 | import org.apache.flink.table.functions.ScalarFunction; |
35 | 35 | import org.apache.flink.table.functions.TableFunction; |
| 36 | +import org.apache.flink.table.functions.UserDefinedFunction; |
| 37 | +import org.apache.flink.table.functions.AggregateFunction; |
| 38 | + |
36 | 39 | import org.slf4j.Logger; |
37 | 40 | import org.slf4j.LoggerFactory; |
38 | 41 |
|
@@ -192,13 +195,25 @@ public static void registerScalaUDF(String classPath, String funcName, TableEnvi |
192 | 195 | public static void registerTableUDF(String classPath, String funcName, TableEnvironment tableEnv, |
193 | 196 | ClassLoader classLoader){ |
194 | 197 | try { |
195 | | - TableFunction udfFunc = Class.forName(classPath, false, classLoader) |
196 | | - .asSubclass(TableFunction.class).newInstance(); |
| 198 | + UserDefinedFunction udfFunc = Class.forName(classPath,false, classLoader) |
| 199 | + .asSubclass(UserDefinedFunction.class).newInstance(); |
197 | 200 |
|
198 | 201 | if(tableEnv instanceof StreamTableEnvironment){ |
199 | | - ((StreamTableEnvironment)tableEnv).registerFunction(funcName, udfFunc); |
| 202 | + if (udfFunc instanceof AggregateFunction){ |
| 203 | + ((StreamTableEnvironment) tableEnv).registerFunction(funcName, (AggregateFunction)udfFunc); |
| 204 | + }else if (udfFunc instanceof TableFunction) { |
| 205 | + ((StreamTableEnvironment) tableEnv).registerFunction(funcName, (TableFunction)udfFunc); |
| 206 | + }else{ |
| 207 | + throw new RuntimeException("no support UserDefinedFunction class for " + udfFunc.getClass().getName()); |
| 208 | + } |
200 | 209 | }else if(tableEnv instanceof BatchTableEnvironment){ |
201 | | - ((BatchTableEnvironment)tableEnv).registerFunction(funcName, udfFunc); |
| 210 | + if (udfFunc instanceof AggregateFunction){ |
| 211 | + ((BatchTableEnvironment) tableEnv).registerFunction(funcName, (AggregateFunction)udfFunc); |
| 212 | + }else if (udfFunc instanceof TableFunction) { |
| 213 | + ((BatchTableEnvironment) tableEnv).registerFunction(funcName, (TableFunction)udfFunc); |
| 214 | + }else{ |
| 215 | + throw new RuntimeException("no support UserDefinedFunction class for " + udfFunc.getClass().getName()); |
| 216 | + } |
202 | 217 | }else{ |
203 | 218 | throw new RuntimeException("no support tableEnvironment class for " + tableEnv.getClass().getName()); |
204 | 219 | } |
|
0 commit comments