wuchong commented on a change in pull request #11874:
URL: https://github.com/apache/flink/pull/11874#discussion_r414251140



##########
File path: 
flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala
##########
@@ -129,23 +130,6 @@ class DecimalTypeTest extends ExpressionTestBase {
       Long.MinValue.toString)
   }
 
-  @Ignore
-  @Test
-  def testDefaultDecimalCasting(): Unit = {

Review comment:
       Shoud we fix this problem? This test is moved from old planner 
`DecimalTypeTest#testDecimalCasting`. And in blink planner, it will throw 
exception when `cast` using `TypeInformation`.   
   
   ```
   java.lang.ClassCastException: 
org.apache.flink.table.types.logical.LegacyTypeInformationType cannot be cast 
to org.apache.flink.table.types.logical.DecimalType
   
        at 
org.apache.flink.table.planner.calcite.FlinkTypeFactory.newRelDataType$1(FlinkTypeFactory.scala:95)
        at 
org.apache.flink.table.planner.calcite.FlinkTypeFactory.createFieldTypeFromLogicalType(FlinkTypeFactory.scala:146)
        at 
org.apache.flink.table.planner.expressions.converter.CustomizedConvertRule.convertCast(CustomizedConvertRule.java:106)
        at 
org.apache.flink.table.planner.expressions.converter.CustomizedConvertRule.lambda$convert$0(CustomizedConvertRule.java:98)
        at java.util.Optional.map(Optional.java:215)
        at 
org.apache.flink.table.planner.expressions.converter.CustomizedConvertRule.convert(CustomizedConvertRule.java:98)
        at 
org.apache.flink.table.planner.expressions.converter.ExpressionConverter.visit(ExpressionConverter.java:97)
        at 
org.apache.flink.table.planner.expressions.converter.ExpressionConverter.visit(ExpressionConverter.java:72)
        at 
org.apache.flink.table.expressions.CallExpression.accept(CallExpression.java:122)
        at 
org.apache.flink.table.planner.plan.QueryOperationConverter.convertExprToRexNode(QueryOperationConverter.java:741)
        at 
org.apache.flink.table.planner.plan.QueryOperationConverter.access$800(QueryOperationConverter.java:132)
        at 
org.apache.flink.table.planner.plan.QueryOperationConverter$SingleRelVisitor.lambda$convertToRexNodes$6(QueryOperationConverter.java:547)
        at 
java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
        at 
java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1382)
        at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481)
        at 
java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471)
        at 
java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708)
        at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
        at 
java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:499)
        at 
org.apache.flink.table.planner.plan.QueryOperationConverter$SingleRelVisitor.convertToRexNodes(QueryOperationConverter.java:548)
        at 
org.apache.flink.table.planner.plan.QueryOperationConverter$SingleRelVisitor.visit(QueryOperationConverter.java:156)
        at 
org.apache.flink.table.planner.plan.QueryOperationConverter$SingleRelVisitor.visit(QueryOperationConverter.java:152)
        at 
org.apache.flink.table.operations.ProjectQueryOperation.accept(ProjectQueryOperation.java:75)
        at 
org.apache.flink.table.planner.plan.QueryOperationConverter.defaultMethod(QueryOperationConverter.java:149)
        at 
org.apache.flink.table.planner.plan.QueryOperationConverter.defaultMethod(QueryOperationConverter.java:131)
        at 
org.apache.flink.table.operations.utils.QueryOperationDefaultVisitor.visit(QueryOperationDefaultVisitor.java:47)
        at 
org.apache.flink.table.operations.ProjectQueryOperation.accept(ProjectQueryOperation.java:75)
        at 
org.apache.flink.table.planner.calcite.FlinkRelBuilder.queryOperation(FlinkRelBuilder.scala:176)
        at 
org.apache.flink.table.planner.expressions.utils.ExpressionTestBase.addTableApiTestExpr(ExpressionTestBase.scala:242)
        at 
org.apache.flink.table.planner.expressions.utils.ExpressionTestBase.addTableApiTestExpr(ExpressionTestBase.scala:236)
        at 
org.apache.flink.table.planner.expressions.utils.ExpressionTestBase.testTableApi(ExpressionTestBase.scala:226)
        at 
org.apache.flink.table.planner.expressions.DecimalTypeTest.testDecimalCasting(DecimalTypeTest.scala:136)
   ```

##########
File path: 
flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/NonDeterministicTests.scala
##########
@@ -23,71 +23,65 @@ import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.expressions.utils.ExpressionTestBase
 import org.apache.flink.types.Row
 
-import org.junit.{Ignore, Test}
+import org.junit.Test
 
 /**
   * Tests that can only be checked manually as they are non-deterministic.
   */
 class NonDeterministicTests extends ExpressionTestBase {
 
-  @Ignore

Review comment:
       This is on purpose. Tests in this file should be manually checked (e.g. 
whether the current timestamp is correct).

##########
File path: 
flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/stringexpr/SetOperatorsTest.scala
##########
@@ -22,24 +22,12 @@ import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.utils.TableTestBase
 
-import org.junit.{Ignore, Test}
+import org.junit.Test
 
 import java.sql.Timestamp
 
 class SetOperatorsTest extends TableTestBase {
 
-  @Ignore("Support in subQuery in ExpressionConverter")
-  @Test
-  def testInWithFilter(): Unit = {

Review comment:
       Why this is removed? I run this test and can get a plan. Is the plan 
incorrect? cc @godfreyhe 
   
   ```
   HashJoin(joinType=[LeftSemiJoin], where=[=(c, a1)], select=[a, b, c], 
build=[right], tryDistinctBuildRow=[true])
   :- Exchange(distribution=[hash[c]], shuffle_mode=[BATCH])
   :  +- TableSourceScan(table=[[default_catalog, default_database, A, source: 
[TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
   +- Exchange(distribution=[hash[a1]])
      +- LocalHashAggregate(groupBy=[a1], select=[a1])
         +- Calc(select=[a AS a1], where=[=(b, _UTF-16LE'two')])
            +- Reused(reference_id=[1])
   ```




----------------------------------------------------------------
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.

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


Reply via email to