Skip to content

Commit

Permalink
[FLINK-3226] Translation of scalar function substring()
Browse files Browse the repository at this point in the history
  • Loading branch information
twalthr authored and vasia committed Mar 18, 2016
1 parent 907a095 commit 674fcc2
Show file tree
Hide file tree
Showing 23 changed files with 453 additions and 56 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -63,8 +63,12 @@ trait ImplicitExpressionOperations {
def count = Count(expr)
def avg = Avg(expr)

def substring(beginIndex: Expression, endIndex: Expression = Literal(Int.MaxValue)) = {
Substring(expr, beginIndex, endIndex)
def substring(beginIndex: Expression, endIndex: Expression) = {
Substring(expr, beginIndex, Some(endIndex))
}

def substring(beginIndex: Expression) = {
Substring(expr, beginIndex)
}

def cast(toType: TypeInformation[_]) = Cast(expr, toType)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,22 +19,24 @@
package org.apache.flink.api.table.codegen

import org.apache.calcite.rex._
import org.apache.calcite.sql.SqlOperator
import org.apache.calcite.sql.`type`.SqlTypeName._
import org.apache.calcite.sql.fun.SqlStdOperatorTable._
import org.apache.flink.api.common.functions.{FlatMapFunction, Function, MapFunction}
import org.apache.flink.api.common.functions.{FlatJoinFunction, FlatMapFunction, Function, MapFunction}
import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
import org.apache.flink.api.common.typeutils.CompositeType
import org.apache.flink.api.java.typeutils.{PojoTypeInfo, TupleTypeInfo}
import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
import org.apache.flink.api.table.TableConfig
import org.apache.flink.api.table.codegen.CodeGenUtils._
import org.apache.flink.api.table.codegen.Indenter.toISC
import org.apache.flink.api.table.codegen.OperatorCodeGen._
import org.apache.flink.api.table.codegen.calls.ScalarFunctions
import org.apache.flink.api.table.codegen.calls.ScalarOperators._
import org.apache.flink.api.table.plan.TypeConverter.sqlTypeToTypeInfo
import org.apache.flink.api.table.typeinfo.RowTypeInfo

import scala.collection.JavaConversions._
import scala.collection.mutable
import org.apache.flink.api.common.functions.FlatJoinFunction

/**
* A code generator for generating Flink [[org.apache.flink.api.common.functions.Function]]s.
Expand Down Expand Up @@ -540,7 +542,7 @@ class CodeGenerator(
}
case INTEGER =>
val decimal = BigDecimal(value.asInstanceOf[java.math.BigDecimal])
if (decimal.isValidShort) {
if (decimal.isValidInt) {
generateNonNullLiteral(resultType, decimal.intValue().toString)
}
else {
Expand Down Expand Up @@ -702,10 +704,19 @@ class CodeGenerator(
requireBoolean(operand)
generateNot(nullCheck, operand)

// casting
case CAST =>
val operand = operands.head
generateCast(nullCheck, operand, resultType)

// advanced scalar functions
case call: SqlOperator =>
val callGen = ScalarFunctions.getCallGenerator(call, operands.map(_.resultType))
callGen
.getOrElse(throw new CodeGenException(s"Unsupported call: $call"))
.generate(this, operands)

// unknown or invalid
case call@_ =>
throw new CodeGenException(s"Unsupported call: $call")
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
/*
* 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
*
* https://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.api.table.codegen.calls

import org.apache.flink.api.table.codegen.{CodeGenerator, GeneratedExpression}

trait CallGenerator {

def generate(
codeGenerator: CodeGenerator,
operands: Seq[GeneratedExpression])
: GeneratedExpression

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* 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
*
* https://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.api.table.codegen.calls

import java.lang.reflect.Method

import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.api.table.codegen.CodeGenUtils._
import org.apache.flink.api.table.codegen.{CodeGenerator, GeneratedExpression}

class MethodCallGenerator(returnType: TypeInformation[_], method: Method) extends CallGenerator {

override def generate(
codeGenerator: CodeGenerator,
operands: Seq[GeneratedExpression])
: GeneratedExpression = {
val resultTerm = newName("result")
val nullTerm = newName("isNull")
val resultTypeTerm = primitiveTypeTermForTypeInfo(returnType)
val defaultValue = primitiveDefaultValue(returnType)

val resultCode = if (codeGenerator.nullCheck) {
s"""
|${operands.map(_.code).mkString("\n")}
|boolean $nullTerm = ${operands.map(_.nullTerm).mkString(" || ")};
|$resultTypeTerm $resultTerm;
|if ($nullTerm) {
| $resultTerm = $defaultValue;
|}
|else {
| $resultTerm = ${method.getDeclaringClass.getCanonicalName}.${method.getName}(
| ${operands.map(_.resultTerm).mkString(", ")});
|}
|""".stripMargin
}
else {
s"""
|${operands.map(_.code).mkString("\n")}
|$resultTypeTerm $resultTerm = ${method.getDeclaringClass.getCanonicalName}.
| ${method.getName}(${operands.map(_.resultTerm).mkString(", ")});
|""".stripMargin
}

GeneratedExpression(resultTerm, nullTerm, resultCode, returnType)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* 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
*
* https://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.api.table.codegen.calls

import java.lang.reflect.Method

import org.apache.calcite.sql.SqlOperator
import org.apache.calcite.sql.fun.SqlStdOperatorTable._
import org.apache.calcite.util.BuiltInMethod
import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
import org.apache.flink.api.common.typeinfo.TypeInformation

import scala.collection.mutable

/**
* Global registry of built-in advanced SQL scalar functions.
*/
object ScalarFunctions {

private val sqlFunctions: mutable.Map[(SqlOperator, Seq[TypeInformation[_]]), CallGenerator] =
mutable.Map()

// ----------------------------------------------------------------------------------------------
addSqlFunctionMethod(
SUBSTRING,
Seq(STRING_TYPE_INFO, INT_TYPE_INFO, INT_TYPE_INFO),
STRING_TYPE_INFO,
BuiltInMethod.SUBSTRING.method)

addSqlFunctionMethod(
SUBSTRING,
Seq(STRING_TYPE_INFO, INT_TYPE_INFO),
STRING_TYPE_INFO,
BuiltInMethod.SUBSTRING.method)

// ----------------------------------------------------------------------------------------------

def getCallGenerator(
call: SqlOperator,
operandTypes: Seq[TypeInformation[_]])
: Option[CallGenerator] = {
sqlFunctions.get((call, operandTypes))
}

// ----------------------------------------------------------------------------------------------

private def addSqlFunctionMethod(
sqlOperator: SqlOperator,
operandTypes: Seq[TypeInformation[_]],
returnType: TypeInformation[_],
method: Method)
: Unit = {
sqlFunctions((sqlOperator, operandTypes)) = new MethodCallGenerator(returnType, method)
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -15,13 +15,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.api.table.codegen
package org.apache.flink.api.table.codegen.calls

import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
import org.apache.flink.api.common.typeinfo.{NumericTypeInfo, BasicTypeInfo, TypeInformation}
import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, NumericTypeInfo, TypeInformation}
import org.apache.flink.api.table.codegen.CodeGenUtils._
import org.apache.flink.api.table.codegen.{CodeGenException, GeneratedExpression}

object OperatorCodeGen {
object ScalarOperators {

def generateArithmeticOperator(
operator: String,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, IntegerTypeInfo}
case class Substring(
str: Expression,
beginIndex: Expression,
endIndex: Expression) extends Expression {
endIndex: Option[Expression] = None) extends Expression {
def typeInfo = {
if (str.typeInfo != BasicTypeInfo.STRING_TYPE_INFO) {
throw new ExpressionException(
Expand All @@ -33,14 +33,23 @@ case class Substring(
throw new ExpressionException(
s"""Begin index must be an integer type in $this, is ${beginIndex.typeInfo}.""")
}
if (!endIndex.typeInfo.isInstanceOf[IntegerTypeInfo[_]]) {
throw new ExpressionException(
s"""End index must be an integer type in $this, is ${endIndex.typeInfo}.""")
endIndex match {
case Some(endIdx) if !endIdx.typeInfo.isInstanceOf[IntegerTypeInfo[_]] =>
throw new ExpressionException(
s"""End index must be an integer type in $this, is ${endIdx.typeInfo}.""")
case _ => // ok
}

BasicTypeInfo.STRING_TYPE_INFO
}

override def children: Seq[Expression] = Seq(str, beginIndex, endIndex)
override def toString = s"($str).substring($beginIndex, $endIndex)"
override def children: Seq[Expression] = endIndex match {
case Some(endIdx) => Seq(str, beginIndex, endIdx)
case None => Seq(str, beginIndex)
}

override def toString = endIndex match {
case Some(endIdx) => s"($str).substring($beginIndex, $endIndex)"
case None => s"($str).substring($beginIndex)"
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -125,13 +125,13 @@ object ExpressionParser extends JavaTokenParsers with PackratParsers {

lazy val substring: PackratParser[Expression] =
atom ~ ".substring(" ~ expression ~ "," ~ expression ~ ")" ^^ {
case e ~ _ ~ from ~ _ ~ to ~ _ => Substring(e, from, to)
case e ~ _ ~ from ~ _ ~ to ~ _ => Substring(e, from, Some(to))

}

lazy val substringWithoutEnd: PackratParser[Expression] =
atom ~ ".substring(" ~ expression ~ ")" ^^ {
case e ~ _ ~ from ~ _ => Substring(e, from, Literal(Integer.MAX_VALUE))
case e ~ _ ~ from ~ _ => Substring(e, from)

}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,17 +52,25 @@ object RexNodeTranslator {
val l = extractAggCalls(b.left, relBuilder)
val r = extractAggCalls(b.right, relBuilder)
(b.makeCopy(List(l._1, r._1)), l._2 ::: r._2)
case s: Substring =>

// Scalar functions
case s@Substring(_, _, Some(endIndex)) =>
val str = extractAggCalls(s.str, relBuilder)
val sta = extractAggCalls(s.beginIndex, relBuilder)
val end = extractAggCalls(s.endIndex, relBuilder)
val end = extractAggCalls(endIndex, relBuilder)
(s.makeCopy(
List(str._1, sta._1, end._1)),
List(str._1, sta._1, Some(end._1))),
(str._2 ::: sta._2) ::: end._2
)
case e@_ =>

case s@Substring(_, _, None) =>
val str = extractAggCalls(s.str, relBuilder)
val sta = extractAggCalls(s.beginIndex, relBuilder)
(s.makeCopy(List(str._1, sta._1, None)), str._2 ::: sta._2)

case e@AnyRef =>
throw new IllegalArgumentException(
s"Expression ${e} of type ${e.getClass()} not supported yet")
s"Expression $e of type ${e.getClass} not supported yet")
}
}

Expand All @@ -72,6 +80,7 @@ object RexNodeTranslator {
def toRexNode(exp: Expression, relBuilder: RelBuilder): RexNode = {

exp match {
// Basic operators
case Literal(value, tpe) =>
relBuilder.literal(value)
case ResolvedFieldReference(name, tpe) =>
Expand Down Expand Up @@ -151,16 +160,24 @@ object RexNodeTranslator {
case UnaryMinus(child) =>
val c = toRexNode(child, relBuilder)
relBuilder.call(SqlStdOperatorTable.UNARY_MINUS, c)
case Substring(string, start, end) =>

// Scalar functions
case Substring(string, start, Some(end)) =>
val str = toRexNode(string, relBuilder)
val sta = toRexNode(start, relBuilder)
val en = toRexNode(end, relBuilder)
relBuilder.call(SqlStdOperatorTable.SUBSTRING, str, sta, en)

case Substring(string, start, None) =>
val str = toRexNode(string, relBuilder)
val sta = toRexNode(start, relBuilder)
relBuilder.call(SqlStdOperatorTable.SUBSTRING, str, sta)

case a: Aggregation =>
throw new IllegalArgumentException(s"Aggregation expression ${a} not allowed at this place")
case e@_ =>
throw new IllegalArgumentException(s"Aggregation expression $a not allowed at this place")
case e@AnyRef =>
throw new IllegalArgumentException(
s"Expression ${e} of type ${e.getClass()} not supported yet")
s"Expression $e of type ${e.getClass} not supported yet")
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,13 +25,11 @@
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.api.table.Table;
import org.apache.flink.api.table.Row;
import org.apache.flink.api.table.codegen.CodeGenException;
import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.table.TableEnvironment;
import org.apache.flink.api.table.test.TableProgramsTestBase;
import org.apache.flink.api.table.test.utils.TableProgramsTestBase;
import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
import org.apache.flink.test.util.MultipleProgramsTestBase;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
Expand Down
Loading

0 comments on commit 674fcc2

Please sign in to comment.