You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/03/14 14:29:20 UTC

[GitHub] [flink] twalthr commented on a change in pull request #19001: [FLINK-26520][table] Implement SEARCH operator in codegen

twalthr commented on a change in pull request #19001:
URL: https://github.com/apache/flink/pull/19001#discussion_r825981925



##########
File path: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala
##########
@@ -209,13 +209,27 @@ object CodeGenUtils {
     case _: JShort => s"((short)$value)"
     case _: JInt => value.toString
     case _: JLong => value.toString + "L"
-    case _: JFloat => value.toString + "f"
-    case _: JDouble => value.toString + "d"
+    case _: JFloat => value match {
+      case JFloat.NEGATIVE_INFINITY => "java.lang.Float.NEGATIVE_INFINITY"
+      case JFloat.POSITIVE_INFINITY => "java.lang.Float.POSITIVE_INFINITY"
+      case _ => value.toString + "f"
+    }
+    case _: JDouble => value match {
+      case JDouble.NEGATIVE_INFINITY => "java.lang.Double.NEGATIVE_INFINITY"
+      case JDouble.POSITIVE_INFINITY => "java.lang.Double.POSITIVE_INFINITY"
+      case _ => value.toString + "d"
+    }
     case sd: StringData =>
       qualifyMethod(BINARY_STRING_DATA_FROM_STRING) + "(\"" +
         EncodingUtils.escapeJava(sd.toString) + "\")"
     case td: TimestampData =>
       s"$TIMESTAMP_DATA.fromEpochMillis(${td.getMillisecond}L, ${td.getNanoOfMillisecond})"
+    case decimalData: DecimalData =>
+      s"""$DECIMAL_UTIL.castFrom(

Review comment:
       nit: wouldn't unscaled long/bytes be cheaper? at least we could use the compact representation for simple numbers.

##########
File path: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala
##########
@@ -313,164 +306,53 @@ object GenerateUtils {
       literalValue = Some(literalValue))
   }
 
+  /**
+   * This function accepts the Flink's internal type system.

Review comment:
       `type system` -> `data structures`
   
   A type system describes data. Data structures contain/represent data.

##########
File path: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala
##########
@@ -420,14 +420,29 @@ class CodeGeneratorContext(val tableConfig: TableConfig) {
     }
 
     val addElementsCode = elements.map { element =>
-      s"""
-         |${element.code}
-         |if (${element.nullTerm}) {
-         |  $fieldTerm.addNull();
-         |} else {
-         |  $fieldTerm.add(${element.resultTerm});
-         |}
-         |""".stripMargin
+      if (element.literalValue.isDefined) {
+        // Don't generate the null check in case the element is a literal expression
+        if (element.literalValue.get != null) {
+          s"""
+             |${element.code}
+             |$fieldTerm.add(${element.resultTerm});
+             |""".stripMargin
+        } else if (element.literalValue.get == null) {
+          s"""
+             |${element.code}

Review comment:
       unnecessary code

##########
File path: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/SearchOperatorGen.scala
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.planner.codegen.calls
+
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory
+import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, CodeGenException, GeneratedExpression}
+import org.apache.flink.table.planner.codegen.CodeGenUtils.newNames
+import org.apache.flink.table.planner.codegen.GenerateUtils.{generateLiteral, generateNullLiteral}
+import org.apache.flink.table.planner.codegen.calls.ScalarOperatorGens._
+import org.apache.flink.table.planner.functions.casting.CastRuleProvider
+import org.apache.flink.table.planner.plan.utils.RexLiteralUtil.toFlinkInternalValue
+import org.apache.flink.table.types.logical.{BooleanType, LogicalType}
+import org.apache.flink.table.types.logical.utils.LogicalTypeMerging.findCommonType
+
+import org.apache.calcite.rex.RexLiteral
+import org.apache.calcite.util.{RangeSets, Sarg}
+
+import java.util.Arrays.asList
+
+import scala.collection.JavaConverters._
+
+/**
+ * Class containing utilities to implement the SEARCH operator.
+ *
+ * This does not implement [[CallGenerator]] as the interface does not fit, because the [[Sarg]]
+ * argument cannot be converted directly to [[GeneratedExpression]].
+ */
+object SearchOperatorGen {
+
+  /**
+   * Generates SEARCH expression using either an HashSet or a concatenation of OR,
+   * depending on whether the elements of the haystack are all literals or not.
+   *
+   * Note that both IN/NOT IN are converted to SEARCH when the set has only constant values,
+   * otherwise the IN/NOT IN are converted to a set of disjunctions. See
+   * [[org.apache.calcite.rex.RexBuilder#makeIn(org.apache.calcite.rex.RexNode, java.util.List)]].
+   */
+  def generateSearch(
+       ctx: CodeGeneratorContext,
+       target: GeneratedExpression,
+       sargLiteral: RexLiteral): GeneratedExpression = {
+    val sarg: Sarg[Nothing] = sargLiteral.getValueAs(classOf[Sarg[Nothing]])
+    val targetType = target.resultType
+    val sargType = FlinkTypeFactory.toLogicalType(sargLiteral.getType)
+
+    val commonType: LogicalType = findCommonType(asList(targetType, sargType))
+      .orElseThrow(() =>
+        new CodeGenException(s"Unable to find common type of $target and $sargLiteral."))
+
+    val needle = generateCast(
+      ctx,
+      target,
+      commonType,
+      nullOnFailure = false
+    )
+
+    // In case the search is among points we use the hashset implementation
+    if (sarg.isPoints || sarg.isComplementedPoints) {
+      val rangeSet = if (sarg.isPoints) sarg.rangeSet else sarg.rangeSet.complement()
+      val haystack = rangeSet
+        .asRanges()
+        .asScala
+        // We need to go through the generateLiteral to normalize the value from calcite
+        .map(r => toFlinkInternalValue(r.lowerEndpoint, sargType))
+        // The elements are constant, we perform the cast at priori

Review comment:
       nit: `a priori` or better `instantly` to help non-native speakers ;)

##########
File path: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RexLiteralUtil.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.planner.plan.utils
+
+import org.apache.flink.table.data.{DecimalData, TimestampData}
+import org.apache.flink.table.data.binary.BinaryStringData
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory
+import org.apache.flink.table.planner.codegen.CodeGenException
+import org.apache.flink.table.planner.utils.TimestampStringUtils.toLocalDateTime
+import org.apache.flink.table.types.logical.{DistinctType, LogicalType}
+import org.apache.flink.table.types.logical.LogicalTypeRoot._
+import org.apache.flink.table.types.logical.utils.LogicalTypeChecks
+
+import org.apache.calcite.avatica.util.ByteString
+import org.apache.calcite.rex.RexLiteral
+import org.apache.calcite.util.{DateString, NlsString, TimestampString, TimeString}
+
+import java.lang.{Boolean => JBoolean, Integer => JInteger, Number => JNumber}
+import java.math.{BigDecimal => JBigDecimal}
+
+import scala.annotation.tailrec
+
+/**
+ * Utilities to work with [[RexLiteral]]
+ */
+object RexLiteralUtil {
+
+  /**
+   * See [[toFlinkInternalValue(Comparable, LogicalType)]].
+   */
+  def toFlinkInternalValue(literal: RexLiteral): (Any, LogicalType) = {
+    val targetType = FlinkTypeFactory.toLogicalType(literal.getType)
+    (toFlinkInternalValue(literal.getValueAs(classOf[Comparable[_]]), targetType), targetType)
+  }
+
+  /**
+   * Convert a value from Calcite's [[Comparable]] type system to Flink internal type system,
+   * and also tries to be a bit flexible by accepting usual Java types such as String and boxed
+   * numerics.
+   *
+   * This function is essentially like [[FlinkTypeFactory.toLogicalType()]] but for values.
+   *
+   * Check [[RexLiteral.valueMatchesType]] for details on the [[Comparable]] type system and
+   * [[org.apache.flink.table.data.RowData]] for details on Flink's internal type system.
+   *
+   * @param value the value in Calcite's [[Comparable]] type system
+   * @param valueType the type of the value
+   * @return the value in Flink's internal type system
+   * @throws IllegalArgumentException in case the class of value does not match the expectations
+   *                                  of valueType
+   */
+  @tailrec
+  def toFlinkInternalValue(value: Comparable[_], valueType: LogicalType): Any = {
+    if (value == null) {
+      return null
+    }
+    valueType.getTypeRoot match {
+      case CHAR | VARCHAR =>
+        value match {
+          case nlsString: NlsString => BinaryStringData.fromString(nlsString.getValue)
+          case string: String => BinaryStringData.fromString(string)
+        }
+
+      case BOOLEAN =>
+        value match {
+          case boolean: JBoolean => boolean
+        }
+
+      case BINARY | VARBINARY =>
+        value match {
+          case byteString: ByteString => byteString.getBytes
+        }
+
+      case DECIMAL =>
+        value match {
+          case bigDecimal: JBigDecimal => DecimalData.fromBigDecimal(
+            bigDecimal,
+            LogicalTypeChecks.getPrecision(valueType),
+            LogicalTypeChecks.getScale(valueType))
+        }
+
+      case TINYINT =>
+        value match {
+          case number: JNumber => number.byteValue()
+        }
+
+      case SMALLINT =>
+        value match {
+          case number: JNumber => number.shortValue()
+        }
+
+      case INTEGER =>
+        value match {
+          case number: JNumber => number.intValue()
+        }
+
+      case BIGINT =>
+        value match {
+          case number: JNumber => number.longValue()
+        }
+
+      case FLOAT =>
+        value match {
+          case number: JNumber => number.floatValue()
+        }
+
+      case DOUBLE =>
+        value match {
+          case number: JNumber => number.doubleValue()
+        }
+
+      case DATE =>
+        value match {
+          case dateStringValue: DateString => dateStringValue.getDaysSinceEpoch
+          case intValue: JInteger => intValue
+        }
+
+      case TIME_WITHOUT_TIME_ZONE =>
+        value match {
+          case timeStringValue: TimeString => timeStringValue.getMillisOfDay
+          case intValue: JInteger => intValue
+        }
+
+      case TIMESTAMP_WITHOUT_TIME_ZONE =>
+        value match {
+          case timestampString: TimestampString =>
+            TimestampData.fromLocalDateTime(toLocalDateTime(timestampString))
+        }
+
+      case TIMESTAMP_WITH_LOCAL_TIME_ZONE =>
+        value match {
+          case timestampString: TimestampString =>
+            TimestampData.fromLocalDateTime(toLocalDateTime(timestampString))
+        }
+
+      case INTERVAL_YEAR_MONTH =>
+        value match {
+          case number: JNumber => number.intValue()
+        }
+
+      case INTERVAL_DAY_TIME =>
+        value match {
+          case number: JNumber => number.longValue()
+        }
+
+      case DISTINCT_TYPE =>
+        toFlinkInternalValue(value, valueType.asInstanceOf[DistinctType].getSourceType)
+
+      case SYMBOL =>
+        value.asInstanceOf[Enum[_]]

Review comment:
       So Calcite symbols remain? Maybe this should be mentioned in the JavaDocs.

##########
File path: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala
##########
@@ -464,19 +465,10 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, nullableInput: Boolean)
   override def visitCall(call: RexCall): GeneratedExpression = {
     val resultType = FlinkTypeFactory.toLogicalType(call.getType)
     if (call.getKind == SqlKind.SEARCH) {

Review comment:
       do we still need this early check here or can we simply use the regular switch/case list below?

##########
File path: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/LikeCallGen.scala
##########
@@ -109,16 +109,16 @@ class LikeCallGen extends CallGenerator {
             val middleMatcher = MIDDLE_PATTERN.matcher(newPattern)
 
             if (noneMatcher.matches()) {
-              val reusePattern = ctx.addReusableStringConstants(newPattern)
+              val reusePattern = ctx.addReusableEscapedStringConstant(newPattern)

Review comment:
       nit: method is quite long `addReusableEscapedStringConstant` -> `addReusableEscapedString`

##########
File path: flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RexLiteralUtil.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.planner.plan.utils
+
+import org.apache.flink.table.data.{DecimalData, TimestampData}
+import org.apache.flink.table.data.binary.BinaryStringData
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory
+import org.apache.flink.table.planner.codegen.CodeGenException
+import org.apache.flink.table.planner.utils.TimestampStringUtils.toLocalDateTime
+import org.apache.flink.table.types.logical.{DistinctType, LogicalType}
+import org.apache.flink.table.types.logical.LogicalTypeRoot._
+import org.apache.flink.table.types.logical.utils.LogicalTypeChecks
+
+import org.apache.calcite.avatica.util.ByteString
+import org.apache.calcite.rex.RexLiteral
+import org.apache.calcite.util.{DateString, NlsString, TimestampString, TimeString}
+
+import java.lang.{Boolean => JBoolean, Integer => JInteger, Number => JNumber}
+import java.math.{BigDecimal => JBigDecimal}
+
+import scala.annotation.tailrec
+
+/**
+ * Utilities to work with [[RexLiteral]]
+ */
+object RexLiteralUtil {

Review comment:
       why did you implement this in Scala? :'( 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org