lincoln-lil commented on code in PR #23478:
URL: https://github.com/apache/flink/pull/23478#discussion_r1358298316
##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala:
##########
@@ -488,6 +488,18 @@ object ScalarOperatorGens {
else if (isNumeric(left.resultType) && isNumeric(right.resultType)) {
generateComparison(ctx, "!=", left, right, resultType)
}
+ // support date/time/timestamp not equalTo string.
+ else if (
+ (isTimePoint(left.resultType) && isCharacterString(right.resultType)) ||
Review Comment:
In order not to lose the simplicity of the operators and the null handling,
I've roughly modified a version that can be found below
```scala
private def wrapExpressionIfNonEq(
isNonEq: Boolean,
equalsExpr: GeneratedExpression,
resultType: LogicalType): GeneratedExpression = {
if (isNonEq) {
equalsExpr
} else {
GeneratedExpression(
s"(!${equalsExpr.resultTerm})",
equalsExpr.nullTerm,
equalsExpr.code,
resultType)
}
}
private def generateEqualAndNonEqual(
ctx: CodeGeneratorContext,
left: GeneratedExpression,
right: GeneratedExpression,
operator: String,
resultType: LogicalType): GeneratedExpression = {
checkImplicitConversionValidity(left, right)
val nonEq = operator match {
case "==" => false
case "!=" => true
case _ => throw new CodeGenException(s"Unsupported boolean comparison
'$operator'.")
}
val canEqual = isInteroperable(left.resultType, right.resultType)
if (isCharacterString(left.resultType) &&
isCharacterString(right.resultType)) {
generateOperatorIfNotNull(ctx, resultType, left, right)(
(leftTerm, rightTerm) => s"${if (nonEq) "!" else
""}$leftTerm.equals($rightTerm)")
}
// numeric types
else if (isNumeric(left.resultType) && isNumeric(right.resultType)) {
generateComparison(ctx, operator, left, right, resultType)
}
// array types
else if (isArray(left.resultType) && canEqual) {
wrapExpressionIfNonEq(
nonEq,
generateArrayComparison(ctx, left, right, resultType),
resultType)
}
// map types
else if (isMap(left.resultType) && canEqual) {
val mapType = left.resultType.asInstanceOf[MapType]
wrapExpressionIfNonEq(
nonEq,
generateMapComparison(
ctx,
left,
right,
mapType.getKeyType,
mapType.getValueType,
resultType),
resultType)
}
// multiset types
else if (isMultiset(left.resultType) && canEqual) {
val multisetType = left.resultType.asInstanceOf[MultisetType]
wrapExpressionIfNonEq(
nonEq,
generateMapComparison(
ctx,
left,
right,
multisetType.getElementType,
new IntType(false),
resultType),
resultType)
}
// comparable types of same type
else if (isComparable(left.resultType) && canEqual) {
generateComparison(ctx, operator, left, right, resultType)
}
// generic types of same type
else if (isRaw(left.resultType) && canEqual) {
val Seq(resultTerm, nullTerm) = newNames("result", "isNull")
val genericSer = ctx.addReusableTypeSerializer(left.resultType)
val ser = s"$genericSer.getInnerSerializer()"
val code =
s"""
|${left.code}
|${right.code}
|boolean $nullTerm = ${left.nullTerm}|| ${right.nullTerm};
|boolean $resultTerm = ${primitiveDefaultValue(resultType)};
|if (!$nullTerm) {
| ${left.resultTerm}.ensureMaterialized($ser);
| ${right.resultTerm}.ensureMaterialized($ser);
| $resultTerm =
| ${if (nonEq) "!" else
""}${left.resultTerm}.getBinarySection().
| equals(${right.resultTerm}.getBinarySection());
|}
|""".stripMargin
GeneratedExpression(resultTerm, nullTerm, code, resultType)
}
// support date/time/timestamp equalTo string.
// for performance, we cast literal string to literal time.
else if (isTimePoint(left.resultType) &&
isCharacterString(right.resultType)) {
if (right.literal) {
generateEqualAndNonEqual(
ctx,
left,
generateCastLiteral(ctx, right, left.resultType),
operator,
resultType)
} else {
generateEqualAndNonEqual(
ctx,
left,
generateCast(ctx, right, left.resultType, nullOnFailure = true),
operator,
resultType)
}
} else if (isTimePoint(right.resultType) &&
isCharacterString(left.resultType)) {
if (left.literal) {
generateEqualAndNonEqual(
ctx,
generateCastLiteral(ctx, left, right.resultType),
right,
operator,
resultType)
} else {
generateEqualAndNonEqual(
ctx,
generateCast(ctx, left, right.resultType, nullOnFailure = true),
right,
operator,
resultType)
}
}
// non comparable types
else {
generateOperatorIfNotNull(ctx, resultType, left, right) {
if (isReference(left.resultType)) {
(leftTerm, rightTerm) => s"${if (nonEq) "!" else
""}$leftTerm.equals($rightTerm)"
} else if (isReference(right.resultType)) {
(leftTerm, rightTerm) => s"${if (nonEq) "!" else
""}$rightTerm.equals($leftTerm)"
} else {
throw new CodeGenException(
s"Incomparable types: ${left.resultType} and " +
s"${right.resultType}")
}
}
}
}
def generateEquals(
ctx: CodeGeneratorContext,
left: GeneratedExpression,
right: GeneratedExpression,
resultType: LogicalType): GeneratedExpression = {
generateEqualAndNonEqual(ctx, left, right, "==", resultType)
}
def generateNotEquals(
ctx: CodeGeneratorContext,
left: GeneratedExpression,
right: GeneratedExpression,
resultType: LogicalType): GeneratedExpression = {
generateEqualAndNonEqual(ctx, left, right, "!=", resultType)
}
```
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]