Details

    Description

      When trying to use Flink REST api to run a job that uses Flink table api with blink planner, we encountered an issue about `Incompatible types of expression and result type.` from org.apache.flink.table.planner.codegen.ExprCodeGenerator$$anonfun$generateResultExpression$1.apply(ExprCodeGenerator.scala:311). This issue only happens after the first request has been handled successfully.

       

      After digging, we found that there are two static caches used inside calcite's

      RelDataTypeFactoryImpl (

      https://github.com/apache/calcite/blob/d9a81b88ad561e7e4cedae93e805e0d7a53a7f1a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java#L352-L376

      ), which will remember the types they have seen. The `canonize` method is called from FlinkTypeFactory 

      https://github.com/apache/flink/blob/89f9dcd70dc3a1433055e17775b2b2a2c796ca94/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala#L292

       

      This causes problem for us because in our experience, we have seen GenericTypeInfo<T> containing different Class<T> instances in two different REST requests, and they do not equal, because https://github.com/apache/flink/blob/89f9dcd70dc3a1433055e17775b2b2a2c796ca94/flink-core/src/main/java/org/apache/flink/api/java/typeutils/GenericTypeInfo.java#L124 is using object equality. After `canonize`, the GenericTypeInfo for other REST requests would be changed to the GenericTypeInfo used for the first REST request, which is cached in RelDataTypeFactoryImpl. And this leads to the incompatible type error mentioned above.

       

      I want to propose using class name for equality comparison inside GenericTypeInfo, and change hashCode method accordingly.

       

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              tianshi Tianshi Zhu
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated: