HADOOP-15157. Zookeeper authentication related properties to support CredentialProviders. (Contributed by Gergo Repas)
This commit is contained in:
parent
880b9d24ff
commit
b202935353
@ -22,7 +22,6 @@
|
||||
import java.security.PrivilegedAction;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
@ -341,14 +340,7 @@ private void initZK() throws HadoopIllegalArgumentException, IOException,
|
||||
}
|
||||
|
||||
// Parse authentication from configuration.
|
||||
String zkAuthConf = conf.get(ZK_AUTH_KEY);
|
||||
zkAuthConf = ZKUtil.resolveConfIndirection(zkAuthConf);
|
||||
List<ZKAuthInfo> zkAuths;
|
||||
if (zkAuthConf != null) {
|
||||
zkAuths = ZKUtil.parseAuth(zkAuthConf);
|
||||
} else {
|
||||
zkAuths = Collections.emptyList();
|
||||
}
|
||||
List<ZKAuthInfo> zkAuths = SecurityUtil.getZKAuthInfos(conf, ZK_AUTH_KEY);
|
||||
|
||||
// Sanity check configuration.
|
||||
Preconditions.checkArgument(zkQuorum != null,
|
||||
|
@ -28,6 +28,7 @@
|
||||
import java.security.PrivilegedAction;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.ServiceLoader;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
@ -48,7 +49,7 @@
|
||||
import org.apache.hadoop.security.token.TokenInfo;
|
||||
import org.apache.hadoop.util.StopWatch;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import org.apache.hadoop.util.ZKUtil;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
//this will need to be replaced someday when there is a suitable replacement
|
||||
@ -719,4 +720,28 @@ public static void setAuthenticationMethod(
|
||||
public static boolean isPrivilegedPort(final int port) {
|
||||
return port < 1024;
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility method to fetch ZK auth info from the configuration.
|
||||
* @throws java.io.IOException if the Zookeeper ACLs configuration file
|
||||
* cannot be read
|
||||
* @throws ZKUtil.BadAuthFormatException if the auth format is invalid
|
||||
*/
|
||||
public static List<ZKUtil.ZKAuthInfo> getZKAuthInfos(Configuration conf,
|
||||
String configKey) throws IOException {
|
||||
char[] zkAuthChars = conf.getPassword(configKey);
|
||||
String zkAuthConf =
|
||||
zkAuthChars != null ? String.valueOf(zkAuthChars) : null;
|
||||
try {
|
||||
zkAuthConf = ZKUtil.resolveConfIndirection(zkAuthConf);
|
||||
if (zkAuthConf != null) {
|
||||
return ZKUtil.parseAuth(zkAuthConf);
|
||||
} else {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
} catch (IOException | ZKUtil.BadAuthFormatException e) {
|
||||
LOG.error("Couldn't read Auth based on {}", configKey);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -20,7 +20,6 @@
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.curator.framework.AuthInfo;
|
||||
@ -32,6 +31,7 @@
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
import org.apache.hadoop.security.SecurityUtil;
|
||||
import org.apache.hadoop.util.ZKUtil;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
import org.apache.zookeeper.data.ACL;
|
||||
@ -100,22 +100,11 @@ public static List<ACL> getZKAcls(Configuration conf) throws IOException {
|
||||
* Utility method to fetch ZK auth info from the configuration.
|
||||
* @throws java.io.IOException if the Zookeeper ACLs configuration file
|
||||
* cannot be read
|
||||
* @throws ZKUtil.BadAuthFormatException if the auth format is invalid
|
||||
*/
|
||||
public static List<ZKUtil.ZKAuthInfo> getZKAuths(Configuration conf)
|
||||
throws IOException {
|
||||
String zkAuthConf = conf.get(CommonConfigurationKeys.ZK_AUTH);
|
||||
try {
|
||||
zkAuthConf = ZKUtil.resolveConfIndirection(zkAuthConf);
|
||||
if (zkAuthConf != null) {
|
||||
return ZKUtil.parseAuth(zkAuthConf);
|
||||
} else {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
} catch (IOException | ZKUtil.BadAuthFormatException e) {
|
||||
LOG.error("Couldn't read Auth based on {}",
|
||||
CommonConfigurationKeys.ZK_AUTH);
|
||||
throw e;
|
||||
}
|
||||
return SecurityUtil.getZKAuthInfos(conf, CommonConfigurationKeys.ZK_AUTH);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -98,8 +98,8 @@ In summary, first, provision the credentials into a provider then configure the
|
||||
|:---- |:---- |:---|
|
||||
|LDAPGroupsMapping |LDAPGroupsMapping is used to look up the groups for a given user in LDAP. The CredentialProvider API is used to protect the LDAP bind password and those needed for SSL.|[LDAP Groups Mapping](GroupsMapping.html#LDAP_Groups_Mapping)|
|
||||
|SSL Passwords |FileBasedKeyStoresFactory leverages the credential provider API in order to resolve the SSL related passwords.|TODO|
|
||||
|HDFS |DFSUtil leverages Configuration.getPassword method to use the credential provider API and/or fallback to the clear text value stored in ssl-server.xml.|TODO|
|
||||
|YARN |WebAppUtils uptakes the use of the credential provider API through the new method on Configuration called getPassword. This provides an alternative to storing the passwords in clear text within the ssl-server.xml file while maintaining backward compatibility.|TODO|
|
||||
|HDFS |DFSUtil leverages Configuration.getPassword method to use the credential provider API and/or fallback to the clear text value stored in ssl-server.xml. Zookeeper based federation state store and failover controller use Configuration.getPassword to get the Zookeeper authentication info, with fallback provided to clear text auth info.|TODO|
|
||||
|YARN |WebAppUtils uptakes the use of the credential provider API through the new method on Configuration called getPassword. This provides an alternative to storing the passwords in clear text within the ssl-server.xml file while maintaining backward compatibility. Zookeeper based resource manager state store uses Configuration.getPassword to get the Zookeeper authentication info, with fallback provided to clear text auth info.|TODO|
|
||||
|KMS |Uses HttpServer2.loadSSLConfiguration that leverages Configuration.getPassword to read SSL related credentials. They may be resolved through Credential Provider and/or from the clear text in the config when allowed.|[KMS](../../hadoop-kms/index.html)|
|
||||
|HttpFS |Uses HttpServer2.loadSSLConfiguration that leverages Configuration.getPassword to read SSL related credentials. They may be resolved through Credential Provider and/or from the clear text in the config when allowed.|[HttpFS Server Setup](../../hadoop-hdfs-httpfs/ServerSetup.html)|
|
||||
|AWS <br/> S3/S3A |Uses Configuration.getPassword to get the S3 credentials. They may be resolved through the credential provider API or from the config for backward compatibility.|[AWS S3/S3A Usage](../../hadoop-aws/tools/hadoop-aws/index.html)|
|
||||
|
@ -21,10 +21,13 @@
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.URI;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.List;
|
||||
|
||||
import javax.security.auth.kerberos.KerberosPrincipal;
|
||||
|
||||
@ -32,14 +35,23 @@
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.security.alias.CredentialProvider;
|
||||
import org.apache.hadoop.security.alias.CredentialProviderFactory;
|
||||
import org.apache.hadoop.security.alias.LocalJavaKeyStoreProvider;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.util.ZKUtil.ZKAuthInfo;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import com.google.common.io.Files;
|
||||
|
||||
public class TestSecurityUtil {
|
||||
|
||||
private static final String ZK_AUTH_VALUE = "a_scheme:a_password";
|
||||
|
||||
@BeforeClass
|
||||
public static void unsetKerberosRealm() {
|
||||
// prevent failures if kinit-ed or on os x with no realm
|
||||
@ -404,4 +416,73 @@ public void testSetAuthenticationMethod() {
|
||||
SecurityUtil.setAuthenticationMethod(KERBEROS, conf);
|
||||
assertEquals("kerberos", conf.get(HADOOP_SECURITY_AUTHENTICATION));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAuthPlainPasswordProperty() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(CommonConfigurationKeys.ZK_AUTH, ZK_AUTH_VALUE);
|
||||
List<ZKAuthInfo> zkAuths = SecurityUtil.getZKAuthInfos(conf,
|
||||
CommonConfigurationKeys.ZK_AUTH);
|
||||
assertEquals(1, zkAuths.size());
|
||||
ZKAuthInfo zkAuthInfo = zkAuths.get(0);
|
||||
assertEquals("a_scheme", zkAuthInfo.getScheme());
|
||||
assertArrayEquals("a_password".getBytes(), zkAuthInfo.getAuth());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAuthPlainTextFile() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
File passwordTxtFile = File.createTempFile(
|
||||
getClass().getSimpleName() + ".testAuthAtPathNotation-", ".txt");
|
||||
Files.write(ZK_AUTH_VALUE, passwordTxtFile, StandardCharsets.UTF_8);
|
||||
try {
|
||||
conf.set(CommonConfigurationKeys.ZK_AUTH,
|
||||
"@" + passwordTxtFile.getAbsolutePath());
|
||||
List<ZKAuthInfo> zkAuths = SecurityUtil.getZKAuthInfos(conf,
|
||||
CommonConfigurationKeys.ZK_AUTH);
|
||||
assertEquals(1, zkAuths.size());
|
||||
ZKAuthInfo zkAuthInfo = zkAuths.get(0);
|
||||
assertEquals("a_scheme", zkAuthInfo.getScheme());
|
||||
assertArrayEquals("a_password".getBytes(), zkAuthInfo.getAuth());
|
||||
} finally {
|
||||
boolean deleted = passwordTxtFile.delete();
|
||||
assertTrue(deleted);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAuthLocalJceks() throws Exception {
|
||||
File localJceksFile = File.createTempFile(
|
||||
getClass().getSimpleName() +".testAuthLocalJceks-", ".localjceks");
|
||||
populateLocalJceksTestFile(localJceksFile.getAbsolutePath());
|
||||
try {
|
||||
String localJceksUri = "localjceks://file/" +
|
||||
localJceksFile.getAbsolutePath();
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
|
||||
localJceksUri);
|
||||
List<ZKAuthInfo> zkAuths = SecurityUtil.getZKAuthInfos(conf,
|
||||
CommonConfigurationKeys.ZK_AUTH);
|
||||
assertEquals(1, zkAuths.size());
|
||||
ZKAuthInfo zkAuthInfo = zkAuths.get(0);
|
||||
assertEquals("a_scheme", zkAuthInfo.getScheme());
|
||||
assertArrayEquals("a_password".getBytes(), zkAuthInfo.getAuth());
|
||||
} finally {
|
||||
boolean deleted = localJceksFile.delete();
|
||||
assertTrue(deleted);
|
||||
}
|
||||
}
|
||||
|
||||
private void populateLocalJceksTestFile(String path) throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH,
|
||||
"localjceks://file/" + path);
|
||||
CredentialProvider provider =
|
||||
CredentialProviderFactory.getProviders(conf).get(0);
|
||||
assertEquals(LocalJavaKeyStoreProvider.class.getName(),
|
||||
provider.getClass().getName());
|
||||
provider.createCredentialEntry(CommonConfigurationKeys.ZK_AUTH,
|
||||
ZK_AUTH_VALUE.toCharArray());
|
||||
provider.flush();
|
||||
}
|
||||
}
|
||||
|
@ -487,7 +487,7 @@ In order to secure the information in ZooKeeper, first add the following to your
|
||||
<value>@/path/to/zk-acl.txt</value>
|
||||
</property>
|
||||
|
||||
Please note the '@' character in these values -- this specifies that the configurations are not inline, but rather point to a file on disk.
|
||||
Please note the '@' character in these values -- this specifies that the configurations are not inline, but rather point to a file on disk. The authentication info may also be read via a CredentialProvider (pls see the CredentialProviderAPI Guide in the hadoop-common project).
|
||||
|
||||
The first configured file specifies a list of ZooKeeper authentications, in the same format as used by the ZK CLI. For example, you may specify something like:
|
||||
|
||||
|
@ -535,7 +535,7 @@ In order to secure the information in ZooKeeper, first add the following to your
|
||||
<value>@/path/to/zk-acl.txt</value>
|
||||
</property>
|
||||
|
||||
Please note the '@' character in these values -- this specifies that the configurations are not inline, but rather point to a file on disk.
|
||||
Please note the '@' character in these values -- this specifies that the configurations are not inline, but rather point to a file on disk. The authentication info may also be read via a CredentialProvider (pls see the CredentialProviderAPI Guide in the hadoop-common project).
|
||||
|
||||
The first configured file specifies a list of ZooKeeper authentications, in the same format as used by the ZK CLI. For example, you may specify something like:
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user