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

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

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

    https://github.com/apache/flink/pull/2275#discussion_r79154968
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/security/SecurityContext.java
 ---
    @@ -155,6 +157,58 @@ public static void install(SecurityConfiguration 
config) throws Exception {
                installedContext = new SecurityContext(loginUser);
        }
     
    +   /*
    +    * This is a temporary fix to support both Kafka and ZK client libraries
    +    * that are expecting the system variable to determine secure cluster
    +    */
    +   private static void populateJaasConfigSystemProperty(Configuration 
configuration) {
    +
    +           //hack since Kafka Login Handler explicitly looks for the 
property or else it throws an exception
    +           
//https://github.com/apache/kafka/blob/0.9.0/clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java#L289
    +           if(null == configuration) {
    +                   System.setProperty("java.security.auth.login.config", 
"");
    +                   return;
    +           }
    +
    +           String baseDir = 
configuration.getString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, null);
    +           if(baseDir == null) {
    +                   System.setProperty("java.security.auth.login.config", 
"");
    +                   return;
    +           }
    +
    +           File f = new File(baseDir);
    +           if(!f.exists() || !f.isDirectory()) {
    +                   LOG.error("Invalid flink base directory {} 
configuration provided", baseDir);
    +                   throw new IllegalConfigurationException("Invalid flink 
base directory configuration provided");
    +           }
    --- End diff --
    
    This check should really be somewhere else. I think you can use 
`ENV_FLINK_CONF_DIR` which is always set. The `FLINK_BASE_DIR_PATH_KEY` should 
be deprecated. It's not used anymore.


> Support for Kerberos Authentication with Keytab Credential
> ----------------------------------------------------------
>
>                 Key: FLINK-3929
>                 URL: https://issues.apache.org/jira/browse/FLINK-3929
>             Project: Flink
>          Issue Type: New Feature
>            Reporter: Eron Wright 
>            Assignee: Vijay Srinivasaraghavan
>              Labels: kerberos, 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._
> Add support for a keytab credential to be associated with the Flink cluster, 
> to facilitate:
> - Kerberos-authenticated data access for connectors
> - Kerberos-authenticated ZooKeeper access
> Support both the standalone and YARN deployment modes.
>  



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

Reply via email to