Mark Nuttall created FLINK-36238:
------------------------------------

             Summary: State_ttl hints on joins don't work as documented
                 Key: FLINK-36238
                 URL: https://issues.apache.org/jira/browse/FLINK-36238
             Project: Flink
          Issue Type: Bug
          Components: Table SQL / API
    Affects Versions: 1.19.1, 1.20.0, 1.19.0
         Environment: Flink 1.19.0, 1.19.1, or 1.20, either via sql-client or 
Table API. 
            Reporter: Mark Nuttall


The docs for the STATE_TTL hint in 
[https://nightlies.apache.org/flink/flink-docs-release-1.19/docs/dev/table/sql/queries/hints/#state-ttl-hints]
 include queries of the form

{{}}

{{-- table alias as hint keySELECT /*+ STATE_TTL('o'='3d', 'l'='1d') */ * 
FROMorders o LEFT JOIN lineitem lON o.o_orderkey = l.l_orderkey;}}

 

However, these do not work in practice, either via the sql-client or Table API 
directly. See 
[https://github.com/mnuttall/flink-state-ttl-bug/|https://github.com/mnuttall/flink-state-ttl-bug/tree/main]
 for a simple reproduce. An error of the form below is generated:

 

{{Exception in thread "main" org.apache.flink.table.api.ValidationException: 
The options of following hints cannot match the name of input tables or views: 
}}
{{`partners, parts` in `STATE_TTL`}}
{{    at 
org.apache.flink.table.planner.plan.optimize.QueryHintsResolver.validateHints(QueryHintsResolver.java:336)}}
{{    at 
org.apache.flink.table.planner.plan.optimize.QueryHintsResolver.resolve(QueryHintsResolver.java:83)}}
{{    at 
org.apache.flink.table.planner.plan.optimize.CommonSubGraphBasedOptimizer.optimize(CommonSubGraphBasedOptimizer.scala:81)}}
{{    at 
org.apache.flink.table.planner.delegation.PlannerBase.optimize(PlannerBase.scala:320)}}
{{    at 
org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:178)}}
{{    at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.translate(TableEnvironmentImpl.java:1296)}}
{{    at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.executeInternal(TableEnvironmentImpl.java:1138)}}
{{    at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:735)}}
{{    at org.apache.flink.DataStreamJob.main(DataStreamJob.java:62)}}

 

This is for the query, 

{{select /*+ STATE_TTL('parts' = '2m', 'partners' = '5m') */ * FROM parts JOIN 
partners ON parts.region = partners.region}}
 
See also 
[https://apache-flink.slack.com/archives/C03G7LJTS2G/p1725455973543489] for a 
discussion of this problem in the Apache Flink Slack's 'troubleshooting' 
channel. 

 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to