[ 
https://issues.apache.org/jira/browse/FLINK-8863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16508305#comment-16508305
 ] 

ASF GitHub Bot commented on FLINK-8863:
---------------------------------------

Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6090#discussion_r194442694
  
    --- Diff: 
flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/config/UDFDescriptor.java
 ---
    @@ -0,0 +1,153 @@
    +/*
    + * 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.client.config;
    +
    +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.table.client.SqlClientException;
    +import org.apache.flink.table.descriptors.ClassTypeDescriptor;
    +import org.apache.flink.table.descriptors.ClassTypeValidator;
    +import org.apache.flink.table.descriptors.DescriptorProperties;
    +import org.apache.flink.table.descriptors.FunctionDescriptor;
    +import org.apache.flink.table.descriptors.FunctionValidator;
    +import org.apache.flink.table.descriptors.PrimitiveTypeDescriptor;
    +import org.apache.flink.table.descriptors.PrimitiveTypeValidator;
    +import org.apache.flink.table.typeutils.TypeStringUtils;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +import static 
org.apache.flink.table.client.config.UDFDescriptor.From.CLASS;
    +
    +/**
    + * Descriptor for user-defined functions.
    + */
    +public class UDFDescriptor extends FunctionDescriptor {
    +
    +   private static final String FROM = "from";
    +
    +   private From from;
    +
    +   private UDFDescriptor(String name, From from) {
    +           super(name);
    +           this.from = from;
    +   }
    +
    +   public From getFrom() {
    +           return from;
    +   }
    +
    +   /**
    +    * Create a UDF descriptor with the given config.
    +    */
    +   public static UDFDescriptor create(Map<String, Object> config) {
    +           if (!config.containsKey(FunctionValidator.FUNCTION_NAME())) {
    +                   throw new SqlClientException("The 'name' attribute of a 
function is missing.");
    +           }
    +
    +           final Object name = 
config.get(FunctionValidator.FUNCTION_NAME());
    +           if (!(name instanceof String) || ((String) name).length() <= 0) 
{
    +                   throw new SqlClientException("Invalid function name '" 
+ name + "'.");
    +           }
    --- End diff --
    
    The following lines contain a lot of casting and assumption about data 
types. We should do it similar to `Source` instead and convert the incoming 
YAML to string-based properties first using `ConfigUtil.normalizeYaml`. Both 
Table API and SQL Client can then share one code path for validation. Similar 
how we do it for sources.


> Add user-defined function support in SQL Client
> -----------------------------------------------
>
>                 Key: FLINK-8863
>                 URL: https://issues.apache.org/jira/browse/FLINK-8863
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API &amp; SQL
>            Reporter: Timo Walther
>            Assignee: Xingcan Cui
>            Priority: Major
>
> This issue is a subtask of part two "Full Embedded SQL Client" of the 
> implementation plan mentioned in 
> [FLIP-24|https://cwiki.apache.org/confluence/display/FLINK/FLIP-24+-+SQL+Client].
>  
> It should be possible to declare user-defined functions in the SQL client. 
> For now, we limit the registration to classes that implement 
> {{ScalarFunction}}, {{TableFunction}}, {{AggregateFunction}}. Functions that 
> are implemented in SQL are not part of this issue. 
> I would suggest to introduce a {{functions}} top-level property. The 
> declaration could look similar to: 
> {code} 
> functions: 
>   - name: testFunction 
>     from: class <-- optional, default: class 
>     class: org.my.MyScalarFunction 
>     constructor: <-- optional, needed for certain types of functions 
>       - 42.0 
>       - class: org.my.Class <-- possibility to create objects via properties 
>         constructor: 
>           - 1 
>           - true 
>           - false 
>           - "whatever" 
>           - type: INT 
>             value: 1 
> {code} 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to