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

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_r194456301
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/ClassTypeDescriptor.scala
 ---
    @@ -0,0 +1,71 @@
    +/*
    + * 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
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +/**
    +  * Descriptor for a class type.
    --- End diff --
    
    Descriptors might become public API in the future. We should add more 
comments and have a more fluent method scheme. I would propose something like:
    
    ```
    ClassInstance().of("org.example.MyFunction")
      .parameter("12.0")
      .parameter("DOUBLE", "12")
      .parameter(Types.DOUBLE, 12)
    ```
    
    I think we don't need to expose the primitive type descriptor. We can use 
it internally though. Alternatively, we could also move functionality to 
`org.apache.flink.table.descriptors.DescriptorProperties` if applicable.


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