LadyForest commented on code in PR #23478:
URL: https://github.com/apache/flink/pull/23478#discussion_r1365165868


##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala:
##########
@@ -345,44 +345,83 @@ object ScalarOperatorGens {
     }
   }
 
-  def generateEquals(
+  private def wrapExpressionIfNonEq(
+      isNonEq: Boolean,
+      equalsExpr: GeneratedExpression,
+      resultType: LogicalType): GeneratedExpression = {
+    if (isNonEq) {
+      GeneratedExpression(
+        s"(!${equalsExpr.resultTerm})",
+        equalsExpr.nullTerm,
+        equalsExpr.code,
+        resultType)
+    } else {
+      equalsExpr
+    }
+  }
+
+  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"$leftTerm.equals($rightTerm)")
+        (leftTerm, rightTerm) => s"${if (nonEq) "!" else 
""}$leftTerm.equals($rightTerm)")

Review Comment:
   Nit
   ```scala
         wrapExpressionIfNonEq(
           nonEq,
           generateOperatorIfNotNull(ctx, resultType, left, right)(
             (leftTerm, rightTerm) => s"$leftTerm.equals($rightTerm)"),
           resultType)
   ```



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala:
##########
@@ -409,32 +448,44 @@ object ScalarOperatorGens {
     // for performance, we cast literal string to literal time.
     else if (isTimePoint(left.resultType) && 
isCharacterString(right.resultType)) {
       if (right.literal) {
-        generateEquals(ctx, left, generateCastLiteral(ctx, right, 
left.resultType), resultType)
+        generateEqualAndNonEqual(
+          ctx,
+          left,
+          generateCastLiteral(ctx, right, left.resultType),
+          operator,
+          resultType)
       } else {
-        generateEquals(
+        generateEqualAndNonEqual(
           ctx,
           left,
           generateCast(ctx, right, left.resultType, nullOnFailure = true),
+          operator,
           resultType)
       }
     } else if (isTimePoint(right.resultType) && 
isCharacterString(left.resultType)) {
       if (left.literal) {
-        generateEquals(ctx, generateCastLiteral(ctx, left, right.resultType), 
right, resultType)
+        generateEqualAndNonEqual(
+          ctx,
+          generateCastLiteral(ctx, left, right.resultType),
+          right,
+          operator,
+          resultType)
       } else {
-        generateEquals(
+        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)) {

Review Comment:
   Nit
   ```scala
           // non comparable types
       else {
         val (newLeft, newRight) = if (isReference(left.resultType)) {
           (left, right)
         } else if (isReference(right.resultType)) {
           (right, left)
         } else {
           throw new CodeGenException(
             s"Incomparable types: ${left.resultType} and " +
               s"${right.resultType}")
         }
   
         wrapExpressionIfNonEq(
           nonEq,
           generateOperatorIfNotNull(ctx, resultType, newLeft, newRight) {
             (leftTerm, rightTerm) => s"$leftTerm.equals($rightTerm)"
           },
           resultType
         )
       }
        
   ```



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala:
##########
@@ -409,32 +448,44 @@ object ScalarOperatorGens {
     // for performance, we cast literal string to literal time.
     else if (isTimePoint(left.resultType) && 
isCharacterString(right.resultType)) {
       if (right.literal) {
-        generateEquals(ctx, left, generateCastLiteral(ctx, right, 
left.resultType), resultType)
+        generateEqualAndNonEqual(

Review Comment:
   Nit: maybe we can simplify the logic here
   ```scala
       else if (
         (isTimePoint(left.resultType) && isCharacterString(right.resultType)) 
|| (isTimePoint(
           right.resultType) && isCharacterString(left.resultType))
       ) {
         val (newLeft, newRight) =
           if (isTimePoint(left.resultType)) (left, right)
           else (right, left)
         generateEqualAndNonEqual(
           ctx,
           newLeft,
           if (newRight.literal) {
             generateCastLiteral(ctx, newRight, newLeft.resultType)
           } else {
             generateCast(ctx, newRight, newLeft.resultType, nullOnFailure = 
true)
           },
           operator,
           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: issues-unsubscr...@flink.apache.org

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

Reply via email to