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

Aljoscha Krettek edited comment on FLINK-14807 at 11/19/19 12:33 PM:
---------------------------------------------------------------------

(edited {{s/sources/sinks/}})

Currently there are no plans for the {{Executor}}/{{JobClient}} to handle 
result retrieval from sinks. It only provides job control/observation.

The {{DataSet}} API currently uses accumulators to ship results back for 
collect-style sinks. The {{JobClient}} will have functionality for getting the 
accumulators (also for running jobs) but I don't think that this is a 
production-ready solution.

The {{DataStream}} API doesn't have collect-style methods but there is 
{{DataStreamUtils}} which has {{collect()}} that ships data via manual 
low-level networking (i.e. sockets). I don't think this is a production-ready 
solution either because it won't work for YARN or K8s deployments.

In the future we should have retrieval of (retained) intermediate result 
partitions (i.e. what is being worked on under the "interactive programming" 
umbrella: 
[FLIP-36|https://cwiki.apache.org/confluence/display/FLINK/FLIP-36%3A+Support+Interactive+Programming+in+Flink]).
 I don't yet know how this will work for the different deployment 
options/network environments but it should be exposed through the {{JobClient}}.


was (Author: aljoscha):
Currently there are no plans for the {{Executor}}/{{JobClient}} to handle 
result retrieval from sources. It only provides job control/observation.

The {{DataSet}} API currently uses accumulators to ship results back for 
collect-style sinks. The {{JobClient}} will have functionality for getting the 
accumulators (also for running jobs) but I don't think that this is a 
production-ready solution.

The {{DataStream}} API doesn't have collect-style methods but there is 
{{DataStreamUtils}} which has {{collect()}} that ships data via manual 
low-level networking (i.e. sockets). I don't think this is a production-ready 
solution either because it won't work for YARN or K8s deployments.

In the future we should have retrieval of (retained) intermediate result 
partitions (i.e. what is being worked on under the "interactive programming" 
umbrella: 
[FLIP-36|https://cwiki.apache.org/confluence/display/FLINK/FLIP-36%3A+Support+Interactive+Programming+in+Flink]).
 I don't yet know how this will work for the different deployment 
options/network environments but it should be exposed through the {{JobClient}}.

> 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
>
> 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