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

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

Github user mxm commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2425#discussion_r84027778
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobClient.java ---
    @@ -725,7 +755,21 @@ else if (response != RETURN_OKAY) {
                        Object msg = 
JobManagerMessages.getRequestBlobManagerPort();
                        Future<Object> futureBlobPort = jobManager.ask(msg, 
askTimeout);
     
    +                   Object secureCookieMsg = 
JobManagerMessages.getRequestBlobManagerSecureCookie();
    +                   Future<Object> futureSecureCookie = 
jobManager.ask(secureCookieMsg, askTimeout);
    +
                        try {
    +                           String secureCookie = null;
    +
    +                           Object cookie = 
Await.result(futureSecureCookie, askTimeout);
    +                           if(cookie instanceof String) {
    +                                   secureCookie = (String) cookie;
    +                           }
    --- End diff --
    
    We are transferring the cookie here from the JobManager? That should never 
be the case. The client has to provide the cookie, otherwise a client must not 
be able to communicate with the JobManager.


> Implement Service-Level Authorization
> -------------------------------------
>
>                 Key: FLINK-3930
>                 URL: https://issues.apache.org/jira/browse/FLINK-3930
>             Project: Flink
>          Issue Type: New Feature
>          Components: Security
>            Reporter: Eron Wright 
>            Assignee: Vijay Srinivasaraghavan
>              Labels: security
>   Original Estimate: 672h
>  Remaining Estimate: 672h
>
> _This issue is part of a series of improvements detailed in the [Secure Data 
> Access|https://docs.google.com/document/d/1-GQB6uVOyoaXGwtqwqLV8BHDxWiMO2WnVzBoJ8oPaAs/edit?usp=sharing]
>  design doc._
> Service-level authorization is the initial authorization mechanism to ensure 
> clients (or servers) connecting to the Flink cluster are authorized to do so. 
>   The purpose is to prevent a cluster from being used by an unauthorized 
> user, whether to execute jobs, disrupt cluster functionality, or gain access 
> to secrets stored within the cluster.
> Implement service-level authorization as described in the design doc.
> - Introduce a shared secret cookie
> - Enable Akka security cookie
> - Implement data transfer authentication
> - Secure the web dashboard



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to