[
https://issues.apache.org/jira/browse/FLINK-3929?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15503261#comment-15503261
]
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_r79355908
--- Diff:
flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/SecureTestEnvironment.java
---
@@ -0,0 +1,249 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.test.util;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.security.SecurityContext;
+import org.apache.hadoop.minikdc.MiniKdc;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.BufferedWriter;
+import java.io.PrintWriter;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * Helper {@link SecureTestEnvironment} to handle MiniKDC lifecycle.
+ * This class can be used to start/stop MiniKDC and create secure
configurations for MiniDFSCluster
+ * and MiniYarn
+ */
+
+public class SecureTestEnvironment {
+
+ protected static final Logger LOG =
LoggerFactory.getLogger(SecureTestEnvironment.class);
+
+ private static MiniKdc kdc;
+
+ private static String testKeytab = null;
+
+ private static String testPrincipal = null;
+
+ private static String testZkServerPrincipal = null;
+
+ private static String testZkClientPrincipal = null;
+
+ private static String testKafkaServerPrincipal = null;
+
+ private static String hadoopServicePrincipal = null;
+
+ private static File baseDirForSecureRun = null;
+
+ public static void prepare(TemporaryFolder tempFolder) {
+
+ try {
+ baseDirForSecureRun = tempFolder.newFolder();
+ LOG.info("Base Directory for Secure Environment: {}",
baseDirForSecureRun);
+
+ String hostName = "localhost";
+ Properties kdcConf = MiniKdc.createConf();
+ if(LOG.isDebugEnabled()) {
+ kdcConf.setProperty(MiniKdc.DEBUG, "true");
+ }
+ kdcConf.setProperty(MiniKdc.KDC_BIND_ADDRESS, hostName);
+ kdc = new MiniKdc(kdcConf, baseDirForSecureRun);
+ kdc.start();
+ LOG.info("Started Mini KDC");
+
+ File keytabFile = new File(baseDirForSecureRun,
"test-users.keytab");
+ testKeytab = keytabFile.getAbsolutePath();
+ testZkServerPrincipal = "zookeeper/127.0.0.1";
+ testZkClientPrincipal = "zk-client/127.0.0.1";
+ testKafkaServerPrincipal = "kafka/" + hostName;
+ hadoopServicePrincipal = "hadoop/" + hostName;
+ testPrincipal = "client/" + hostName;
+
+ kdc.createPrincipal(keytabFile, testPrincipal,
testZkServerPrincipal,
+ hadoopServicePrincipal,
+ testZkClientPrincipal,
+ testKafkaServerPrincipal);
+
+ testPrincipal = testPrincipal + "@" + kdc.getRealm();
+ testZkServerPrincipal = testZkServerPrincipal + "@" +
kdc.getRealm();
+ testZkClientPrincipal = testZkClientPrincipal + "@" +
kdc.getRealm();
+ testKafkaServerPrincipal = testKafkaServerPrincipal +
"@" + kdc.getRealm();
+ hadoopServicePrincipal = hadoopServicePrincipal + "@" +
kdc.getRealm();
+
+
LOG.info("-------------------------------------------------------------------");
+ LOG.info("Test Principal: {}", testPrincipal);
+ LOG.info("Test ZK Server Principal: {}",
testZkServerPrincipal);
+ LOG.info("Test ZK Client Principal: {}",
testZkClientPrincipal);
+ LOG.info("Test Kafka Server Principal: {}",
testKafkaServerPrincipal);
+ LOG.info("Test Hadoop Service Principal: {}",
hadoopServicePrincipal);
+ LOG.info("Test Keytab: {}", testKeytab);
+
LOG.info("-------------------------------------------------------------------");
+
+ //Security Context is established to allow non hadoop
applications that requires JAAS
+ //based SASL/Kerberos authentication to work. However,
for Hadoop specific applications
+ //the context can be reinitialized with Hadoop
configuration by calling
+ //ctx.setHadoopConfiguration() for the UGI
implementation to work properly.
+ //See Yarn test case module for reference
+ createJaasConfig(baseDirForSecureRun);
+ SecurityContext.SecurityConfiguration ctx = new
SecurityContext.SecurityConfiguration();
+ Configuration flinkConfig = new Configuration();
+
flinkConfig.setString(ConfigConstants.SECURITY_KEYTAB_KEY, testKeytab);
+
flinkConfig.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY, testPrincipal);
+
flinkConfig.setBoolean(ConfigConstants.ZOOKEEPER_SASL_DISABLE, false);
+
flinkConfig.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY,
baseDirForSecureRun.getAbsolutePath());
+ ctx.setFlinkConfiguration(flinkConfig);
+ TestingSecurityContext.install(ctx,
getClientSecurityConfigurationMap());
+
+ populateSystemEnvVariables();
+
+ } catch(Exception e) {
+ LOG.error("Exception occured while preparing secure
environment. Reason: {}", e);
+ throw new RuntimeException(e);
+ }
+
+ }
+
+ public static void cleanup() {
+
+ LOG.info("Cleaning up Secure Environment");
+
+ if( kdc != null) {
+ kdc.stop();
+ LOG.info("Stopped KDC server");
+ }
+
+ resetSystemEnvVariables();
+
+ testKeytab = null;
+ testPrincipal = null;
+ testZkServerPrincipal = null;
+ hadoopServicePrincipal = null;
+ baseDirForSecureRun = null;
+
+ }
+
+ private static void populateSystemEnvVariables() {
+
+ if(LOG.isDebugEnabled()) {
+ System.setProperty("FLINK_JAAS_DEBUG", "true");
+ System.setProperty("sun.security.krb5.debug", "true");
+ }
+
+ System.setProperty("java.security.krb5.conf",
kdc.getKrb5conf().getAbsolutePath());
+
+ System.setProperty("zookeeper.authProvider.1",
"org.apache.zookeeper.server.auth.SASLAuthenticationProvider");
+
System.setProperty("zookeeper.kerberos.removeHostFromPrincipal", "true");
+
System.setProperty("zookeeper.kerberos.removeRealmFromPrincipal", "true");
+ }
+
+ private static void resetSystemEnvVariables() {
+ System.clearProperty("java.security.krb5.conf");
+ System.clearProperty("FLINK_JAAS_DEBUG");
+ System.clearProperty("sun.security.krb5.debug");
+
+ System.clearProperty("zookeeper.authProvider.1");
+
System.clearProperty("zookeeper.kerberos.removeHostFromPrincipal");
+
System.clearProperty("zookeeper.kerberos.removeRealmFromPrincipal");
+ }
+
+ public static org.apache.flink.configuration.Configuration
populateFlinkSecureConfigurations(
+ @Nullable org.apache.flink.configuration.Configuration
flinkConf) {
+
+ org.apache.flink.configuration.Configuration conf;
+
+ if(flinkConf== null) {
+ conf = new
org.apache.flink.configuration.Configuration();
+ } else {
+ conf = flinkConf;
+ }
+
+ conf.setString(ConfigConstants.SECURITY_KEYTAB_KEY ,
testKeytab);
+ conf.setString(ConfigConstants.SECURITY_PRINCIPAL_KEY ,
testPrincipal);
+
+ return conf;
+ }
+
+ public static Map<String,
TestingSecurityContext.ClientSecurityConfiguration>
getClientSecurityConfigurationMap() {
+
+ Map<String, TestingSecurityContext.ClientSecurityConfiguration>
clientSecurityConfigurationMap = new HashMap<>();
+
+ if(testZkServerPrincipal != null ) {
+ TestingSecurityContext.ClientSecurityConfiguration
zkServer =
+ new
TestingSecurityContext.ClientSecurityConfiguration(testZkServerPrincipal,
testKeytab,
+ "Server", "zk-server");
+ clientSecurityConfigurationMap.put("Server",zkServer);
+ }
+
+ if(testZkClientPrincipal != null ) {
+ TestingSecurityContext.ClientSecurityConfiguration
zkClient =
+ new
TestingSecurityContext.ClientSecurityConfiguration(testZkClientPrincipal,
testKeytab,
+ "Client", "zk-client");
+ clientSecurityConfigurationMap.put("Client",zkClient);
+ }
+
+ if(testKafkaServerPrincipal != null ) {
+ TestingSecurityContext.ClientSecurityConfiguration
kafkaServer =
+ new
TestingSecurityContext.ClientSecurityConfiguration(testKafkaServerPrincipal,
testKeytab,
+ "KafkaServer",
"kafka-server");
+
clientSecurityConfigurationMap.put("KafkaServer",kafkaServer);
+ }
+
+ return clientSecurityConfigurationMap;
+ }
+
+ public static String getTestKeytab() {
+ return testKeytab;
+ }
+
+ public static String getHadoopServicePrincipal() {
+ return hadoopServicePrincipal;
+ }
+
+ /*
+ * Helper method to create a temporary JAAS configuration file to ger
around the Kafka and ZK SASL
+ * implementation lookup java.security.auth.login.config
+ */
+ private static void createJaasConfig(File baseDirForSecureRun) {
+
+ try(FileWriter fw = new FileWriter(new
File(baseDirForSecureRun,SecurityContext.JAAS_CONF_FILENAME), true);
+ BufferedWriter bw = new BufferedWriter(fw);
+ PrintWriter out = new PrintWriter(bw))
+ {
+ out.println("sample {");
+ out.println("useKeyTab=false");
+ out.println("useTicketCache=true;");
+ out.println("};");
+ } catch (IOException e) {
+ LOG.error("Exception occured while trying to create
JAAS config. Reason: {}", e.getMessage());
--- End diff --
Should be changed to `LOG.error("Exception occured while trying to create
JAAS config.", e);`
> 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)