Skip to content

Commit

Permalink
[FLINK-11881][table-planner-blink] Introduce code generated typed sor…
Browse files Browse the repository at this point in the history
…ter to blink table (apache#7958)
  • Loading branch information
JingsongLi authored and KurtYoung committed Mar 14, 2019
1 parent 0c062a3 commit e20a9f8
Show file tree
Hide file tree
Showing 61 changed files with 1,821 additions and 131 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,8 @@ package org.apache.flink.table.calcite

import java.util
import java.util.Properties

import com.google.common.collect.ImmutableList
import org.apache.calcite.config.{CalciteConnectionConfigImpl, CalciteConnectionProperty}
import org.apache.calcite.config.{CalciteConnectionConfigImpl, CalciteConnectionProperty, NullCollation}
import org.apache.calcite.jdbc.CalciteSchema
import org.apache.calcite.plan.RelOptTable.ViewExpander
import org.apache.calcite.plan._
Expand Down Expand Up @@ -221,4 +220,25 @@ object FlinkPlannerImpl {
rootSchema(schema.getParentSchema)
}
}

/**
* the null default direction if not specified. Consistent with HIVE/SPARK/MYSQL/BLINK-RUNTIME.
* So the default value only is set [[NullCollation.LOW]] for keeping consistent with
* BLINK-RUNTIME.
* [[NullCollation.LOW]] means null values appear first when the order is ASC (ascending), and
* ordered last when the order is DESC (descending).
*/
val defaultNullCollation: NullCollation = NullCollation.LOW

/** Returns the default null direction if not specified. */
def getNullDefaultOrders(ascendings: Array[Boolean]): Array[Boolean] = {
ascendings.map { asc =>
FlinkPlannerImpl.defaultNullCollation.last(!asc)
}
}

