From 9f726c5a15373201e7e0893df9915ac12f05cc5c Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Thu, 1 Feb 2024 21:31:03 +0100 Subject: [PATCH] FlinkRelMdPopulationSizeTest --- .../planner/expressions/TemporalTypesTest.scala | 2 +- .../metadata/FlinkRelMdDistinctRowCountTest.scala | 12 ++++++------ .../metadata/FlinkRelMdPopulationSizeTest.scala | 14 +++++++------- .../planner/plan/utils/FlinkRelMdUtilTest.scala | 7 ------- 4 files changed, 14 insertions(+), 21 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala index 6819dab298eb8d..b5a6827134953a 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala @@ -1441,7 +1441,7 @@ class TemporalTypesTest extends ExpressionTestBase { testExpectedSqlException( s"TIMESTAMPDIFF(SECOND, ${timestampLtz("1970-01-01 00:00:00.123")}, 'test_string_type')", "Cannot apply 'TIMESTAMPDIFF' to arguments of type" + - " 'TIMESTAMPDIFF(, , )'." + + " 'TIMESTAMPDIFF(, , )'." + " Supported form(s): 'TIMESTAMPDIFF(, , )'" ) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCountTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCountTest.scala index 9ad9a6de76e9df..568307603f8985 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCountTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCountTest.scala @@ -613,7 +613,7 @@ class FlinkRelMdDistinctRowCountTest extends FlinkRelMdHandlerTestBase { 2.0e7, mq.getDistinctRowCount(logicalInnerJoinNotOnUniqueKeys, ImmutableBitSet.of(0), null)) assertEquals( - 5.0569644545e8, + 5.0569644724678594e8, mq.getDistinctRowCount(logicalInnerJoinNotOnUniqueKeys, ImmutableBitSet.of(1), null), 1e-2) @@ -621,14 +621,14 @@ class FlinkRelMdDistinctRowCountTest extends FlinkRelMdHandlerTestBase { 2.0e7, mq.getDistinctRowCount(logicalLeftJoinOnUniqueKeys, ImmutableBitSet.of(0), null)) assertEquals( - 5.0569644545e8, + 5.0569644724678594e8, mq.getDistinctRowCount(logicalLeftJoinOnUniqueKeys, ImmutableBitSet.of(1), null), 1e-2) assertEquals( 2.0e7, mq.getDistinctRowCount(logicalLeftJoinNotOnUniqueKeys, ImmutableBitSet.of(0), null)) assertEquals( - 5.0569644545e8, + 5.0569644724678594e8, mq.getDistinctRowCount(logicalLeftJoinNotOnUniqueKeys, ImmutableBitSet.of(1), null), 1e-2) @@ -644,7 +644,7 @@ class FlinkRelMdDistinctRowCountTest extends FlinkRelMdHandlerTestBase { 2.0e7, mq.getDistinctRowCount(logicalRightJoinNotOnUniqueKeys, ImmutableBitSet.of(0), null)) assertEquals( - 5.0569644545e8, + 5.0569644724678594e8, mq.getDistinctRowCount(logicalRightJoinNotOnUniqueKeys, ImmutableBitSet.of(1), null), 1e-2) @@ -652,14 +652,14 @@ class FlinkRelMdDistinctRowCountTest extends FlinkRelMdHandlerTestBase { 2.0e7, mq.getDistinctRowCount(logicalFullJoinOnUniqueKeys, ImmutableBitSet.of(0), null)) assertEquals( - 5.0569644545e8, + 5.0569644724678594e8, mq.getDistinctRowCount(logicalFullJoinOnUniqueKeys, ImmutableBitSet.of(1), null), 1e-2) assertEquals( 2.0e7, mq.getDistinctRowCount(logicalFullJoinNotOnUniqueKeys, ImmutableBitSet.of(0), null)) assertEquals( - 5.0569644545e8, + 5.0569644724678594e8, mq.getDistinctRowCount(logicalFullJoinNotOnUniqueKeys, ImmutableBitSet.of(1), null), 1e-2) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdPopulationSizeTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdPopulationSizeTest.scala index 9c9925b8348e6f..8de367956090ef 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdPopulationSizeTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdPopulationSizeTest.scala @@ -334,15 +334,15 @@ class FlinkRelMdPopulationSizeTest extends FlinkRelMdHandlerTestBase { assertEquals(1.0, mq.getPopulationSize(logicalLeftJoinNotOnUniqueKeys, ImmutableBitSet.of())) assertEquals(2.0e7, mq.getPopulationSize(logicalLeftJoinNotOnUniqueKeys, ImmutableBitSet.of(0))) assertEquals( - 5.0569644545e8, + 5.0569644724678594e8, mq.getPopulationSize(logicalLeftJoinNotOnUniqueKeys, ImmutableBitSet.of(1)), 1e-2) assertEquals( - 8.0e8, + 7.999999791508117e8, mq.getPopulationSize(logicalLeftJoinNotOnUniqueKeys, ImmutableBitSet.of(1, 5)), 1e-2) assertEquals( - 7.9377199253e8, + 7.937727457918736e8, mq.getPopulationSize(logicalLeftJoinNotOnUniqueKeys, ImmutableBitSet.of(0, 6)), 1e-2) @@ -352,15 +352,15 @@ class FlinkRelMdPopulationSizeTest extends FlinkRelMdHandlerTestBase { mq.getPopulationSize(logicalRightJoinOnLHSUniqueKeys, ImmutableBitSet.of(0)), 1e-2) assertEquals( - 1.975207027e7, + 1.9752070389525224e7, mq.getPopulationSize(logicalRightJoinOnLHSUniqueKeys, ImmutableBitSet.of(1)), 1e-2) assertEquals( - 2.0e7, + 1.999999987845058e7, mq.getPopulationSize(logicalRightJoinOnLHSUniqueKeys, ImmutableBitSet.of(1, 5)), 1e-2) assertEquals( - 1.999606902e7, + 1.9996088147299763e7, mq.getPopulationSize(logicalRightJoinOnLHSUniqueKeys, ImmutableBitSet.of(0, 6)), 1e-2) @@ -368,7 +368,7 @@ class FlinkRelMdPopulationSizeTest extends FlinkRelMdHandlerTestBase { assertEquals(2.0e7, mq.getPopulationSize(logicalFullJoinWithoutEquiCond, ImmutableBitSet.of(0))) assertEquals(8.0e8, mq.getPopulationSize(logicalFullJoinWithoutEquiCond, ImmutableBitSet.of(1))) assertEquals( - 8.0e15, + 6.295509444597865e15, mq.getPopulationSize(logicalFullJoinWithoutEquiCond, ImmutableBitSet.of(1, 5))) assertEquals( 5.112e10, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/FlinkRelMdUtilTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/FlinkRelMdUtilTest.scala index f8fc56d48d84c1..bfd1a8098c776c 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/FlinkRelMdUtilTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/utils/FlinkRelMdUtilTest.scala @@ -31,18 +31,11 @@ class FlinkRelMdUtilTest { Assertions.assertEquals( BigDecimal(0.31606027941427883), BigDecimal.valueOf(FlinkRelMdUtil.numDistinctVals(0.5, 0.5))) - - // This case should be removed once CALCITE-4351 is fixed. - Assertions.assertEquals(Double.NaN, RelMdUtil.numDistinctVals(0.5, 0.5)) } @Test def testNumDistinctValsWithLargeInputs(): Unit = { Assertions.assertNotEquals(0.0, FlinkRelMdUtil.numDistinctVals(1e18, 1e10)) Assertions.assertEquals(9.99999993922529e9, FlinkRelMdUtil.numDistinctVals(1e18, 1e10), 1d) - // this test will fail once CALCITE-4351 is fixed - // in that case FlinkRelMdUtil#numDistinctVals should be removed - // see FLINK-19780 - Assertions.assertEquals(0.0, RelMdUtil.numDistinctVals(1e18, 1e10)) } }