LadyForest commented on code in PR #23752:
URL: https://github.com/apache/flink/pull/23752#discussion_r1443650062


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/QueryHintsResolver.java:
##########
@@ -84,6 +85,12 @@ protected RelNode visitBiRel(BiRel biRel) {
         Set<RelHint> existentKVHints = new HashSet<>();
 
         List<RelHint> oldHints = ((Hintable) biRel).getHints();
+        List<RelHint> oldQueryHints = FlinkHints.getAllQueryHints(oldHints);
+        // has no hints, return directly.
+        if (oldQueryHints.isEmpty()) {
+            return super.visitChildren(biRel);
+        }
+

Review Comment:
   If a hint is tagged to the rel node, in which condition will it neither be a 
join hint nor a state ttl hint?



##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalJoin.scala:
##########
@@ -116,13 +123,27 @@ class StreamPhysicalJoin(
   }
 
   override def translateToExecNode(): ExecNode[_] = {
+    val stateTtlFromHint = new util.HashMap[JInt, JLong]
+    getHints
+      .filter(hint => StateTtlHint.isStateTtlHint(hint.hintName))
+      .forEach {
+        hint =>
+          hint.kvOptions.forEach(
+            (input, ttl) =>
+              stateTtlFromHint
+                .put(
+                  if (input == FlinkHints.LEFT_INPUT) 0 else 1,

Review Comment:
   Considering the inclusion of processing logic for the agg node, I am 
currently exploring the possibility of pre-converting the Duration into 
milliseconds. This approach presents two notable advantages: firstly, it 
eliminates code duplication, and secondly, it ensures consistency in the time 
unit of the ttl hint value within the explain function.
   
   You can take the PoC for a reference
   
https://github.com/LadyForest/flink/blob/258d7e06fdd4e3970030f947a232a34bd4d3e7dc/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java#L661



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/stream/StateTtlHintTest.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.hints.stream;
+
+import org.apache.flink.table.api.ExplainDetail;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.utils.PlanKind;
+import org.apache.flink.table.planner.utils.StreamTableTestUtil;
+import org.apache.flink.table.planner.utils.TableTestBase;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import scala.Enumeration;
+
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Test for {@link org.apache.flink.table.planner.hint.StateTtlHint}. */
+class StateTtlHintTest extends TableTestBase {
+
+    protected StreamTableTestUtil util;
+
+    @BeforeEach
+    void before() {
+        util = streamTestUtil(TableConfig.getDefault());
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE T1 (\n"
+                                + "  a1 BIGINT,\n"
+                                + "  b1 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values'\n"
+                                + ")");
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE T2 (\n"
+                                + "  a2 BIGINT,\n"
+                                + "  b2 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values'\n"
+                                + ")");
+
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE T3 (\n"
+                                + "  a3 BIGINT,\n"
+                                + "  b3 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values'\n"
+                                + ")");
+
+        util.tableEnv().executeSql("CREATE View V4 as select a3 as a4, b3 as 
b4 from T3");
+
+        util.tableEnv()
+                .executeSql("create view V5 as select T1.* from T1 join T2 on 
T1.a1 = T2.a2");
+    }
+
+    @Test
+    void testSimpleJoinStateTtlHintWithEachSide() {
+        String sql =
+                "select /*+ STATE_TTL('T2' = '2d', 'T1' = '1d') */* from T1 
join T2 on T1.a1 = T2.a2";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintOnlyWithRightSide() {
+        String sql = "select /*+ STATE_TTL('T2' = '2d') */* from T1 join T2 on 
T1.a1 = T2.a2";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintWithContinuousJoin() {
+        String sql =
+                "select /*+ STATE_TTL('T2' = '2d', 'T3' = '3d', 'T1' = '1d') 
*/* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintWithMultiLevelJoin() {
+        String sql =
+                "select /*+ STATE_TTL('T2' = '2d', 'T3' = '3d', 'T1' = '1d') 
*/* from T1 "
+                        + "join (select T2.* from T2 join T3 on T2.b2 = T3.b3) 
TMP on T1.a1 = TMP.b2";
+        assertThatThrownBy(() -> verify(sql))
+                .isInstanceOf(ValidationException.class)
+                .hasMessageContaining(
+                        "The options of following hints cannot match the name 
of input tables or views: \n`%s` in `%s`",
+                        "T2, T3", "STATE_TTL");
+    }
+
+    @Test
+    void testJoinStateTtlHintWithOneUnknownTable() {
+        String sql =
+                "select /*+ STATE_TTL('T5' = '2d', 'T1' = '1d') */* from T1 
join T2 on T1.a1 = T2.a2";
+
+        assertThatThrownBy(() -> verify(sql))
+                .isInstanceOf(ValidationException.class)
+                .hasMessageContaining(
+                        "The options of following hints cannot match the name 
of input tables or views: \n`%s` in `%s`",
+                        "T5", "STATE_TTL");
+    }
+
+    @Test
+    void testJoinStateTtlHintWithTwoUnknownTables() {
+        String sql =
+                "select /*+ STATE_TTL('T5' = '2d', 'T6' = '1d') */* from T1 
join T2 on T1.a1 = T2.a2";
+
+        assertThatThrownBy(() -> verify(sql))
+                .isInstanceOf(ValidationException.class)
+                .hasMessageContaining(
+                        "The options of following hints cannot match the name 
of input tables or views: \n`%s` in `%s`",
+                        "T5, T6", "STATE_TTL");
+    }
+
+    @Test
+    void testJoinStateTtlHintWithView() {
+        String sql =
+                "select /*+ STATE_TTL('T1' = '2d', 'V4' = '1d') */* from T1 
join V4 on T1.a1 = V4.a4";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintWithUnknownView() {
+        String sql =
+                "select /*+ STATE_TTL('T1' = '2d', 'V8' = '1d') */* from T1 
join V4 on T1.a1 = V4.a4";
+        assertThatThrownBy(() -> verify(sql))
+                .isInstanceOf(ValidationException.class)
+                .hasMessageContaining(
+                        "The options of following hints cannot match the name 
of input tables or views: \n`%s` in `%s`",
+                        "V8", "STATE_TTL");
+    }
+
+    @Test
+    void testMultiJoinStateTtlHint() {
+        String sql =
+                "select /*+ STATE_TTL('T2' = '2d', 'T3' = '3d'), 
STATE_TTL('T1' = '1d', 'T2' = '8d') */* from T1, T2, T3 where T1.a1 = T2.a2 and 
T2.b2 = T3.b3";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintWithDuplicatedArgs() {
+        String sql =
+                "select /*+ STATE_TTL('T2' = '2d', 'T2' = '1d') */* from T1 
join T2 on T1.a1 = T2.a2";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintNotPropagateIntoView() {
+        String sql = "select /*+ STATE_TTL('T1' = '1d')*/T1.* from T1 join V5 
on T1.a1 = V5.a1";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintNotPropagateOutOfView() {
+        String sql =
+                "select T1.* from T1 join (select /*+ STATE_TTL('T1' = 
'2d')*/T1.* from T1 join T2 on T1.a1 = T2.a2) tmp on T1.a1 = tmp.a1";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintWithAgg() {
+        String sql =
+                "select /*+ STATE_TTL('T1' = '1d', 'T2' = '2d')*/T1.b1, 
sum(T1.a1) from T1 join T2 on T1.b1 = T2.b2 group by T1.b1";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintWithCommonJoinHint() {

Review Comment:
   Nit: What about `testJoinStateTtlHintWithCommonJoinHint` -> 
`testStateTtlHintWithJoinHint`?



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/stream/StateTtlHintTest.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.hints.stream;
+
+import org.apache.flink.table.api.ExplainDetail;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.utils.PlanKind;
+import org.apache.flink.table.planner.utils.StreamTableTestUtil;
+import org.apache.flink.table.planner.utils.TableTestBase;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import scala.Enumeration;
+
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Test for {@link org.apache.flink.table.planner.hint.StateTtlHint}. */
+class StateTtlHintTest extends TableTestBase {
+
+    protected StreamTableTestUtil util;
+
+    @BeforeEach
+    void before() {
+        util = streamTestUtil(TableConfig.getDefault());
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE T1 (\n"
+                                + "  a1 BIGINT,\n"
+                                + "  b1 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values'\n"
+                                + ")");
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE T2 (\n"
+                                + "  a2 BIGINT,\n"
+                                + "  b2 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values'\n"
+                                + ")");
+
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE T3 (\n"
+                                + "  a3 BIGINT,\n"
+                                + "  b3 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values'\n"
+                                + ")");
+
+        util.tableEnv().executeSql("CREATE View V4 as select a3 as a4, b3 as 
b4 from T3");
+
+        util.tableEnv()
+                .executeSql("create view V5 as select T1.* from T1 join T2 on 
T1.a1 = T2.a2");
+    }
+
+    @Test
+    void testSimpleJoinStateTtlHintWithEachSide() {
+        String sql =
+                "select /*+ STATE_TTL('T2' = '2d', 'T1' = '1d') */* from T1 
join T2 on T1.a1 = T2.a2";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintOnlyWithRightSide() {
+        String sql = "select /*+ STATE_TTL('T2' = '2d') */* from T1 join T2 on 
T1.a1 = T2.a2";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintWithContinuousJoin() {
+        String sql =
+                "select /*+ STATE_TTL('T2' = '2d', 'T3' = '3d', 'T1' = '1d') 
*/* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintWithMultiLevelJoin() {
+        String sql =
+                "select /*+ STATE_TTL('T2' = '2d', 'T3' = '3d', 'T1' = '1d') 
*/* from T1 "
+                        + "join (select T2.* from T2 join T3 on T2.b2 = T3.b3) 
TMP on T1.a1 = TMP.b2";
+        assertThatThrownBy(() -> verify(sql))
+                .isInstanceOf(ValidationException.class)
+                .hasMessageContaining(
+                        "The options of following hints cannot match the name 
of input tables or views: \n`%s` in `%s`",
+                        "T2, T3", "STATE_TTL");
+    }
+
+    @Test
+    void testJoinStateTtlHintWithOneUnknownTable() {
+        String sql =
+                "select /*+ STATE_TTL('T5' = '2d', 'T1' = '1d') */* from T1 
join T2 on T1.a1 = T2.a2";
+
+        assertThatThrownBy(() -> verify(sql))
+                .isInstanceOf(ValidationException.class)
+                .hasMessageContaining(
+                        "The options of following hints cannot match the name 
of input tables or views: \n`%s` in `%s`",
+                        "T5", "STATE_TTL");
+    }
+
+    @Test
+    void testJoinStateTtlHintWithTwoUnknownTables() {
+        String sql =
+                "select /*+ STATE_TTL('T5' = '2d', 'T6' = '1d') */* from T1 
join T2 on T1.a1 = T2.a2";
+
+        assertThatThrownBy(() -> verify(sql))
+                .isInstanceOf(ValidationException.class)
+                .hasMessageContaining(
+                        "The options of following hints cannot match the name 
of input tables or views: \n`%s` in `%s`",
+                        "T5, T6", "STATE_TTL");
+    }
+
+    @Test
+    void testJoinStateTtlHintWithView() {
+        String sql =
+                "select /*+ STATE_TTL('T1' = '2d', 'V4' = '1d') */* from T1 
join V4 on T1.a1 = V4.a4";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintWithUnknownView() {
+        String sql =
+                "select /*+ STATE_TTL('T1' = '2d', 'V8' = '1d') */* from T1 
join V4 on T1.a1 = V4.a4";
+        assertThatThrownBy(() -> verify(sql))
+                .isInstanceOf(ValidationException.class)
+                .hasMessageContaining(
+                        "The options of following hints cannot match the name 
of input tables or views: \n`%s` in `%s`",
+                        "V8", "STATE_TTL");
+    }
+
+    @Test
+    void testMultiJoinStateTtlHint() {

Review Comment:
   `testMultiJoinStateTtlHint` -> `testDuplicateJoinStateTtlHint`?
   
   Is it a better idea to throw exceptions here?  
   I think the `STATE_TTL` hint is different from the join hint on this point.
   Duplicate state ttl hint with contradictable hint values may result in
   unexpected results, while for join strategy, it's more about the performance.
   WDYT?
   ```
   stateTtlHints=[[[STATE_TTL options:{RIGHT=2d}][STATE_TTL options:{LEFT=1d, 
RIGHT=8d}]]]
   ```



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/stream/StateTtlHintTest.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.hints.stream;
+
+import org.apache.flink.table.api.ExplainDetail;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.utils.PlanKind;
+import org.apache.flink.table.planner.utils.StreamTableTestUtil;
+import org.apache.flink.table.planner.utils.TableTestBase;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import scala.Enumeration;
+
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Test for {@link org.apache.flink.table.planner.hint.StateTtlHint}. */
+class StateTtlHintTest extends TableTestBase {
+
+    protected StreamTableTestUtil util;
+
+    @BeforeEach
+    void before() {
+        util = streamTestUtil(TableConfig.getDefault());
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE T1 (\n"
+                                + "  a1 BIGINT,\n"
+                                + "  b1 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values'\n"
+                                + ")");
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE T2 (\n"
+                                + "  a2 BIGINT,\n"
+                                + "  b2 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values'\n"
+                                + ")");
+
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE T3 (\n"
+                                + "  a3 BIGINT,\n"
+                                + "  b3 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values'\n"
+                                + ")");
+
+        util.tableEnv().executeSql("CREATE View V4 as select a3 as a4, b3 as 
b4 from T3");
+
+        util.tableEnv()
+                .executeSql("create view V5 as select T1.* from T1 join T2 on 
T1.a1 = T2.a2");
+    }
+
+    @Test
+    void testSimpleJoinStateTtlHintWithEachSide() {
+        String sql =
+                "select /*+ STATE_TTL('T2' = '2d', 'T1' = '1d') */* from T1 
join T2 on T1.a1 = T2.a2";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintOnlyWithRightSide() {
+        String sql = "select /*+ STATE_TTL('T2' = '2d') */* from T1 join T2 on 
T1.a1 = T2.a2";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintWithContinuousJoin() {

Review Comment:
   Nit: `continousJoin` is confusing. I guess you want to say `cascadeJoin`



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/ConfigureOperatorLevelStateTtlJsonITCase.java:
##########
@@ -43,6 +44,57 @@ class ConfigureOperatorLevelStateTtlJsonITCase extends 
JsonPlanTestBase {
 
     @Test
     void testDifferentStateTtlForDifferentOneInputOperator() throws Exception {
+        innerTestDeduplicateAndGroupAggregate(

Review Comment:
   Please restore the comment annotated on the agg test to help others 
understand the logic
   ```java
   // with deduplicate state's TTL as 6s, record (+I,2,Jerry,2,99.9) will 
duplicate itself
           // 
+-------------------+--------------------------------------+------------------+
           // |        data       | diff(last_arriving, first_arriving) | 
within_time_range |
           // 
+-------------------+-------------------------------------+-------------------+
           // | 1,Tom,1,199.9     |                 4s                  |       
  Y         |
           // 
+-------------------+-------------------------------------+-------------------+
           // | 2,Jerry,2,99.9    |                 10s                  |      
  N         |
           // 
+-------------------+-------------------------------------+-------------------+
           // | 3,Tom,1,29.9      |                 0s                  |       
  Y         |
           // 
+-------------------+-------------------------------------+-------------------+
           // | 4,Olivia,1,100    |                 2s                  |       
  Y         |
           // 
+-------------------+-------------------------------------+-------------------+
           // | 5,Michael,3,599.9 |                 0s                  |       
  Y         |
           // 
+-------------------+-------------------------------------+-------------------+
           // | 6,Olivia,3,1000   |                 0s                  |       
  Y         |
           // 
+-------------------+-------------------------------------+-------------------+
   
           // with group-aggregate state's TTL as 9s, record 
(+I,2,Jerry,2,99.9) will be counted twice
   ```



##########
flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/stream/StateTtlHintTest.xml:
##########
@@ -0,0 +1,282 @@
+<?xml version="1.0" ?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to you under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<Root>
+  <TestCase name="testJoinStateTtlHintNotPropagateIntoView">
+    <Resource name="sql">
+      <![CDATA[select /*+ STATE_TTL('T1' = '1d')*/T1.* from T1 join V5 on 
T1.a1 = V5.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], 
stateTtlHints=[[[STATE_TTL inheritPath:[0] options:{T1=1d}]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V5]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[V5]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1, b1])
++- Join(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], 
leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], 
stateTtlHints=[[[STATE_TTL options:{LEFT=1d}]]])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- TableSourceScan(table=[[default_catalog, default_database, T1, 
project=[a1, b1], metadata=[]]], fields=[a1, b1])
+   +- Exchange(distribution=[hash[a1]])
+      +- Calc(select=[a1])
+         +- Join(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], 
leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
+            :- Exchange(distribution=[hash[a1]])
+            :  +- Calc(select=[a1])
+            :     +- TableSourceScan(table=[[default_catalog, 
default_database, T1, project=[a1, b1], metadata=[]]], fields=[a1, b1])
+            +- Exchange(distribution=[hash[a2]])
+               +- TableSourceScan(table=[[default_catalog, default_database, 
T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinStateTtlHintNotPropagateOutOfView">
+    <Resource name="sql">
+      <![CDATA[select T1.* from T1 join (select /*+ STATE_TTL('T1' = 
'2d')*/T1.* from T1 join T2 on T1.a1 = T2.a2) tmp on T1.a1 = tmp.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[tmp]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], 
stateTtlHints=[[[STATE_TTL inheritPath:[0] options:{T1=2d}]]], hints=[[[ALIAS 
inheritPath:[0] options:[tmp]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]], 
hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1, b1])
++- Join(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], 
leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- TableSourceScan(table=[[default_catalog, default_database, T1, 
project=[a1, b1], metadata=[]]], fields=[a1, b1])
+   +- Exchange(distribution=[hash[a1]])
+      +- Calc(select=[a1])
+         +- Join(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], 
leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], 
stateTtlHints=[[[STATE_TTL options:{LEFT=2d}]]])
+            :- Exchange(distribution=[hash[a1]])
+            :  +- Calc(select=[a1])
+            :     +- TableSourceScan(table=[[default_catalog, 
default_database, T1, project=[a1, b1], metadata=[]]], fields=[a1, b1])
+            +- Exchange(distribution=[hash[a2]])
+               +- TableSourceScan(table=[[default_catalog, default_database, 
T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinStateTtlHintOnlyWithRightSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ STATE_TTL('T2' = '2d') */* from T1 join T2 on T1.a1 
= T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], 
stateTtlHints=[[[STATE_TTL inheritPath:[0] options:{T2=2d}]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], 
hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Join(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], 
leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], 
stateTtlHints=[[[STATE_TTL options:{RIGHT=2d}]]])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], 
fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], 
fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinStateTtlHintWithAgg">
+    <Resource name="sql">
+      <![CDATA[select /*+ STATE_TTL('T1' = '1d', 'T2' = '2d')*/T1.b1, 
sum(T1.a1) from T1 join T2 on T1.b1 = T2.b2 group by T1.b1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
++- LogicalProject(b1=[$1], a1=[$0])
+   +- LogicalJoin(condition=[=($1, $3)], joinType=[inner], 
stateTtlHints=[[[STATE_TTL inheritPath:[0, 0] options:{T1=1d, T2=2d}]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], 
hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+GroupAggregate(groupBy=[b1], select=[b1, SUM(a1) AS EXPR$1])
++- Exchange(distribution=[hash[b1]])
+   +- Calc(select=[b1, a1])
+      +- Join(joinType=[InnerJoin], where=[=(b1, b2)], select=[a1, b1, b2], 
leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], 
stateTtlHints=[[[STATE_TTL options:{LEFT=1d, RIGHT=2d}]]])
+         :- Exchange(distribution=[hash[b1]])
+         :  +- TableSourceScan(table=[[default_catalog, default_database, 
T1]], fields=[a1, b1])
+         +- Exchange(distribution=[hash[b2]])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[b2], metadata=[]]], fields=[b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinStateTtlHintWithCommonJoinHint">
+    <Resource name="sql">
+      <![CDATA[select /*+ STATE_TTL('T1' = '1d', 'T2' = '2d'), BROADCAST(T1) 
*/T1.b1, sum(T1.a1) from T1 join T2 on T1.b1 = T2.b2 group by T1.b1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
++- LogicalProject(b1=[$1], a1=[$0])
+   +- LogicalJoin(condition=[=($1, $3)], joinType=[inner], 
joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1]]]], 
stateTtlHints=[[[STATE_TTL inheritPath:[0, 0] options:{T1=1d, T2=2d}]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], 
hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+GroupAggregate(groupBy=[b1], select=[b1, SUM(a1) AS EXPR$1])
++- Exchange(distribution=[hash[b1]])
+   +- Calc(select=[b1, a1])
+      +- Join(joinType=[InnerJoin], where=[=(b1, b2)], select=[a1, b1, b2], 
leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], 
joinHints=[[[BROADCAST options:[LEFT]]]], stateTtlHints=[[[STATE_TTL 
options:{LEFT=1d, RIGHT=2d}]]])
+         :- Exchange(distribution=[hash[b1]])
+         :  +- TableSourceScan(table=[[default_catalog, default_database, 
T1]], fields=[a1, b1])
+         +- Exchange(distribution=[hash[b2]])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[b2], metadata=[]]], fields=[b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinStateTtlHintWithContinuousJoin">
+    <Resource name="sql">
+      <![CDATA[select /*+ STATE_TTL('T2' = '2d', 'T3' = '3d', 'T1' = '1d') */* 
from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], 
stateTtlHints=[[[STATE_TTL inheritPath:[0, 0] options:{T1=1d, T2=2d, T3=3d}]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], 
stateTtlHints=[[[STATE_TTL inheritPath:[0, 0, 0] options:{T1=1d, T2=2d, 
T3=3d}]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], 
hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], 
hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Join(joinType=[InnerJoin], where=[=(b2, b3)], select=[a1, b1, a2, b2, a3, b3], 
leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], 
stateTtlHints=[[[STATE_TTL options:{RIGHT=3d}]]])
+:- Exchange(distribution=[hash[b2]])
+:  +- Join(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], 
leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], 
stateTtlHints=[[[STATE_TTL options:{LEFT=1d, RIGHT=2d}]]])
+:     :- Exchange(distribution=[hash[a1]])
+:     :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], 
fields=[a1, b1])
+:     +- Exchange(distribution=[hash[a2]])
+:        +- TableSourceScan(table=[[default_catalog, default_database, T2]], 
fields=[a2, b2])
++- Exchange(distribution=[hash[b3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], 
fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinStateTtlHintWithDuplicatedArgs">
+    <Resource name="sql">
+      <![CDATA[select /*+ STATE_TTL('T2' = '2d', 'T2' = '1d') */* from T1 join 
T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], 
stateTtlHints=[[[STATE_TTL inheritPath:[0] options:{T2=1d}]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], 
hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Join(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], 
leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], 
stateTtlHints=[[[STATE_TTL options:{RIGHT=1d}]]])

Review Comment:
   The same concern here.



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/stream/StateTtlHintTest.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.hints.stream;
+
+import org.apache.flink.table.api.ExplainDetail;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.utils.PlanKind;
+import org.apache.flink.table.planner.utils.StreamTableTestUtil;
+import org.apache.flink.table.planner.utils.TableTestBase;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import scala.Enumeration;
+
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Test for {@link org.apache.flink.table.planner.hint.StateTtlHint}. */
+class StateTtlHintTest extends TableTestBase {
+
+    protected StreamTableTestUtil util;
+
+    @BeforeEach
+    void before() {
+        util = streamTestUtil(TableConfig.getDefault());
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE T1 (\n"
+                                + "  a1 BIGINT,\n"
+                                + "  b1 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values'\n"
+                                + ")");
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE T2 (\n"
+                                + "  a2 BIGINT,\n"
+                                + "  b2 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values'\n"
+                                + ")");
+
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE T3 (\n"
+                                + "  a3 BIGINT,\n"
+                                + "  b3 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values'\n"
+                                + ")");
+
+        util.tableEnv().executeSql("CREATE View V4 as select a3 as a4, b3 as 
b4 from T3");
+
+        util.tableEnv()
+                .executeSql("create view V5 as select T1.* from T1 join T2 on 
T1.a1 = T2.a2");
+    }
+
+    @Test
+    void testSimpleJoinStateTtlHintWithEachSide() {
+        String sql =
+                "select /*+ STATE_TTL('T2' = '2d', 'T1' = '1d') */* from T1 
join T2 on T1.a1 = T2.a2";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintOnlyWithRightSide() {
+        String sql = "select /*+ STATE_TTL('T2' = '2d') */* from T1 join T2 on 
T1.a1 = T2.a2";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintWithContinuousJoin() {
+        String sql =
+                "select /*+ STATE_TTL('T2' = '2d', 'T3' = '3d', 'T1' = '1d') 
*/* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3";
+        verify(sql);
+    }
+
+    @Test
+    void testJoinStateTtlHintWithMultiLevelJoin() {

Review Comment:
   Nit: `testJoinStateTtlHintWithSubqueryContainsJoin`?



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

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

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


Reply via email to