Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-33171][table planner] Consistent implicit type coercion support for equal and non-equal comparisons for codegen #23478

Merged
merged 5 commits into from
Nov 6, 2023

Conversation

fengjiajie
Copy link
Contributor

@fengjiajie fengjiajie commented Sep 30, 2023

What is the purpose of the change

When executing the following SQL:

SELECT
    time1,
    time1 = '2023-09-30 18:22:42.123' AS eq1,
    NOT (time1 = '2023-09-30 18:22:42.123') AS notEq1
FROM table1;

the result is as follows:

+----+-------------------------+--------+--------+
| op |                   time1 |    eq1 | notEq1 |
+----+-------------------------+--------+--------+
| +I | 2023-09-30 18:22:42.123 |   TRUE |   TRUE |
| +I | 2023-09-30 18:22:42.124 |  FALSE |   TRUE |
+----+-------------------------+--------+--------+
2 rows in set

The "notEq1" in the first row should be FALSE.

import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.api.java.typeutils.RowTypeInfo;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.datastream.DataStreamSource;
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.Schema;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
import org.apache.flink.types.Row;

public class TimePointNotEqualTest {
    public static void main(String[] args) throws Exception {
        StreamExecutionEnvironment env =
                StreamExecutionEnvironment.getExecutionEnvironment(new Configuration());
        env.setParallelism(1);

        DataStreamSource<Long> longDataStreamSource = env.fromSequence(0, 1);
        RowTypeInfo rowTypeInfo =
                new RowTypeInfo(new TypeInformation[] {Types.LONG}, new String[] {"time1"});
        SingleOutputStreamOperator<Row> map =
                longDataStreamSource.map(new RichMapFunction<Long, Row>() {
                    @Override
                    public Row map(Long value) {
                        Row row = new Row(1);
                        row.setField(0, 1696069362123L + value);
                        return row;
                    }
                }, rowTypeInfo);

        StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env);
        Schema schema = Schema.newBuilder()
                .column("time1", DataTypes.TIMESTAMP_LTZ(3).bridgedTo(Long.class))
                .build();
        tableEnv.createTemporaryView("table1", map, schema);

        tableEnv.sqlQuery("SELECT "
                + "time1," // 2023-09-30 18:22:42.123
                + "time1 = '2023-09-30 18:22:42.123' AS eq1," // expect TRUE
                + "NOT (time1 = '2023-09-30 18:22:42.123') AS notEq1 " // expect FALSE but TRUE
                + "FROM table1").execute().print();
    }
}

Brief change log

Add TimePoint not equalTo String code in ScalarOperatorGens

Verifying this change

Running the same SQL above will yield the correct result:

+----+-------------------------+--------+--------+
| op |                   time1 |    eq1 | notEq1 |
+----+-------------------------+--------+--------+
| +I | 2023-09-30 18:22:42.123 |   TRUE |  FALSE |
| +I | 2023-09-30 18:22:42.124 |  FALSE |   TRUE |
+----+-------------------------+--------+--------+
2 rows in set

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): (yes / no)
  • The public API, i.e., is any changed class annotated with @Public(Evolving): (yes / no)
  • The serializers: (yes / no / don't know)
  • The runtime per-record code paths (performance sensitive): (yes / no / don't know)
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (yes / no / don't know)
  • The S3 file system connector: (yes / no / don't know)

Documentation

  • Does this pull request introduce a new feature? (yes / no)
  • If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)

@flinkbot
Copy link
Collaborator

flinkbot commented Sep 30, 2023

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run azure re-run the last Azure build

@fengjiajie fengjiajie changed the base branch from release-1.17 to master September 30, 2023 11:37
Copy link
Contributor

@LadyForest LadyForest left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the contribution. The fix looks good to me.

Copy link
Contributor

@lincoln-lil lincoln-lil left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@fengjiajie thanks for contributing this! I left two comments.

@@ -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)) ||
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add optimizations for string literals similar to the handling of generateEquals?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If the condition is met, generate code using generateEquals, where the optimizations are also applied and the code is reused.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm wondering why generateNotEquals doesn't reuse the logic of generateEquals (which ends up wrapped in a not expression)? This is the direct cause of the current problem: the logic for data types of generatingNotEquals and generatingEquals is not aligned (In addition to the timepoint change here, the multiset and generic types are handled differently).
So why don't we switch to full reuse of generatingEquals to eliminate these inconsistent behaviors altogether ? Because at the moment I don't see a semantic need to differentiate between equal and nonEqual handling (including three-valued logic).
cc @LadyForest @libenchao because I saw your discussion in jira.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is an interesting idea, I like it.

I'm not sure about why this is designed to have a dedicated method originally, I'm guessing that maybe it can directly translate to left != right which may have slight better performance than !(left == right), and will be more complex considering null handling.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

  
  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)
  }

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@lincoln-lil This is a better solution! Should I include it in the code commit or assign the issue to you? I can also help add some corresponding unit tests if needed.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