/** Returns the default null direction if not specified. */
def getNullDefaultOrder(ascending: Boolean): Boolean = {
FlinkPlannerImpl.defaultNullCollation.last(!ascending)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,21 @@

package org.apache.flink.table.codegen

import org.apache.flink.api.common.ExecutionConfig
import org.apache.flink.api.common.typeinfo.{AtomicType => AtomicTypeInfo}
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.core.memory.MemorySegment
import org.apache.flink.table.`type`._
import org.apache.flink.table.calcite.FlinkPlannerImpl
import org.apache.flink.table.dataformat._
import org.apache.flink.table.typeutils.TypeCheckUtils

import java.lang.reflect.Method
import java.lang.{Boolean => JBoolean, Byte => JByte, Character => JChar, Double => JDouble, Float => JFloat, Integer => JInt, Long => JLong, Short => JShort}
import java.util.concurrent.atomic.AtomicInteger

import scala.collection.mutable

object CodeGenUtils {

// ------------------------------- DEFAULT TERMS ------------------------------------------
Expand All @@ -36,9 +42,12 @@ object CodeGenUtils {
// -------------------------- CANONICAL CLASS NAMES ---------------------------------------

val BINARY_ROW: String = className[BinaryRow]
val BINARY_ARRAY: String = className[BinaryArray]
val BINARY_GENERIC: String = className[BinaryGeneric[_]]
val BINARY_STRING: String = className[BinaryString]
val BASE_ROW: String = className[BaseRow]
val GENERIC_ROW: String = className[GenericRow]
val SEGMENT: String = className[MemorySegment]

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

Expand Down Expand Up @@ -108,6 +117,7 @@ object CodeGenUtils {
case InternalTypes.TIMESTAMP => boxedTypeTermForType(InternalTypes.LONG)

case InternalTypes.STRING => BINARY_STRING
case InternalTypes.BINARY => "byte[]"

case _: DecimalType => className[Decimal]
// BINARY is also an ArrayType and uses BinaryArray internally too
Expand Down Expand Up @@ -230,4 +240,197 @@ object CodeGenUtils {
}
}

def baseRowFieldReadAccess(
ctx: CodeGeneratorContext, pos: Int, rowTerm: String, fieldType: InternalType) : String =
baseRowFieldReadAccess(ctx, pos.toString, rowTerm, fieldType)

def baseRowFieldReadAccess(
ctx: CodeGeneratorContext, pos: String, rowTerm: String, fieldType: InternalType) : String =
fieldType match {
case InternalTypes.INT => s"$rowTerm.getInt($pos)"
case InternalTypes.LONG => s"$rowTerm.getLong($pos)"
case InternalTypes.SHORT => s"$rowTerm.getShort($pos)"
case InternalTypes.BYTE => s"$rowTerm.getByte($pos)"
case InternalTypes.FLOAT => s"$rowTerm.getFloat($pos)"
case InternalTypes.DOUBLE => s"$rowTerm.getDouble($pos)"
case InternalTypes.BOOLEAN => s"$rowTerm.getBoolean($pos)"
case InternalTypes.STRING => s"$rowTerm.getString($pos)"
case InternalTypes.BINARY => s"$rowTerm.getBinary($pos)"
case dt: DecimalType => s"$rowTerm.getDecimal($pos, ${dt.precision()}, ${dt.scale()})"
case InternalTypes.CHAR => s"$rowTerm.getChar($pos)"
case _: TimestampType => s"$rowTerm.getLong($pos)"
case _: DateType => s"$rowTerm.getInt($pos)"
case InternalTypes.TIME => s"$rowTerm.getInt($pos)"
case _: ArrayType => s"$rowTerm.getArray($pos)"
case _: MapType => s"$rowTerm.getMap($pos)"
case rt: RowType => s"$rowTerm.getRow($pos, ${rt.getArity})"
case _: GenericType[_] => s"$rowTerm.getGeneric($pos)"
}

/**
* Generates code for comparing two field.
*/
def genCompare(
ctx: CodeGeneratorContext,
t: InternalType,
nullsIsLast: Boolean,
c1: String,
c2: String): String = t match {
case InternalTypes.BOOLEAN => s"($c1 == $c2 ? 0 : ($c1 ? 1 : -1))"
case _: PrimitiveType | _: DateType | _: TimeType | _: TimestampType =>
s"($c1 > $c2 ? 1 : $c1 < $c2 ? -1 : 0)"
case InternalTypes.BINARY =>
val sortUtil = classOf[org.apache.flink.table.runtime.sort.SortUtil].getCanonicalName
s"$sortUtil.compareBinary($c1, $c2)"
case at: ArrayType =>
val compareFunc = newName("compareArray")
val compareCode = genArrayCompare(
ctx,
FlinkPlannerImpl.getNullDefaultOrder(true), at, "a", "b")
val funcCode: String =
s"""
public int $compareFunc($BINARY_ARRAY a, $BINARY_ARRAY b) {
$compareCode
return 0;
}
"""
ctx.addReusableMember(funcCode)
s"$compareFunc($c1, $c2)"
case rowType: RowType =>
val orders = rowType.getFieldTypes.map(_ => true)
val comparisons = genRowCompare(
ctx,
rowType.getFieldTypes.indices.toArray,
rowType.getFieldTypes,
orders,
FlinkPlannerImpl.getNullDefaultOrders(orders),
"a",
"b")
val compareFunc = newName("compareRow")
val funcCode: String =
s"""
public int $compareFunc($BASE_ROW a, $BASE_ROW b) {
$comparisons
return 0;
}
"""
ctx.addReusableMember(funcCode)
s"$compareFunc($c1, $c2)"
case gt: GenericType[_] =>
val ser = ctx.addReusableObject(gt.getSerializer, "serializer")
val comp = ctx.addReusableObject(
gt.getTypeInfo.asInstanceOf[AtomicTypeInfo[_]].createComparator(true, new ExecutionConfig),
"comparator")
s"""
|$comp.compare(
| $BINARY_GENERIC.getJavaObjectFromBinaryGeneric($c1, $ser),
| $BINARY_GENERIC.getJavaObjectFromBinaryGeneric($c2, $ser)
|)
""".stripMargin
case other if other.isInstanceOf[AtomicType] => s"$c1.compareTo($c2)"
}

/**
* Generates code for comparing array.
*/
def genArrayCompare(
ctx: CodeGeneratorContext, nullsIsLast: Boolean, t: ArrayType, a: String, b: String)
: String = {
val nullIsLastRet = if (nullsIsLast) 1 else -1
val elementType = t.getElementType
val fieldA = newName("fieldA")
val isNullA = newName("isNullA")
val lengthA = newName("lengthA")
val fieldB = newName("fieldB")
val isNullB = newName("isNullB")
val lengthB = newName("lengthB")
val minLength = newName("minLength")
val i = newName("i")
val comp = newName("comp")
val typeTerm = primitiveTypeTermForType(elementType)
s"""
int $lengthA = a.numElements();
int $lengthB = b.numElements();
int $minLength = ($lengthA > $lengthB) ? $lengthB : $lengthA;
for (int $i = 0; $i < $minLength; $i++) {
boolean $isNullA = a.isNullAt($i);
boolean $isNullB = b.isNullAt($i);
if ($isNullA && $isNullB) {
// Continue to compare the next element
} else if ($isNullA) {
return $nullIsLastRet;
} else if ($isNullB) {
return ${-nullIsLastRet};
} else {
$typeTerm $fieldA = ${baseRowFieldReadAccess(ctx, i, a, elementType)};
$typeTerm $fieldB = ${baseRowFieldReadAccess(ctx, i, b, elementType)};
int $comp = ${genCompare(ctx, elementType, nullsIsLast, fieldA, fieldB)};
if ($comp != 0) {
return $comp;
}
}
}

if ($lengthA < $lengthB) {
return -1;
} else if ($lengthA > $lengthB) {
return 1;
}
"""
}

/**
* Generates code for comparing row keys.
*/
def genRowCompare(
ctx: CodeGeneratorContext,
keys: Array[Int],
keyTypes: Array[InternalType],
orders: Array[Boolean],
nullsIsLast: Array[Boolean],
row1: String,
row2: String): String = {

val compares = new mutable.ArrayBuffer[String]

for (i <- keys.indices) {
val index = keys(i)

val symbol = if (orders(i)) "" else "-"

val nullIsLastRet = if (nullsIsLast(i)) 1 else -1

val t = keyTypes(i)

val typeTerm = primitiveTypeTermForType(t)
val fieldA = newName("fieldA")
val isNullA = newName("isNullA")
val fieldB = newName("fieldB")
val isNullB = newName("isNullB")
val comp = newName("comp")

val code =
s"""
|boolean $isNullA = $row1.isNullAt($index);
|boolean $isNullB = $row2.isNullAt($index);
|if ($isNullA && $isNullB) {
| // Continue to compare the next element
|} else if ($isNullA) {
| return $nullIsLastRet;
|} else if ($isNullB) {
| return ${-nullIsLastRet};
|} else {
| $typeTerm $fieldA = ${baseRowFieldReadAccess(ctx, index, row1, t)};
| $typeTerm $fieldB = ${baseRowFieldReadAccess(ctx, index, row2, t)};
| int $comp = ${genCompare(ctx, t, nullsIsLast(i), fieldA, fieldB)};
| if ($comp != 0) {
| return $symbol$comp;
| }
|}
""".stripMargin
compares += code
}
compares.mkString
}

}
Loading

0 comments on commit e20a9f8

Please sign in to comment.