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

Stephan Ewen commented on FLINK-14807:
--------------------------------------

I appreciate the enthusiasm of diving into implementation details, but there 
are still quite bigger conceptual problems.

We are introducing a new communication plane here, that is nothing one should 
do lightly/quickly, this has among the biggest implications on setups and user 
experience of any changes. I think there are still so many conceptual questions:

*Do we need to expose additional ports on the master (JobManager)?*

It looks like this. That has big implications, meaning users need to open 
additional ports at the firewall.
On systems like K8s users need to create a second service, to access the "data 
streaming back" port from the outside. If you have multiple job managers 
running (standby pods), you need to handle the fact that the service may route 
to the wrong job manager, even route to a different job manager each time. 
There was a lot of work invested in the REST API, leader election, dynamic 
leader routing, merging blob server requests into the REST requests to make 
this a stable experience.

*Introducing an additional external protocol*

Often, HTTP is the only protocol that can be routed between cluster and outside 
clients. That was one reason why all client/JM communication is now HTTP/REST. 
An additional protocol for result fetching would have issues in many setups and 
counteract the current effort.

*What is the security story?*

How does this connection get secured? Encryption, authentication? This would be 
another "external communication" point, meaning the setup is not as simple as 
for internal communication.
 The REST endpoint supports having a proxy in front of it, because it is HTTP 
only. What would be the story of that additional endpoint?


==> Fetching results through the REST API would help solve these issues.


> Add Table#collect api for fetching data to client
> -------------------------------------------------
>
>                 Key: FLINK-14807
>                 URL: https://issues.apache.org/jira/browse/FLINK-14807
>             Project: Flink
>          Issue Type: New Feature
>          Components: Table SQL / API
>    Affects Versions: 1.9.1
>            Reporter: Jeff Zhang
>            Priority: Major
>              Labels: usability
>             Fix For: 1.11.0
>
>         Attachments: table-collect-draft.patch, table-collect.png
>
>
> Currently, it is very unconvinient for user to fetch data of flink job unless 
> specify sink expclitly and then fetch data from this sink via its api (e.g. 
> write to hdfs sink, then read data from hdfs). However, most of time user 
> just want to get the data and do whatever processing he want. So it is very 
> necessary for flink to provide api Table#collect for this purpose. 
>  
> Other apis such as Table#head, Table#print is also helpful.  
>  



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

Reply via email to