forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-13225][table-planner-blink] Fix type inference for hive udf
- Loading branch information
1 parent
f695a76
commit e08117f
Showing
6 changed files
with
205 additions
and
5 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
80 changes: 80 additions & 0 deletions
80
...blink/src/main/java/org/apache/flink/table/planner/functions/utils/HiveFunctionUtils.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,80 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.table.planner.functions.utils; | ||
|
||
import org.apache.flink.table.planner.calcite.FlinkTypeFactory; | ||
import org.apache.flink.table.types.DataType; | ||
import org.apache.flink.table.types.logical.LogicalType; | ||
import org.apache.flink.table.types.utils.TypeConversions; | ||
|
||
import org.apache.calcite.rel.type.RelDataType; | ||
|
||
import java.io.Serializable; | ||
import java.lang.reflect.InvocationTargetException; | ||
import java.lang.reflect.Method; | ||
|
||
import static org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType; | ||
|
||
/** | ||
* Hack utils for hive function. | ||
*/ | ||
public class HiveFunctionUtils { | ||
|
||
public static boolean isHiveFunc(Object function) { | ||
try { | ||
getSetArgsMethod(function); | ||
return true; | ||
} catch (NoSuchMethodException e) { | ||
return false; | ||
} | ||
} | ||
|
||
private static Method getSetArgsMethod(Object function) throws NoSuchMethodException { | ||
return function.getClass().getMethod( | ||
"setArgumentTypesAndConstants", Object[].class, DataType[].class); | ||
|
||
} | ||
|
||
static Serializable invokeSetArgs( | ||
Serializable function, Object[] constantArguments, LogicalType[] argTypes) { | ||
try { | ||
// See hive HiveFunction | ||
Method method = getSetArgsMethod(function); | ||
method.invoke(function, constantArguments, TypeConversions.fromLogicalToDataType(argTypes)); | ||
return function; | ||
} catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { | ||
throw new RuntimeException(e); | ||
} | ||
} | ||
|
||
static RelDataType invokeGetResultType( | ||
Object function, Object[] constantArguments, LogicalType[] argTypes, | ||
FlinkTypeFactory typeFactory) { | ||
try { | ||
// See hive HiveFunction | ||
Method method = function.getClass() | ||
.getMethod("getHiveResultType", Object[].class, DataType[].class); | ||
DataType resultType = (DataType) method.invoke( | ||
function, constantArguments, TypeConversions.fromLogicalToDataType(argTypes)); | ||
return typeFactory.createFieldTypeFromLogicalType(fromDataTypeToLogicalType(resultType)); | ||
} catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { | ||
throw new RuntimeException(e); | ||
} | ||
} | ||
} |
85 changes: 85 additions & 0 deletions
85
...k/src/main/java/org/apache/flink/table/planner/functions/utils/HiveScalarSqlFunction.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,85 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.table.planner.functions.utils; | ||
|
||
import org.apache.flink.table.functions.ScalarFunction; | ||
import org.apache.flink.table.planner.calcite.FlinkTypeFactory; | ||
import org.apache.flink.table.types.logical.LogicalType; | ||
import org.apache.flink.util.InstantiationUtil; | ||
|
||
import org.apache.calcite.rel.type.RelDataType; | ||
import org.apache.calcite.sql.type.SqlReturnTypeInference; | ||
|
||
import java.io.IOException; | ||
import java.util.List; | ||
|
||
import scala.Some; | ||
|
||
import static org.apache.flink.table.planner.functions.utils.HiveFunctionUtils.invokeGetResultType; | ||
import static org.apache.flink.table.planner.functions.utils.HiveFunctionUtils.invokeSetArgs; | ||
import static org.apache.flink.table.runtime.types.ClassLogicalTypeConverter.getDefaultExternalClassForType; | ||
|
||
/** | ||
* Hive {@link ScalarSqlFunction}. | ||
* Override getFunction to clone function and invoke {@code HiveScalarFunction#setArgumentTypesAndConstants}. | ||
* Override SqlReturnTypeInference to invoke {@code HiveScalarFunction#getHiveResultType} instead of | ||
* {@code HiveScalarFunction#getResultType(Class[])}. | ||
* | ||
* @deprecated TODO hack code, its logical should be integrated to ScalarSqlFunction | ||
*/ | ||
@Deprecated | ||
public class HiveScalarSqlFunction extends ScalarSqlFunction { | ||
|
||
private final ScalarFunction function; | ||
|
||
public HiveScalarSqlFunction( | ||
String name, String displayName, | ||
ScalarFunction function, FlinkTypeFactory typeFactory) { | ||
super(name, displayName, function, typeFactory, new Some<>(createReturnTypeInference(function, typeFactory))); | ||
this.function = function; | ||
} | ||
|
||
@Override | ||
public ScalarFunction makeFunction(Object[] constantArguments, LogicalType[] argTypes) { | ||
ScalarFunction clone; | ||
try { | ||
clone = InstantiationUtil.clone(function); | ||
} catch (IOException | ClassNotFoundException e) { | ||
throw new RuntimeException(e); | ||
} | ||
return (ScalarFunction) invokeSetArgs(clone, constantArguments, argTypes); | ||
} | ||
|
||
private static SqlReturnTypeInference createReturnTypeInference( | ||
ScalarFunction function, FlinkTypeFactory typeFactory) { | ||
return opBinding -> { | ||
List<RelDataType> sqlTypes = opBinding.collectOperandTypes(); | ||
LogicalType[] parameters = UserDefinedFunctionUtils.getOperandTypeArray(opBinding); | ||
|
||
Object[] constantArguments = new Object[sqlTypes.size()]; | ||
for (int i = 0; i < sqlTypes.size(); i++) { | ||
if (!opBinding.isOperandNull(i, false) && opBinding.isOperandLiteral(i, false)) { | ||
constantArguments[i] = opBinding.getOperandLiteralValue( | ||
i, getDefaultExternalClassForType(parameters[i])); | ||
} | ||
} | ||
return invokeGetResultType(function, constantArguments, parameters, typeFactory); | ||
}; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters