[ 
https://issues.apache.org/jira/browse/FLINK-38030?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Hongshun Wang updated FLINK-38030:
----------------------------------
    Description: 
My catalog is implements org.apache.flink.table.catalog.Catalog rather than 
extends AbstractCatalog. When execute TableEnvironment.explainSql, exception 
will be thrown: 

 
{code:java}
java.lang.ClassCastException: class com.demo.FlinkCatalog cannot be cast to 
class org.apache.flink.table.catalog.AbstractCatalog 
(com.alibaba.fluss.flink.catalog.FlinkCatalog and 
org.apache.flink.table.catalog.AbstractCatalog are in unnamed module of loader 
'app')
        at 
org.apache.flink.table.planner.lineage.TableLineageDatasetImpl.<init>(TableLineageDatasetImpl.java:52)
       at 
org.apache.flink.table.planner.lineage.TableLineageUtils.createTableLineageDataset(TableLineageUtils.java:53)
        at 
org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecTableSourceScan.translateToPlanInternal(CommonExecTableSourceScan.java:205)
  at 
org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:168)
   at 
org.apache.flink.table.planner.delegation.StreamPlanner.$anonfun$translateToPlan$1(StreamPlanner.scala:85)
   at 
scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:286)   at 
scala.collection.Iterator.foreach(Iterator.scala:943)        at 
scala.collection.Iterator.foreach$(Iterator.scala:943)       at 
scala.collection.AbstractIterator.foreach(Iterator.scala:1431)       at 
scala.collection.IterableLike.foreach(IterableLike.scala:74) at 
scala.collection.IterableLike.foreach$(IterableLike.scala:73)        at 
scala.collection.AbstractIterable.foreach(Iterable.scala:56) at 
scala.collection.TraversableLike.map(TraversableLike.scala:286)      at 
scala.collection.TraversableLike.map$(TraversableLike.scala:279)     at 
scala.collection.AbstractTraversable.map(Traversable.scala:108)      at 
org.apache.flink.table.planner.delegation.StreamPlanner.translateToPlan(StreamPlanner.scala:84)
      at 
org.apache.flink.table.planner.delegation.PlannerBase.getExplainGraphs(PlannerBase.scala:627)
        at 
org.apache.flink.table.planner.delegation.StreamPlanner.explain(StreamPlanner.scala:103)
     at 
org.apache.flink.table.planner.delegation.StreamPlanner.explain(StreamPlanner.scala:51)
      at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.explainInternal(TableEnvironmentImpl.java:753)
      at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.explainSql(TableEnvironmentImpl.java:733)
   at 
org.apache.flink.table.api.TableEnvironment.explainSql(TableEnvironment.java:1155)
 {code}
 
AbstractCatalog is an internal class, thus we don't want to use it outside.

  was:
My catalog is implements org.apache.flink.table.catalog.Catalog rather than 
extends AbstractCatalog. When execute TableEnvironment.explainSql, exception 
will be thrown: 

 
{code:java}
java.lang.ClassCastException: class com.demo.FlinkCatalog cannot be cast to 
class org.apache.flink.table.catalog.AbstractCatalog 
(com.alibaba.fluss.flink.catalog.FlinkCatalog and 
org.apache.flink.table.catalog.AbstractCatalog are in unnamed module of loader 
'app')
        at 
org.apache.flink.table.planner.lineage.TableLineageDatasetImpl.<init>(TableLineageDatasetImpl.java:52)
       at 
org.apache.flink.table.planner.lineage.TableLineageUtils.createTableLineageDataset(TableLineageUtils.java:53)
        at 
org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecTableSourceScan.translateToPlanInternal(CommonExecTableSourceScan.java:205)
  at 
org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:168)
   at 
org.apache.flink.table.planner.delegation.StreamPlanner.$anonfun$translateToPlan$1(StreamPlanner.scala:85)
   at 
scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:286)   at 
scala.collection.Iterator.foreach(Iterator.scala:943)        at 
scala.collection.Iterator.foreach$(Iterator.scala:943)       at 
scala.collection.AbstractIterator.foreach(Iterator.scala:1431)       at 
scala.collection.IterableLike.foreach(IterableLike.scala:74) at 
scala.collection.IterableLike.foreach$(IterableLike.scala:73)        at 
scala.collection.AbstractIterable.foreach(Iterable.scala:56) at 
scala.collection.TraversableLike.map(TraversableLike.scala:286)      at 
scala.collection.TraversableLike.map$(TraversableLike.scala:279)     at 
scala.collection.AbstractTraversable.map(Traversable.scala:108)      at 
org.apache.flink.table.planner.delegation.StreamPlanner.translateToPlan(StreamPlanner.scala:84)
      at 