handling in 'wrapExpressionIfNonEq' should be reversed

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@fengjiajie just feel free to move forward, and let's fix this issue completely!

@fengjiajie
Copy link
Contributor Author

@LadyForest Thank you for your review, and I have seen the discussion on Jira.

@fengjiajie
Copy link
Contributor Author

@lincoln-lil
Thank you for your review. Are the 5 datatypes you mentioned referring to DATE, TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITH_TIME_ZONE, and TIMESTAMP_WITH_LOCAL_TIME_ZONE?
In the previous test code, there were f15, f21, and f22 corresponding to the first 3 types. I attempted to add the last two, but encountered an exception:

org.apache.flink.table.api.TableException: Type is not supported: TIMESTAMP_WITH_TIME_ZONE
	at org.apache.flink.table.planner.calcite.FlinkTypeFactory.newRelDataType$1(FlinkTypeFactory.scala:152)

So now I have only added f23, which corresponds to the TIMESTAMP_WITH_LOCAL_TIME_ZONE. If any further modifications are needed, please let me know. Thank you.

@fengjiajie
Copy link
Contributor Author

@flinkbot run azure

Copy link
Contributor

@lincoln-lil lincoln-lil left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@fengjiajie thanks for the updates! Just two minor comments.

@@ -488,6 +488,39 @@ object ScalarOperatorGens {
else if (isNumeric(left.resultType) && isNumeric(right.resultType)) {
generateComparison(ctx, "!=", left, right, resultType)
}
// support date/time/timestamp not equalTo string.
// for performance, we cast literal string to literal time.
else if (isTimePoint(left.resultType) && isCharacterString(right.resultType)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we extract common method with generateEquals

@lincoln-lil
Copy link
Contributor

@lincoln-lil Thank you for your review. Are the 5 datatypes you mentioned referring to DATE, TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITH_TIME_ZONE, and TIMESTAMP_WITH_LOCAL_TIME_ZONE? In the previous test code, there were f15, f21, and f22 corresponding to the first 3 types. I attempted to add the last two, but encountered an exception:

org.apache.flink.table.api.TableException: Type is not supported: TIMESTAMP_WITH_TIME_ZONE
	at org.apache.flink.table.planner.calcite.FlinkTypeFactory.newRelDataType$1(FlinkTypeFactory.scala:152)

So now I have only added f23, which corresponds to the TIMESTAMP_WITH_LOCAL_TIME_ZONE. If any further modifications are needed, please let me know. Thank you.

@fengjiajie The 5 types are correct, I checked the code in FlinkTypeFactory and also verified it in sql client, the TIMESTAMP_WITH_TIME_ZONE type is unsupported, this can be a separate issue to follow.

errors in sql client when create a column with TIMESTAMP WITH TIME ZONE:

Flink SQL> CREATE TABLE Bid (
>   bid STRING,
>   price BIGINT,
>   rowtime TIMESTAMP WITH TIME ZONE,
>   -- declare user_action_time as event time attribute and use 5 seconds delayed watermark strategy
>   WATERMARK FOR rowtime AS rowtime - INTERVAL '5' SECOND
> ) WITH ( 'connector' = 'datagen');
[ERROR] Could not execute SQL statement. Reason:
org.apache.flink.sql.parser.impl.ParseException: Encountered "TIME" at line 4, column 26.
Was expecting:
    "LOCAL" ...

@fengjiajie
Copy link
Contributor Author

@lincoln-lil
Hi, I changed back to the previous approach. Please see if this is workable: Keep the 'if' statement to indicate the case of handling TimePoint and TimeString, and call generateEquals to reuse code and optimize.

@fengjiajie
Copy link
Contributor Author

@lincoln-lil Based on the approach you provided, I attempted to write some test cases. However, there are two types that are not covered:

  1. non comparable types: I don't know how to construct case for this one.
  2. multiset types: The framework throws an error when using a multiset.
    testSqlApi("f26 = MULTISET['b', 'a', 'b']", "TRUE")
    
    got exception:
    org.opentest4j.AssertionFailedError: Expression is converted into more than a Calc operation. Use a different test method. ==> expected: <true> but was: <false>
     at org.apache.flink.table.planner.expressions.utils.ExpressionTestBase.addTestExpr(ExpressionTestBase.scala:324)
    

@lincoln-lil
Copy link
Contributor

@lincoln-lil Based on the approach you provided, I attempted to write some test cases. However, there are two types that are not covered:

  1. non comparable types: I don't know how to construct case for this one.

  2. multiset types: The framework throws an error when using a multiset.

    testSqlApi("f26 = MULTISET['b', 'a', 'b']", "TRUE")
    

    got exception:

    org.opentest4j.AssertionFailedError: Expression is converted into more than a Calc operation. Use a different test method. ==> expected: <true> but was: <false>
     at org.apache.flink.table.planner.expressions.utils.ExpressionTestBase.addTestExpr(ExpressionTestBase.scala:324)
    

@fengjiajie for the non comparable types, I think we can ref to the TypeCheckUtils#isComparable:

    public static boolean isComparable(LogicalType type) {
        return !isRaw(type)
                && !isMap(type)
                && !isMultiset(type)
                && !isRow(type)
                && !isArray(type)
                && !isStructuredType(type);
    }

for the multiset type, one viable way is using the COLLECT[1] aggregate function to construct it.

  1. https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/functions/systemfunctions/#aggregate-functions

@fengjiajie fengjiajie changed the title [FLINK-33171][table planner] Table SQL support Not Equal for TimePoint type and TimeString [FLINK-33171][table planner] Unified the implementation of generateEquals and generateNotEquals Oct 16, 2023
@fengjiajie fengjiajie changed the title [FLINK-33171][table planner] Unified the implementation of generateEquals and generateNotEquals [FLINK-33171][table planner] Consistent implicit type coercion support for equal and non-equal comparisons for codegen Oct 16, 2023
@fengjiajie
Copy link
Contributor Author

  1. When constructing "non-comparable types," there is no problem with testSqlApi("NULL = f30", "NULL"), but testSqlApi("NULL = f30", "NULL") throws an exception:
    Caused by: org.apache.calcite.sql.validate.SqlValidatorException: Cannot apply '=' to arguments of type '<NULL> = <RECORDTYPE:PEEK_NO_EXPAND(VARCHAR(2147483647) F0, VARCHAR(2147483647) F1)>'. Supported form(s): '<COMPARABLE_TYPE> = <COMPARABLE_TYPE>'
  2. When constructing "non-comparable types," I haven't figured out how to cover the branch:
    } else if (isReference(right.resultType)) { because the preceding
    if (isReference(left.resultType)) { always takes precedence.
  3. I have written an SQL test for MULTISET in the MiscITCase. I'm not sure if this is appropriate.

@lincoln-lil
Copy link
Contributor

  1. When constructing "non-comparable types," there is no problem with testSqlApi("NULL = f30", "NULL"), but testSqlApi("NULL = f30", "NULL") throws an exception:
    Caused by: org.apache.calcite.sql.validate.SqlValidatorException: Cannot apply '=' to arguments of type '<NULL> = <RECORDTYPE:PEEK_NO_EXPAND(VARCHAR(2147483647) F0, VARCHAR(2147483647) F1)>'. Supported form(s): '<COMPARABLE_TYPE> = <COMPARABLE_TYPE>'
  2. When constructing "non-comparable types," I haven't figured out how to cover the branch:
    } else if (isReference(right.resultType)) { because the preceding
    if (isReference(left.resultType)) { always takes precedence.
  3. I have written an SQL test for MULTISET in the MiscITCase. I'm not sure if this is appropriate.

@fengjiajie IIUC, we should add an explicit type cast to null so that the field type can be deterministic, e.g., 'cast(null as bigint)' to construct a empty bigint column. For the multiset type, I'm fine with the current MiscITCase or the CalcItCase.
Btw, during the community review process, we generally recommend using the append commits, which makes it easier for the reviewer to track changes during the process and squash them when merging at the end.

cc @LadyForest If you have time to help check it again that would be great :)

@LadyForest
Copy link
Contributor

  1. When constructing "non-comparable types," there is no problem with testSqlApi("NULL = f30", "NULL"), but testSqlApi("NULL = f30", "NULL") throws an exception:
    Caused by: org.apache.calcite.sql.validate.SqlValidatorException: Cannot apply '=' to arguments of type '<NULL> = <RECORDTYPE:PEEK_NO_EXPAND(VARCHAR(2147483647) F0, VARCHAR(2147483647) F1)>'. Supported form(s): '<COMPARABLE_TYPE> = <COMPARABLE_TYPE>'
  2. When constructing "non-comparable types," I haven't figured out how to cover the branch:
    } else if (isReference(right.resultType)) { because the preceding
    if (isReference(left.resultType)) { always takes precedence.
  3. I have written an SQL test for MULTISET in the MiscITCase. I'm not sure if this is appropriate.

@fengjiajie IIUC, we should add an explicit type cast to null so that the field type can be deterministic, e.g., 'cast(null as bigint)' to construct a empty bigint column. For the multiset type, I'm fine with the current MiscITCase or the CalcItCase. Btw, during the community review process, we generally recommend using the append commits, which makes it easier for the reviewer to track changes during the process and squash them when merging at the end.

cc @LadyForest If you have time to help check it again that would be great :)

I'm sorry for not getting back to you sooner. I'll take a look as soon as possible.

@LadyForest
Copy link
Contributor

newRelDataType

@lincoln-lil Thank you for your review. Are the 5 datatypes you mentioned referring to DATE, TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITH_TIME_ZONE, and TIMESTAMP_WITH_LOCAL_TIME_ZONE? In the previous test code, there were f15, f21, and f22 corresponding to the first 3 types. I attempted to add the last two, but encountered an exception:

org.apache.flink.table.api.TableException: Type is not supported: TIMESTAMP_WITH_TIME_ZONE
	at org.apache.flink.table.planner.calcite.FlinkTypeFactory.newRelDataType$1(FlinkTypeFactory.scala:152)

So now I have only added f23, which corresponds to the TIMESTAMP_WITH_LOCAL_TIME_ZONE. If any further modifications are needed, please let me know. Thank you.

@fengjiajie The 5 types are correct, I checked the code in FlinkTypeFactory and also verified it in sql client, the TIMESTAMP_WITH_TIME_ZONE type is unsupported, this can be a separate issue to follow.

errors in sql client when create a column with TIMESTAMP WITH TIME ZONE:

Flink SQL> CREATE TABLE Bid (
>   bid STRING,
>   price BIGINT,
>   rowtime TIMESTAMP WITH TIME ZONE,
>   -- declare user_action_time as event time attribute and use 5 seconds delayed watermark strategy
>   WATERMARK FOR rowtime AS rowtime - INTERVAL '5' SECOND
> ) WITH ( 'connector' = 'datagen');
[ERROR] Could not execute SQL statement. Reason:
org.apache.flink.sql.parser.impl.ParseException: Encountered "TIME" at line 4, column 26.
Was expecting:
    "LOCAL" ...

Thanks for bringing up this issue. I think fixing this issue requires both support from the syntax and type factory.

Copy link
Contributor

@LadyForest LadyForest left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the update, looks good to me in general, and I just left some minor comments.

@fengjiajie
Copy link
Contributor Author

Thanks for the update, looks good to me in general, and I just left some minor comments.

Thank you for the suggestion. It has been modified.

Copy link
Contributor

@LadyForest LadyForest left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the update, looks good to me!

Copy link
Contributor

@lincoln-lil lincoln-lil left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@fengjiajie thanks for fixing this!

@LadyForest
Copy link
Contributor

@flinkbot run azure

Copy link
Contributor

@LadyForest LadyForest left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @fengjiajie, thanks for the update.

I noticed that the TPCDS test failed after the change. I have reconsidered and realized that the equal and non-equal comparisons of string types cannot directly invoke the "wrapExpressionIfNonEq" function.

@fengjiajie
Copy link
Contributor Author

@LadyForest Thank you for helping troubleshoot the issue.

@fengjiajie
Copy link
Contributor Author

@lincoln-lil @LadyForest When you have time, please take a look if any further modifications are needed, thanks.

@LadyForest
Copy link
Contributor

@lincoln-lil @LadyForest When you have time, please take a look if any further modifications are needed, thanks.

I'm sorry for the late reply; I'll take a look as soon as possible.

Copy link
Contributor

@LadyForest LadyForest left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the update, LGTM

@LadyForest LadyForest merged commit 04b98e6 into apache:master Nov 6, 2023
@LadyForest
Copy link
Contributor

Hi @fengjiajie, could you pick the fix to branch release-1.18 and release-1.17 as well?

@fengjiajie
Copy link
Contributor Author

Hi @fengjiajie, could you pick the fix to branch release-1.18 and release-1.17 as well?

Hi @LadyForest , thank you for moving this forward. cherry-picks to 1.17 and 1.18:

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
5 participants