You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/04/26 06:20:26 UTC

[GitHub] [flink] dianfu commented on a change in pull request #11901: [FLINK-16971][python][sql-client] Support Python UDF in SQL Client.

dianfu commented on a change in pull request #11901:
URL: https://github.com/apache/flink/pull/11901#discussion_r415188144



##########
File path: flink-table/flink-sql-client/bin/sql-client.sh
##########
@@ -58,18 +58,19 @@ 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")

Review comment:
       I think we should check if the flink python jar already exists in the classpath just like how we did for the jar offlink-sql-client.

##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/PythonFunctionValidator.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.descriptors;
+
+/**
+ * Validator of python function.
+ */
+public class PythonFunctionValidator implements DescriptorValidator {

Review comment:
       Annotate as `@Internal`

##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java
##########
@@ -696,6 +710,32 @@ 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> addPythonFunctionDependency(List<URL> dependencies) {
+		List<URL> newDependencies = new ArrayList<>(dependencies);
+		URL location = PythonFunctionRunner.class.getProtectionDomain().getCodeSource().getLocation();

Review comment:
       Can we use reflection and then this module will not depend on flink-python any more?

##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java
##########
@@ -696,6 +710,32 @@ 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> addPythonFunctionDependency(List<URL> dependencies) {

Review comment:
       rename to addPythonDependency

##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java
##########
@@ -517,6 +526,7 @@ private void createTableEnvironment(
 			CatalogManager catalogManager,
 			ModuleManager moduleManager,
 			FunctionCatalog functionCatalog) {
+		config.addConfiguration(flinkConfig);

Review comment:
       What's the aim of this change?

##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/FunctionService.java
##########
@@ -87,11 +108,37 @@ 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);
+				try {
+					Class pythonFunctionFactory = Class.forName(

Review comment:
       Many code here is duplicate with `FunctionDefinitionUtil.createPythonFunctionDefinition`. Could we abstract it a little to remove the duplication?

##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/descriptors/FunctionDescriptor.java
##########
@@ -29,13 +29,22 @@
 public class FunctionDescriptor implements Descriptor {
 	private String from;
 	private ClassInstance classInstance;
+	private String fullyQualifiedName;
 
 	/**
 	 * Creates a function from a class description.
 	 */
 	public FunctionDescriptor fromClass(ClassInstance classType) {
 		from = FunctionDescriptorValidator.FROM_VALUE_CLASS;
 		this.classInstance = classType;
+		this.fullyQualifiedName = null;
+		return this;
+	}
+
+	public FunctionDescriptor fromPython(String fullyQualifiedName) {
+		from = FunctionDescriptorValidator.FROM_VALUE_PYTHON;

Review comment:
       Add precondition check that `classInstance` should be null?

##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java
##########
@@ -696,6 +710,32 @@ 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> addPythonFunctionDependency(List<URL> dependencies) {
+		List<URL> newDependencies = new ArrayList<>(dependencies);
+		URL location = PythonFunctionRunner.class.getProtectionDomain().getCodeSource().getLocation();
+		try {
+			if (Paths.get(location.toURI()).toFile().isFile()) {
+				newDependencies.add(location);
+			} else {
+				LOG.warn(
+					"Python UDF detected but flink-python jar not found. " +
+						"If you starts SQL-Client via `sql-client.sh`, " +
+						"please add the flink-python jar via `-j` command option manually.");
+			}
+		} catch (URISyntaxException e) {
+			LOG.warn("Python UDF detected but flink-python jar not found. " +

Review comment:
       Could we avoid this duplication lines?

##########
File path: flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java
##########
@@ -198,6 +201,22 @@ public void testFunctions() throws Exception {
 		assertArrayEquals(expected, actual);
 	}
 
+	@Test
+	public void testPythonFunction() throws Exception {

Review comment:
       The test case could be improved as following:
   ```
   PowerMockito.stub(PowerMockito.method(PythonFunctionFactory.class, "getPythonFunctionFactory", ReadableConfig.class)).toReturn(
   			(PythonFunctionFactory) (moduleName, objectName) -> new PythonScalarFunction(null, null, null, null, null, false, null));
   ```




----------------------------------------------------------------
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:
users@infra.apache.org