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

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

Github user shaoxuan-wang commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3809#discussion_r114692276
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/AggSqlFunction.scala
 ---
    @@ -0,0 +1,177 @@
    +/*
    + * 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.utils
    +
    +import org.apache.calcite.rel.`type`.RelDataType
    +import org.apache.calcite.sql._
    +import org.apache.calcite.sql.`type`._
    +import org.apache.calcite.sql.`type`.SqlOperandTypeChecker.Consistency
    +import org.apache.calcite.sql.parser.SqlParserPos
    +import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction
    +import org.apache.flink.api.common.typeinfo._
    +import org.apache.flink.table.api.ValidationException
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.functions.AggregateFunction
    +import 
org.apache.flink.table.functions.utils.AggSqlFunction.{createOperandTypeChecker,
 createOperandTypeInference, createReturnTypeInference}
    +import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
    +
    +/**
    +  * Calcite wrapper for user-defined aggregate functions.
    +  *
    +  * @param name function name (used by SQL parser)
    +  * @param aggregateFunction aggregate function to be called
    +  * @param returnType the type information of returned value
    +  * @param typeFactory type factory for converting Flink's between 
Calcite's types
    +  */
    +class AggSqlFunction(
    +    name: String,
    +    aggregateFunction: AggregateFunction[_, _],
    +    returnType: TypeInformation[_],
    +    typeFactory: FlinkTypeFactory)
    +  extends SqlUserDefinedAggFunction(
    +    new SqlIdentifier(name, SqlParserPos.ZERO),
    +    createReturnTypeInference(returnType, typeFactory),
    +    createOperandTypeInference(aggregateFunction, typeFactory),
    +    createOperandTypeChecker(aggregateFunction),
    +    // Do not need to provide a calcite aggregateFunction here. Flink 
aggregateion function
    +    // will be generated when translating the calcite relnode to flink 
runtime execution plan
    +    null
    +  ) {
    +
    +  def getFunction: AggregateFunction[_, _] = aggregateFunction
    +}
    +
    +object AggSqlFunction {
    +
    +  def apply(
    +      name: String,
    +      aggregateFunction: AggregateFunction[_, _],
    +      returnType: TypeInformation[_],
    +      typeFactory: FlinkTypeFactory): AggSqlFunction = {
    +
    +    new AggSqlFunction(name, aggregateFunction, returnType, typeFactory)
    +  }
    +
    +  private[flink] def createOperandTypeInference(
    +      aggregateFunction: AggregateFunction[_, _],
    +      typeFactory: FlinkTypeFactory)
    +  : SqlOperandTypeInference = {
    +    /**
    +      * Operand type inference based on [[AggregateFunction]] given 
information.
    +      */
    +    new SqlOperandTypeInference {
    +      override def inferOperandTypes(
    +          callBinding: SqlCallBinding,
    +          returnType: RelDataType,
    +          operandTypes: Array[RelDataType]): Unit = {
    +
    +        val operandTypeInfo = getOperandTypeInfo(callBinding)
    +
    +        val foundSignature = 
getAccumulateMethodSignature(aggregateFunction, operandTypeInfo)
    +          .getOrElse(throw new ValidationException(s"Operand types of 
could not be inferred."))
    +
    +        val inferredTypes = getParameterTypes(aggregateFunction, 
foundSignature.drop(1))
    +          .map(typeFactory.createTypeFromTypeInfo)
    +
    +        for (i <- operandTypes.indices) {
    +          if (i < inferredTypes.length - 1) {
    +            operandTypes(i) = inferredTypes(i)
    +          } else if (null != inferredTypes.last.getComponentType) {
    +            // last argument is a collection, the array type
    +            operandTypes(i) = inferredTypes.last.getComponentType
    +          } else {
    +            operandTypes(i) = inferredTypes.last
    +          }
    +        }
    +      }
    +    }
    +  }
    +
    +  private[flink] def createReturnTypeInference(
    +      resultType: TypeInformation[_],
    +      typeFactory: FlinkTypeFactory)
    +  : SqlReturnTypeInference = {
    +
    +    new SqlReturnTypeInference {
    +      override def inferReturnType(opBinding: SqlOperatorBinding): 
RelDataType = {
    +        typeFactory.createTypeFromTypeInfo(resultType)
    +      }
    +    }
    +  }
    +
    +  private[flink] def createOperandTypeChecker(aggregateFunction: 
AggregateFunction[_, _])
    +  : SqlOperandTypeChecker = {
    +
    +    val signatures = getMethodSignatures(aggregateFunction, "accumulate")
    +
    +    /**
    +      * Operand type checker based on [[AggregateFunction]] given 
information.
    +      */
    +    new SqlOperandTypeChecker {
    +      override def getAllowedSignatures(op: SqlOperator, opName: String): 
String = {
    +        s"$opName[${signaturesToString(aggregateFunction, "accumulate")}]"
    +      }
    +
    +      override def getOperandCountRange: SqlOperandCountRange = {
    +        var min = 255
    +        var max = -1
    +        signatures.foreach(
    +          sig => {
    +            val inputSig = sig.drop(1)
    +            //do not count accumulator as input
    +            var len = inputSig.length
    +            if (len > 0 && inputSig(inputSig.length - 1).isArray) {
    +              max = 254 // according to JVM spec 4.3.3
    --- End diff --
    
    We want to make sure number of parameters of all codeGened functions within 
the [min, max], otherwise it will trigger the JVM failure. For accumulate and 
retract method, the max (for Operand which does not take into account the 
accumulator) here should be 253, as function pointer itself takes one, and 
accumulator takes one. Yes, we should consider to add test cases for huge 
amount of inputs and Array case.


> Add support to register UDAGG in Table and SQL API
> --------------------------------------------------
>
>                 Key: FLINK-5906
>                 URL: https://issues.apache.org/jira/browse/FLINK-5906
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: Fabian Hueske
>            Assignee: Shaoxuan Wang
>




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to