[
https://issues.apache.org/jira/browse/BEAM-7758?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16889634#comment-16889634
]
Rahul Patwari commented on BEAM-7758:
-------------------------------------
"_All existing table provider can be used in this programmatic way.
SqlTransform is not limited to "SELECT * FROM PCOLLECTION" but can do "SELECT *
FROM TABLE_A", if you have a TABLE_A in table provider._"
- But none of the existing Table Providers expose any methods to add a Table
to the TableProvider. To do so, the TableProvider has to be extended like this:
[https://github.com/apache/beam/blob/c2f0d282337f3ae0196a7717712396a5a41fdde1/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTestTableProvider.java#L41]
"_You don't need to apply sql query to unbounded PCollection. Your unbounded
Pcollection are created within SqlTransform from a Table which is gotten from
table provider._"
- Do you mean to say that SlowChangingCacheTableProvider should be constructed
with two tables(one for unbounded PCollection, one for slowChangingCacheTable
PCollectionView) and perform a join query? If it is so, we have to take window
properties for the unbounded table to perform the join.
{code:java}
Pipeline pipeline = Pipeline.create(options);
SlowChangingCacheTableProvider provider =
SlowChangingCacheTableProvider.builder().withUnboundedTable(unboundedTable).withCacheTable(cacheTable).build();
SqlTransform query =
SqlTransform.query("join_query")
.withDefaultTableProvider("slowChangingCache", provider);
PCollection<Row> queryResult = pipeline.apply("Run SQL Query", query);
{code}
Doesn't this limit Joining [Slowly changing cache table] only with [unbounded
PCollection created from Table]? How to join [Slowly changing cache table]
with [unbounded PCollection not created from Table(SqlTranform applied on an
unbounded PCollection)]?
> Table returns PCollectionView in BeamSQL
> ----------------------------------------
>
> Key: BEAM-7758
> URL: https://issues.apache.org/jira/browse/BEAM-7758
> Project: Beam
> Issue Type: New Feature
> Components: dsl-sql
> Reporter: Rui Wang
> Assignee: Rahul Patwari
> Priority: Major
>
> We might be able to define a table with properties that says this table
> return a PCollectionView. By doing so we will have a trigger based
> PCollectionView available in SQL rel nodes.
> Relevant thread:
> https://lists.apache.org/thread.html/602121ee49886590ce4975f66aa0d270b4a5d64575337fca8bef1232@%3Cdev.beam.apache.org%3E
--
This message was sent by Atlassian JIRA
(v7.6.14#76016)