[ 
https://issues.apache.org/jira/browse/FLINK-8107?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16387448#comment-16387448
 ] 

lincoln.lee commented on FLINK-8107:
------------------------------------

@Timo Walther

It seems not a bug here. The above case can successfully be validated after 
minor modification:
{code:java}
 val table = tEnv.sqlQuery("SELECT t.item FROM entity, UNNEST(entity.myarr) AS 
t(item)"){code}
or 
{code:java}
val table = tEnv.sqlQuery("SELECT t.item FROM UNNEST(select myarr from entity) 
AS t(item)"){code}
UNNEST accepts a collection value expression which can be produced by an input 
column reference or a subquery (or even a collection constructed by specific 
elements).

The 'entity' in the query "SELECT t.item.* FROM product, UNNEST(entity.myarr) 
AS t (item)" can not be recognized as a table.

 

> UNNEST causes cyclic type checking exception
> --------------------------------------------
>
>                 Key: FLINK-8107
>                 URL: https://issues.apache.org/jira/browse/FLINK-8107
>             Project: Flink
>          Issue Type: Bug
>          Components: Table API & SQL
>    Affects Versions: 1.4.0
>            Reporter: Timo Walther
>            Priority: Major
>
> The following query causes an assertion error:
> {code}
>   def main(args: Array[String]): Unit = {
>     val env = ExecutionEnvironment.getExecutionEnvironment
>     val tEnv = TableEnvironment.getTableEnvironment(env)
>     val input2 = env.fromElements(
>       WC("hello", 1, Array(1, 2, 3)),
>       WC("hello", 1, Array(1, 2, 3)),
>       WC("ciao", 1, Array(1, 2, 3))
>     )
>     tEnv.registerDataSet("entity", input2)
>     tEnv.registerDataSet("product", input2, 'product)
>     val table = tEnv.sqlQuery("SELECT t.item.* FROM product, 
> UNNEST(entity.myarr) AS t (item)")
>     table.toDataSet[Row].print()
>   }
>   case class WC(word: String, frequency: Long, myarr: Array[Int])
> {code}
> It leads to:
> {code}
> Exception in thread "main" java.lang.AssertionError: Cycle detected during 
> type-checking
>       at 
> org.apache.calcite.sql.validate.AbstractNamespace.validate(AbstractNamespace.java:93)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateNamespace(SqlValidatorImpl.java:945)
>       at 
> org.apache.calcite.sql.validate.AbstractNamespace.getRowType(AbstractNamespace.java:115)
>       at 
> org.apache.calcite.sql.validate.AbstractNamespace.getRowTypeSansSystemColumns(AbstractNamespace.java:122)
>       at 
> org.apache.calcite.sql.validate.AliasNamespace.validateImpl(AliasNamespace.java:71)
>       at 
> org.apache.calcite.sql.validate.AbstractNamespace.validate(AbstractNamespace.java:84)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateNamespace(SqlValidatorImpl.java:945)
>       at 
> org.apache.calcite.sql.validate.AbstractNamespace.getRowType(AbstractNamespace.java:115)
>       at 
> org.apache.calcite.sql.validate.AliasNamespace.getRowType(AliasNamespace.java:41)
>       at 
> org.apache.calcite.sql.validate.DelegatingScope.resolveInNamespace(DelegatingScope.java:101)
>       at org.apache.calcite.sql.validate.ListScope.resolve(ListScope.java:191)
>       at 
> org.apache.calcite.sql.validate.ListScope.findQualifyingTableNames(ListScope.java:156)
>       at 
> org.apache.calcite.sql.validate.DelegatingScope.fullyQualify(DelegatingScope.java:326)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateIdentifier(SqlValidatorImpl.java:2785)
>       at 
> org.apache.calcite.sql.SqlIdentifier.validateExpr(SqlIdentifier.java:324)
>       at org.apache.calcite.sql.SqlOperator.validateCall(SqlOperator.java:407)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateCall(SqlValidatorImpl.java:5084)
>       at 
> org.apache.calcite.sql.validate.UnnestNamespace.validateImpl(UnnestNamespace.java:52)
>       at 
> org.apache.calcite.sql.validate.AbstractNamespace.validate(AbstractNamespace.java:84)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateNamespace(SqlValidatorImpl.java:945)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateQuery(SqlValidatorImpl.java:926)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:2961)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:2946)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateJoin(SqlValidatorImpl.java:2998)
>       at 
> org.apache.flink.table.calcite.FlinkCalciteSqlValidator.validateJoin(FlinkCalciteSqlValidator.scala:67)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateFrom(SqlValidatorImpl.java:2955)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateSelect(SqlValidatorImpl.java:3206)
>       at 
> org.apache.calcite.sql.validate.SelectNamespace.validateImpl(SelectNamespace.java:60)
>       at 
> org.apache.calcite.sql.validate.AbstractNamespace.validate(AbstractNamespace.java:84)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateNamespace(SqlValidatorImpl.java:945)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateQuery(SqlValidatorImpl.java:926)
>       at org.apache.calcite.sql.SqlSelect.validate(SqlSelect.java:226)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validateScopedExpression(SqlValidatorImpl.java:901)
>       at 
> org.apache.calcite.sql.validate.SqlValidatorImpl.validate(SqlValidatorImpl.java:611)
>       at 
> org.apache.flink.table.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:89)
>       at 
> org.apache.flink.table.api.TableEnvironment.sqlQuery(TableEnvironment.scala:561)
>       at 
> org.apache.flink.table.examples.scala.WordCountSQL$.main(WordCountSQL.scala:57)
>       at 
> org.apache.flink.table.examples.scala.WordCountSQL.main(WordCountSQL.scala)
> {code}
> I think this is a Calcite bug but we have to check that first.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to