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

Rui Li commented on FLINK-13197:
--------------------------------

I hit some issue when trying to query views created in Hive. The view is 
created as:
{code}
create table src (key int, val string);
create view v as select key, count(*) from src group by key having count(*) > 1;
{code}
When querying this view from Flink, I hit the following exception:
{noformat}
org.apache.flink.table.api.TableException: Could not expand view. Types 
mismatch.
 Expected row type: RecordType(INTEGER key, BIGINT _c1)
 Expanded view type: RecordType(INTEGER key, BIGINT EXPR$1)


        at 
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.expandView(FlinkPlannerImpl.scala:191)
        at 
org.apache.flink.table.planner.catalog.SqlCatalogViewTable.convertToRel(SqlCatalogViewTable.java:56)
        at 
org.apache.flink.table.planner.plan.schema.ExpandingPreparingTable.expand(ExpandingPreparingTable.java:59)
        at 
org.apache.flink.table.planner.plan.schema.ExpandingPreparingTable.toRel(ExpandingPreparingTable.java:55)
{noformat}

The expected row type is what we get from {{HiveCatalog}}, and the expanded 
view type is what we get after compiling the expanded query of the view.
I wonder whether it's necessary to check the equality of these two, given that 
we can't anticipate what we'll get from the catalog. I think we can at least be 
more lenient here, e.g. it doesn't make sense to compare the column names, 
perhaps nullability either.

[~dwysakowicz] could you please share your thoughts? BTW I'm using the blink 
planner.

> support querying Hive's view in Flink
> -------------------------------------
>
>                 Key: FLINK-13197
>                 URL: https://issues.apache.org/jira/browse/FLINK-13197
>             Project: Flink
>          Issue Type: Improvement
>          Components: Connectors / Hive
>            Reporter: Bowen Li
>            Assignee: Rui Li
>            Priority: Major
>             Fix For: 1.10.0
>
>
> One goal of HiveCatalog and hive integration is to enable Flink-Hive 
> interoperability, that is Flink should understand existing Hive meta-objects, 
> and Hive meta-objects created thru Flink should be understood by Hive.
> Taking an example of a Hive view v1 in HiveCatalog and database hc.db. Unlike 
> an equivalent Flink view whose full path in expanded query should be 
> hc.db.v1, the Hive view's full path in the expanded query should be db.v1 
> such that Hive can understand it, no matter it's created by Hive or Flink.
> [~lirui] can you help to ensure that Flink can also query Hive's view in both 
> Flink planner and Blink planner?
> cc [~xuefuz]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to