org.apache.flink.table.planner.delegation.PlannerBase.getExplainGraphs(PlannerBase.scala:627)
        at 
org.apache.flink.table.planner.delegation.StreamPlanner.explain(StreamPlanner.scala:103)
     at 
org.apache.flink.table.planner.delegation.StreamPlanner.explain(StreamPlanner.scala:51)
      at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.explainInternal(TableEnvironmentImpl.java:753)
      at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.explainSql(TableEnvironmentImpl.java:733)
   at 
org.apache.flink.table.api.TableEnvironment.explainSql(TableEnvironment.java:1155)
 {code}
 

 


> Table.explainSql throw exception: xxx cannot be cast to class 
> org.apache.flink.table.catalog.AbstractCatalog
> ------------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-38030
>                 URL: https://issues.apache.org/jira/browse/FLINK-38030
>             Project: Flink
>          Issue Type: Improvement
>          Components: Table SQL / Planner
>    Affects Versions: 2.0.0
>            Reporter: Hongshun Wang
>            Priority: Major
>             Fix For: 2.1
>
>
> My catalog is implements org.apache.flink.table.catalog.Catalog rather than 
> extends AbstractCatalog. When execute TableEnvironment.explainSql, exception 
> will be thrown: 
>  
> {code:java}
> java.lang.ClassCastException: class com.demo.FlinkCatalog cannot be cast to 
> class org.apache.flink.table.catalog.AbstractCatalog 
> (com.alibaba.fluss.flink.catalog.FlinkCatalog and 
> org.apache.flink.table.catalog.AbstractCatalog are in unnamed module of 
> loader 'app')
>       at 
> org.apache.flink.table.planner.lineage.TableLineageDatasetImpl.<init>(TableLineageDatasetImpl.java:52)
>        at 
> org.apache.flink.table.planner.lineage.TableLineageUtils.createTableLineageDataset(TableLineageUtils.java:53)
>         at 
> org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecTableSourceScan.translateToPlanInternal(CommonExecTableSourceScan.java:205)
>   at 
> org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase.translateToPlan(ExecNodeBase.java:168)
>    at 
> org.apache.flink.table.planner.delegation.StreamPlanner.$anonfun$translateToPlan$1(StreamPlanner.scala:85)
>    at 
> scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:286)   
> at scala.collection.Iterator.foreach(Iterator.scala:943)        at 
> scala.collection.Iterator.foreach$(Iterator.scala:943)       at 
> scala.collection.AbstractIterator.foreach(Iterator.scala:1431)       at 
> scala.collection.IterableLike.foreach(IterableLike.scala:74) at 
> scala.collection.IterableLike.foreach$(IterableLike.scala:73)        at 
> scala.collection.AbstractIterable.foreach(Iterable.scala:56) at 
> scala.collection.TraversableLike.map(TraversableLike.scala:286)      at 
> scala.collection.TraversableLike.map$(TraversableLike.scala:279)     at 
> scala.collection.AbstractTraversable.map(Traversable.scala:108)      at 
> org.apache.flink.table.planner.delegation.StreamPlanner.translateToPlan(StreamPlanner.scala:84)
>       at 
> org.apache.flink.table.planner.delegation.PlannerBase.getExplainGraphs(PlannerBase.scala:627)
>         at 
> org.apache.flink.table.planner.delegation.StreamPlanner.explain(StreamPlanner.scala:103)
>      at 
> org.apache.flink.table.planner.delegation.StreamPlanner.explain(StreamPlanner.scala:51)
>       at 
> org.apache.flink.table.api.internal.TableEnvironmentImpl.explainInternal(TableEnvironmentImpl.java:753)
>       at 
> org.apache.flink.table.api.internal.TableEnvironmentImpl.explainSql(TableEnvironmentImpl.java:733)
>    at 
> org.apache.flink.table.api.TableEnvironment.explainSql(TableEnvironment.java:1155)
>  {code}
>  
> AbstractCatalog is an internal class, thus we don't want to use it outside.



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

Reply via email to