Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-16113

ExpressionReducer shouldn't escape the reduced string value

    XMLWordPrintableJSON

Details

    Description

      ExpressionReducer shouldn't escape the reduced string value, the escaping should only happen in code generation, otherwise the output result is inccorect.

      The problem is this line I guess: https://github.com/apache/flink/blob/master/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala#L142

      Here is a simple example to reproduce the problem:

        val smallTupleData3: Seq[(Int, Long, String)] = {
          val data = new mutable.MutableList[(Int, Long, String)]
          data.+=((1, 1L, "你好"))
          data.+=((2, 2L, "你好"))
          data.+=((3, 2L, "你好世界"))
          data
        }
      
        @Test
        def test(): Unit = {
          val t = env.fromCollection(smallTupleData3)
            .toTable(tEnv, 'a, 'b, 'c)
          tEnv.createTemporaryView("MyTable", t)
          val sqlQuery = s"select * from MyTable where c = '你好'"
      
          val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row]
          val sink = new TestingAppendSink
          result.addSink(sink)
          env.execute()
          println(sink.getAppendResults.mkString("\n"))
        }
      

      The output:

      1,1,\u4F60\u597D
      2,2,\u4F60\u597D
      

      This is also mentioned in user mailing list: http://apache-flink.147419.n8.nabble.com/ParquetTableSource-blink-table-planner-tp1696p1720.html

      Attachments

        Issue Links

          Activity

            People

              jark Jark Wu
              jark Jark Wu
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 40m
                  40m