dianfu commented on a change in pull request #11901:
URL: https://github.com/apache/flink/pull/11901#discussion_r415323127
##########
File path: flink-table/flink-sql-client/bin/sql-client.sh
##########
@@ -58,18 +58,25 @@ log_setting=(-Dlog.file="$log"
-Dlog4j.configuration=file:"$FLINK_CONF_DIR"/log4
# get path of jar in /opt if it exist
FLINK_SQL_CLIENT_JAR=$(find "$FLINK_OPT_DIR" -regex ".*flink-sql-client.*.jar")
+FLINK_PYTHON_JAR=$(find "$FLINK_OPT_DIR" -regex ".*flink-python.*.jar")
# check if SQL client is already in classpath and must not be shipped manually
-if [[ "$CC_CLASSPATH" =~ .*flink-sql-client.*.jar ]]; then
+if [[ "$CC_CLASSPATH" =~ .*flink-sql-client.*.jar && "$CC_CLASSPATH" =~
.*flink-python.*.jar ]]; then
# start client without jar
exec $JAVA_RUN $JVM_ARGS "${log_setting[@]}" -classpath "`manglePathList
"$CC_CLASSPATH:$INTERNAL_HADOOP_CLASSPATHS"`"
org.apache.flink.table.client.SqlClient "$@"
# check if SQL client jar is in /opt
elif [ -n "$FLINK_SQL_CLIENT_JAR" ]; then
Review comment:
In this case, maybe it's because `flink-sql-client` is in the classpath
and `flink-python` is not.
##########
File path:
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java
##########
@@ -696,6 +709,33 @@ private void registerTemporalTable(TemporalTableEntry
temporalTableEntry) {
}
}
+ private boolean hasPythonFunction(Environment environment) {
+ return environment.getFunctions().values().stream().anyMatch(f
->
+ FunctionDescriptorValidator.FROM_VALUE_PYTHON.equals(
+
f.getDescriptor().toProperties().get(FunctionDescriptorValidator.FROM)));
+ }
+
+ private List<URL> addPythonDependency(List<URL> dependencies) {
+ List<URL> newDependencies = new ArrayList<>(dependencies);
+ try {
+ URL location = Class.forName(
+ "org.apache.flink.python.PythonFunctionRunner",
+ false,
+ Thread.currentThread().getContextClassLoader())
+
.getProtectionDomain().getCodeSource().getLocation();
+ if (Paths.get(location.toURI()).toFile().isFile()) {
+ newDependencies.add(location);
+ } else {
+ throw new FlinkException("flink-python module
detected but is not a jar file: " + location + ".");
Review comment:
Should we use SqlExecutionException?
##########
File path:
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java
##########
@@ -696,6 +709,33 @@ private void registerTemporalTable(TemporalTableEntry
temporalTableEntry) {
}
}
+ private boolean hasPythonFunction(Environment environment) {
+ return environment.getFunctions().values().stream().anyMatch(f
->
+ FunctionDescriptorValidator.FROM_VALUE_PYTHON.equals(
+
f.getDescriptor().toProperties().get(FunctionDescriptorValidator.FROM)));
+ }
+
+ private List<URL> addPythonDependency(List<URL> dependencies) {
+ List<URL> newDependencies = new ArrayList<>(dependencies);
+ try {
+ URL location = Class.forName(
+ "org.apache.flink.python.PythonFunctionRunner",
+ false,
+ Thread.currentThread().getContextClassLoader())
+
.getProtectionDomain().getCodeSource().getLocation();
+ if (Paths.get(location.toURI()).toFile().isFile()) {
+ newDependencies.add(location);
+ } else {
+ throw new FlinkException("flink-python module
detected but is not a jar file: " + location + ".");
+ }
+ } catch (URISyntaxException | ClassNotFoundException |
FlinkException e) {
+ LOG.warn("Python UDF detected but flink-python jar not
found. " +
Review comment:
Should we throw exception in this case?
##########
File path:
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionService.java
##########
@@ -87,11 +108,22 @@ public static UserDefinedFunction createFunction(
new FunctionDescriptorValidator().validate(properties);
}
- // instantiate
- Object instance = generateInstance(
- HierarchyDescriptorValidator.EMPTY_PREFIX,
- properties,
- classLoader);
+ Object instance;
+ switch (properties.getString(FunctionDescriptorValidator.FROM))
{
+ case FunctionDescriptorValidator.FROM_VALUE_CLASS:
+ // instantiate
+ instance = generateInstance(
+
HierarchyDescriptorValidator.EMPTY_PREFIX,
+ properties,
+ classLoader);
+ break;
+ case FunctionDescriptorValidator.FROM_VALUE_PYTHON:
+ String fullyQualifiedName =
properties.getString(PythonFunctionValidator.FULLY_QUALIFIED_NAME);
+ instance =
PythonFunctionUtils.getPythonFunction(fullyQualifiedName, config);
+ break;
+ default:
+ throw new ValidationException("Unsupported
function descriptor: " + properties);
Review comment:
Use `String.format`
##########
File path:
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java
##########
@@ -696,6 +709,33 @@ private void registerTemporalTable(TemporalTableEntry
temporalTableEntry) {
}
}
+ private boolean hasPythonFunction(Environment environment) {
+ return environment.getFunctions().values().stream().anyMatch(f
->
+ FunctionDescriptorValidator.FROM_VALUE_PYTHON.equals(
+
f.getDescriptor().toProperties().get(FunctionDescriptorValidator.FROM)));
+ }
+
+ private List<URL> addPythonDependency(List<URL> dependencies) {
+ List<URL> newDependencies = new ArrayList<>(dependencies);
+ try {
+ URL location = Class.forName(
+ "org.apache.flink.python.PythonFunctionRunner",
+ false,
+ Thread.currentThread().getContextClassLoader())
+
.getProtectionDomain().getCodeSource().getLocation();
+ if (Paths.get(location.toURI()).toFile().isFile()) {
+ newDependencies.add(location);
+ } else {
+ throw new FlinkException("flink-python module
detected but is not a jar file: " + location + ".");
Review comment:
Use `String.format` instead of +
##########
File path:
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/python/utils/PythonFunctionUtils.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.functions.python.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.table.functions.python.PythonFunction;
+
+import java.lang.reflect.InvocationTargetException;
+
+/**
+ * Utilities for creating PythonFunction.
+ */
+@Internal
+public enum PythonFunctionUtils {
Review comment:
extra empty space before PythonFunctionUtils
##########
File path:
flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionService.java
##########
@@ -87,11 +108,22 @@ public static UserDefinedFunction createFunction(
new FunctionDescriptorValidator().validate(properties);
}
- // instantiate
- Object instance = generateInstance(
- HierarchyDescriptorValidator.EMPTY_PREFIX,
- properties,
- classLoader);
+ Object instance;
+ switch (properties.getString(FunctionDescriptorValidator.FROM))
{
+ case FunctionDescriptorValidator.FROM_VALUE_CLASS:
+ // instantiate
+ instance = generateInstance(
+
HierarchyDescriptorValidator.EMPTY_PREFIX,
+ properties,
+ classLoader);
+ break;
+ case FunctionDescriptorValidator.FROM_VALUE_PYTHON:
+ String fullyQualifiedName =
properties.getString(PythonFunctionValidator.FULLY_QUALIFIED_NAME);
+ instance =
PythonFunctionUtils.getPythonFunction(fullyQualifiedName, config);
+ break;
+ default:
+ throw new ValidationException("Unsupported
function descriptor: " + properties);
Review comment:
There is no need to make the `properties` in the exception. Just
`properties.getString(FunctionDescriptorValidator.FROM)` is enough? What do you
think?
##########
File path:
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java
##########
@@ -696,6 +709,33 @@ private void registerTemporalTable(TemporalTableEntry
temporalTableEntry) {
}
}
+ private boolean hasPythonFunction(Environment environment) {
Review comment:
minor: containsPythonFunction?
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]