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 2022/06/09 13:25:37 UTC

[GitHub] [flink] wuchong commented on a diff in pull request #19742: [FLINK-27651][table] Support CREATE FUNCTION USING JAR syntax

wuchong commented on code in PR #19742:
URL: https://github.com/apache/flink/pull/19742#discussion_r893475903


##########
flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl:
##########
@@ -305,9 +306,52 @@ SqlCreate SqlCreateFunction(Span s, boolean replace, boolean isTemporary) :
             <PYTHON>   { functionLanguage = "PYTHON"; }
         )
     ]
+    [ <USING>  {
+        if ("SQL".equals(functionLanguage) || "PYTHON".equals(functionLanguage)) {
+            throw new ParseException(
+                String.format("USING JAR syntax is not applicable to %s language.", functionLanguage));

Review Comment:
   Please use `throw SqlUtil.newContextException` to throw exception during parsing. 



##########
flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java:
##########
@@ -1314,6 +1314,25 @@ void testCreateFunction() {
                         "CREATE SYSTEM FUNCTION is not supported, "
                                 + "system functions can only be registered as temporary "
                                 + "function, you can use CREATE TEMPORARY SYSTEM FUNCTION instead.");
+
+        // test create function using jar
+        sql("create temporary function function1 as 'org.apache.fink.function.function1' language java using jar 'file:///path/to/test.jar'")
+                .ok(
+                        "CREATE TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1' LANGUAGE JAVA USING JAR 'file:///path/to/test.jar'");
+
+        sql("create temporary function function1 as 'org.apache.fink.function.function1' language scala using jar '/path/to/test.jar'")
+                .ok(
+                        "CREATE TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.fink.function.function1' LANGUAGE SCALA USING JAR '/path/to/test.jar'");

Review Comment:
   1. Test `CREATE TEMPORARY SYSTEM FUNCTION...`
   2. Test `USING FILE ... `



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/catalog/CatalogFunction.java:
##########
@@ -57,6 +59,9 @@ public interface CatalogFunction {
     /**
      * Distinguish if the function is a generic function.
      *
+     * @deprecated This method is currently only used in hive to determine if a function is a

Review Comment:
   There is already a deprecated message below. 



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/resource/ResourceUri.java:
##########
@@ -0,0 +1,66 @@
+/*
+ *  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.resource;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import java.util.Objects;
+
+/** Description of function resource information. */
+@PublicEvolving
+public class ResourceUri {
+
+    private final ResourceType resourceType;
+    private final String uri;
+
+    public ResourceUri(ResourceType resourceType, String uri) {
+        this.resourceType = resourceType;
+        this.uri = uri;
+    }
+
+    public ResourceType getResourceType() {
+        return resourceType;
+    }
+
+    public String getUri() {

Review Comment:
   Please add Javadocs to the public methods.



##########
flink-table/flink-table-common/src/main/java/org/apache/flink/table/resource/ResourceUri.java:
##########
@@ -0,0 +1,66 @@
+/*
+ *  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.resource;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import java.util.Objects;
+
+/** Description of function resource information. */
+@PublicEvolving
+public class ResourceUri {
+
+    private final ResourceType resourceType;
+    private final String uri;
+
+    public ResourceUri(ResourceType resourceType, String uri) {
+        this.resourceType = resourceType;
+        this.uri = uri;
+    }
+
+    public ResourceType getResourceType() {

Review Comment:
   Please add Javadocs to the public methods.



##########
flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl:
##########
@@ -305,9 +306,52 @@ SqlCreate SqlCreateFunction(Span s, boolean replace, boolean isTemporary) :
             <PYTHON>   { functionLanguage = "PYTHON"; }
         )
     ]
+    [ <USING>  {
+        if ("SQL".equals(functionLanguage) || "PYTHON".equals(functionLanguage)) {
+            throw new ParseException(
+                String.format("USING JAR syntax is not applicable to %s language.", functionLanguage));

Review Comment:
   Use the full syntax in exception message `CREATE FUNCTION USING JAR`



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org