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

ASF GitHub Bot commented on FLINK-7040:
---------------------------------------

Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/4569
  
    @tillrohrmann I've addressed most of your comments.
    
    I have implemented the `HandlerResponse` as you suggested (casting option) 
for now, and include all query parameter as a list.
    
    ## Unaddressed comments:
    * Cleaner shutdown of netty components in server
    * Closing of netty connection (_somewhere_)
    * RestEndpointITCase: client/server shutdown in finally block
    
    ##
    
    ## Parameter Rework
    * Remove the `ParameterMapper` and related code
    * `MessageHeaders` now has a generic `MessageParameters` argument
    * `MessageParameters` is an abstract class that contains a collection of 
`MessageParameter`s that are suppported for the given request, and provides a 
utiltiy method for checking whether these parameters were resolved
    * `MessageParameter` is also an abstract class, that defines the key, 
accepts a value (i.e. resolve the parameter), defines whether the parameter is 
mandatory/optional and whether it is a query/path parameter
    
    ### Usage
    
    Let's say we want to have a request for getting the details of a job, with 
a path parameter for the job id :
    
    We first define a path parameter for the job id (or reuse an existing one!):
    
    ```
    public final class JobIDPathParameter extends MessagePathParameter {
        public JobIDPathParameter () {
                super("jobid", MessageParameterRequisiteness.MANDATORY);
        }
    
        public void resolve(JobID jobID) {
                super.resolve(jobID.toString());
        }
    }
    ```
    
    We then define the sum of all parameters for the request:
    
    ```
    public final class JobDetailsParameters extends MessageParameters {
        private final JobIDPathParameter jobID= new JobIDPathParameter ();
    
        @Override
        public Collection<MessageParameter> getParameters() {
                return Collections.singleton(jobID);
        }
    }
    ```
    
    And finally, we include it in the JobDetailsHeaders:
    ```
    public final class JobDetailsHeaders extends 
MessageHeaders<JobDetailsRequest, JobDetailsResponse, 
JobDetailsMessageParameters> {
    ...
    ```
    
    The usage for the client would then look like this:
    ```
    JobID jobID = ...
    JobDetailsParameters parameters = headers.getUnresolvedParameters();
    JobDetailsParameters.jobID.resolve(jobID);
    client.sendRequest(headers, parameters, request);
    ```



> Flip-6 client-cluster communication
> -----------------------------------
>
>                 Key: FLINK-7040
>                 URL: https://issues.apache.org/jira/browse/FLINK-7040
>             Project: Flink
>          Issue Type: New Feature
>          Components: Cluster Management, Mesos
>            Reporter: Till Rohrmann
>            Assignee: Chesnay Schepler
>            Priority: Critical
>              Labels: flip-6
>
> With the new Flip-6 architecture, the client will communicate with the 
> cluster in a RESTful manner.
> The cluster shall support the following REST calls:
> * List jobs (GET): Get list of all running jobs on the cluster
> * Submit job (POST): Submit a job to the cluster (only supported in session 
> mode)
> * Lookup job leader (GET): Gets the JM leader for the given job
> * Get job status (GET): Get the status of an executed job (and maybe the 
> JobExecutionResult)
> * Cancel job (PUT): Cancel the given job
> * Stop job (PUT): Stops the given job
> * Take savepoint (POST): Take savepoint for given job (How to return the 
> savepoint under which the savepoint was stored? Maybe always having to 
> specify a path)
> * Get KV state (GET): Gets the KV state for the given job and key (Queryable 
> state)
> * Poll/subscribe to notifications for job (GET, WebSocket): Polls new 
> notifications from the execution of the given job/Opens WebSocket to receive 
> notifications
> The first four REST calls will be served by the REST endpoint running in the 
> application master/cluster entrypoint. The other calls will be served by a 
> REST endpoint running along side to the JobManager.
> Detailed information about different implementations and their pros and cons 
> can be found in this document:
> https://docs.google.com/document/d/1eIX6FS9stwraRdSUgRSuLXC1sL7NAmxtuqIXe_jSi-k/edit?usp=sharing
> The implementation will most likely be Netty based.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to