|
@@ -18,65 +18,25 @@
|
|
|
|
|
|
package org.apache.hadoop.minikdc;
|
|
package org.apache.hadoop.minikdc;
|
|
import org.apache.commons.io.Charsets;
|
|
import org.apache.commons.io.Charsets;
|
|
-import org.apache.commons.io.FileUtils;
|
|
|
|
-import org.apache.commons.io.IOUtils;
|
|
|
|
-import org.apache.commons.lang.text.StrSubstitutor;
|
|
|
|
-import org.apache.directory.api.ldap.model.schema.SchemaManager;
|
|
|
|
-import org.apache.directory.api.ldap.schemaextractor.SchemaLdifExtractor;
|
|
|
|
-import org.apache.directory.api.ldap.schemaextractor.impl.DefaultSchemaLdifExtractor;
|
|
|
|
-import org.apache.directory.api.ldap.schemaloader.LdifSchemaLoader;
|
|
|
|
-import org.apache.directory.api.ldap.schemamanager.impl.DefaultSchemaManager;
|
|
|
|
-import org.apache.directory.server.constants.ServerDNConstants;
|
|
|
|
-import org.apache.directory.server.core.DefaultDirectoryService;
|
|
|
|
-import org.apache.directory.server.core.api.CacheService;
|
|
|
|
-import org.apache.directory.server.core.api.DirectoryService;
|
|
|
|
-import org.apache.directory.server.core.api.InstanceLayout;
|
|
|
|
-import org.apache.directory.server.core.api.schema.SchemaPartition;
|
|
|
|
-import org.apache.directory.server.core.kerberos.KeyDerivationInterceptor;
|
|
|
|
-import org.apache.directory.server.core.partition.impl.btree.jdbm.JdbmIndex;
|
|
|
|
-import org.apache.directory.server.core.partition.impl.btree.jdbm.JdbmPartition;
|
|
|
|
-import org.apache.directory.server.core.partition.ldif.LdifPartition;
|
|
|
|
-import org.apache.directory.server.kerberos.KerberosConfig;
|
|
|
|
-import org.apache.directory.server.kerberos.kdc.KdcServer;
|
|
|
|
-import org.apache.directory.server.kerberos.shared.crypto.encryption.KerberosKeyFactory;
|
|
|
|
-import org.apache.directory.server.kerberos.shared.keytab.Keytab;
|
|
|
|
-import org.apache.directory.server.kerberos.shared.keytab.KeytabEntry;
|
|
|
|
-import org.apache.directory.server.protocol.shared.transport.AbstractTransport;
|
|
|
|
-import org.apache.directory.server.protocol.shared.transport.TcpTransport;
|
|
|
|
-import org.apache.directory.server.protocol.shared.transport.UdpTransport;
|
|
|
|
-import org.apache.directory.server.xdbm.Index;
|
|
|
|
-import org.apache.directory.shared.kerberos.KerberosTime;
|
|
|
|
-import org.apache.directory.shared.kerberos.codec.types.EncryptionType;
|
|
|
|
-import org.apache.directory.shared.kerberos.components.EncryptionKey;
|
|
|
|
-import org.apache.directory.api.ldap.model.entry.DefaultEntry;
|
|
|
|
-import org.apache.directory.api.ldap.model.entry.Entry;
|
|
|
|
-import org.apache.directory.api.ldap.model.ldif.LdifEntry;
|
|
|
|
-import org.apache.directory.api.ldap.model.ldif.LdifReader;
|
|
|
|
-import org.apache.directory.api.ldap.model.name.Dn;
|
|
|
|
-import org.apache.directory.api.ldap.model.schema.registries.SchemaLoader;
|
|
|
|
|
|
+import org.apache.kerby.kerberos.kerb.KrbException;
|
|
|
|
+import org.apache.kerby.kerberos.kerb.server.KdcConfigKey;
|
|
|
|
+import org.apache.kerby.kerberos.kerb.server.SimpleKdcServer;
|
|
|
|
+import org.apache.kerby.util.IOUtil;
|
|
|
|
+import org.apache.kerby.util.NetworkUtil;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
-import java.io.BufferedReader;
|
|
|
|
import java.io.File;
|
|
import java.io.File;
|
|
import java.io.FileInputStream;
|
|
import java.io.FileInputStream;
|
|
import java.io.InputStream;
|
|
import java.io.InputStream;
|
|
import java.io.InputStreamReader;
|
|
import java.io.InputStreamReader;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
-import java.io.StringReader;
|
|
|
|
-import java.lang.reflect.Method;
|
|
|
|
-import java.net.InetSocketAddress;
|
|
|
|
-import java.text.MessageFormat;
|
|
|
|
-import java.util.ArrayList;
|
|
|
|
import java.util.Arrays;
|
|
import java.util.Arrays;
|
|
-import java.util.HashMap;
|
|
|
|
import java.util.HashSet;
|
|
import java.util.HashSet;
|
|
-import java.util.List;
|
|
|
|
import java.util.Locale;
|
|
import java.util.Locale;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
import java.util.Properties;
|
|
import java.util.Properties;
|
|
import java.util.Set;
|
|
import java.util.Set;
|
|
-import java.util.UUID;
|
|
|
|
|
|
|
|
/**
|
|
/**
|
|
* Mini KDC based on Apache Directory Server that can be embedded in testcases
|
|
* Mini KDC based on Apache Directory Server that can be embedded in testcases
|
|
@@ -84,9 +44,8 @@ import java.util.UUID;
|
|
* <p>
|
|
* <p>
|
|
* <b>From within testcases:</b>
|
|
* <b>From within testcases:</b>
|
|
* <p>
|
|
* <p>
|
|
- * MiniKdc sets 2 System properties when started and un-sets them when stopped:
|
|
|
|
|
|
+ * MiniKdc sets one System property when started and un-set when stopped:
|
|
* <ul>
|
|
* <ul>
|
|
- * <li>java.security.krb5.conf: set to the MiniKDC real/host/port</li>
|
|
|
|
* <li>sun.security.krb5.debug: set to the debug value provided in the
|
|
* <li>sun.security.krb5.debug: set to the debug value provided in the
|
|
* configuration</li>
|
|
* configuration</li>
|
|
* </ul>
|
|
* </ul>
|
|
@@ -116,7 +75,7 @@ public class MiniKdc {
|
|
public static final String SUN_SECURITY_KRB5_DEBUG =
|
|
public static final String SUN_SECURITY_KRB5_DEBUG =
|
|
"sun.security.krb5.debug";
|
|
"sun.security.krb5.debug";
|
|
|
|
|
|
- public static void main(String[] args) throws Exception {
|
|
|
|
|
|
+ public static void main(String[] args) throws Exception {
|
|
if (args.length < 4) {
|
|
if (args.length < 4) {
|
|
System.out.println("Arguments: <WORKDIR> <MINIKDCPROPERTIES> " +
|
|
System.out.println("Arguments: <WORKDIR> <MINIKDCPROPERTIES> " +
|
|
"<KEYTABFILE> [<PRINCIPALS>]+");
|
|
"<KEYTABFILE> [<PRINCIPALS>]+");
|
|
@@ -229,13 +188,17 @@ public class MiniKdc {
|
|
}
|
|
}
|
|
|
|
|
|
private Properties conf;
|
|
private Properties conf;
|
|
- private DirectoryService ds;
|
|
|
|
- private KdcServer kdc;
|
|
|
|
|
|
+ private SimpleKdcServer simpleKdc;
|
|
private int port;
|
|
private int port;
|
|
private String realm;
|
|
private String realm;
|
|
private File workDir;
|
|
private File workDir;
|
|
private File krb5conf;
|
|
private File krb5conf;
|
|
|
|
+ private String transport;
|
|
|
|
+ private boolean krb5Debug;
|
|
|
|
|
|
|
|
+ public void setTransport(String transport) {
|
|
|
|
+ this.transport = transport;
|
|
|
|
+ }
|
|
/**
|
|
/**
|
|
* Creates a MiniKdc.
|
|
* Creates a MiniKdc.
|
|
*
|
|
*
|
|
@@ -253,9 +216,9 @@ public class MiniKdc {
|
|
+ missingProperties);
|
|
+ missingProperties);
|
|
}
|
|
}
|
|
this.workDir = new File(workDir, Long.toString(System.currentTimeMillis()));
|
|
this.workDir = new File(workDir, Long.toString(System.currentTimeMillis()));
|
|
- if (! workDir.exists()
|
|
|
|
- && ! workDir.mkdirs()) {
|
|
|
|
- throw new RuntimeException("Cannot create directory " + workDir);
|
|
|
|
|
|
+ if (!this.workDir.exists()
|
|
|
|
+ && !this.workDir.mkdirs()) {
|
|
|
|
+ throw new RuntimeException("Cannot create directory " + this.workDir);
|
|
}
|
|
}
|
|
LOG.info("Configuration:");
|
|
LOG.info("Configuration:");
|
|
LOG.info("---------------------------------------------------------------");
|
|
LOG.info("---------------------------------------------------------------");
|
|
@@ -299,6 +262,7 @@ public class MiniKdc {
|
|
}
|
|
}
|
|
|
|
|
|
public File getKrb5conf() {
|
|
public File getKrb5conf() {
|
|
|
|
+ krb5conf = new File(System.getProperty(JAVA_SECURITY_KRB5_CONF));
|
|
return krb5conf;
|
|
return krb5conf;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -308,226 +272,81 @@ public class MiniKdc {
|
|
* @throws Exception thrown if the MiniKdc could not be started.
|
|
* @throws Exception thrown if the MiniKdc could not be started.
|
|
*/
|
|
*/
|
|
public synchronized void start() throws Exception {
|
|
public synchronized void start() throws Exception {
|
|
- if (kdc != null) {
|
|
|
|
|
|
+ if (simpleKdc != null) {
|
|
throw new RuntimeException("Already started");
|
|
throw new RuntimeException("Already started");
|
|
}
|
|
}
|
|
- initDirectoryService();
|
|
|
|
- initKDCServer();
|
|
|
|
|
|
+ simpleKdc = new SimpleKdcServer();
|
|
|
|
+ prepareKdcServer();
|
|
|
|
+ simpleKdc.init();
|
|
|
|
+ resetDefaultRealm();
|
|
|
|
+ simpleKdc.start();
|
|
|
|
+ LOG.info("MiniKdc stated.");
|
|
}
|
|
}
|
|
|
|
|
|
- private void initDirectoryService() throws Exception {
|
|
|
|
- ds = new DefaultDirectoryService();
|
|
|
|
- ds.setInstanceLayout(new InstanceLayout(workDir));
|
|
|
|
-
|
|
|
|
- CacheService cacheService = new CacheService();
|
|
|
|
- ds.setCacheService(cacheService);
|
|
|
|
-
|
|
|
|
- // first load the schema
|
|
|
|
- InstanceLayout instanceLayout = ds.getInstanceLayout();
|
|
|
|
- File schemaPartitionDirectory = new File(
|
|
|
|
- instanceLayout.getPartitionsDirectory(), "schema");
|
|
|
|
- SchemaLdifExtractor extractor = new DefaultSchemaLdifExtractor(
|
|
|
|
- instanceLayout.getPartitionsDirectory());
|
|
|
|
- extractor.extractOrCopy();
|
|
|
|
-
|
|
|
|
- SchemaLoader loader = new LdifSchemaLoader(schemaPartitionDirectory);
|
|
|
|
- SchemaManager schemaManager = new DefaultSchemaManager(loader);
|
|
|
|
- schemaManager.loadAllEnabled();
|
|
|
|
- ds.setSchemaManager(schemaManager);
|
|
|
|
- // Init the LdifPartition with schema
|
|
|
|
- LdifPartition schemaLdifPartition = new LdifPartition(schemaManager);
|
|
|
|
- schemaLdifPartition.setPartitionPath(schemaPartitionDirectory.toURI());
|
|
|
|
-
|
|
|
|
- // The schema partition
|
|
|
|
- SchemaPartition schemaPartition = new SchemaPartition(schemaManager);
|
|
|
|
- schemaPartition.setWrappedPartition(schemaLdifPartition);
|
|
|
|
- ds.setSchemaPartition(schemaPartition);
|
|
|
|
-
|
|
|
|
- JdbmPartition systemPartition = new JdbmPartition(ds.getSchemaManager());
|
|
|
|
- systemPartition.setId("system");
|
|
|
|
- systemPartition.setPartitionPath(new File(
|
|
|
|
- ds.getInstanceLayout().getPartitionsDirectory(),
|
|
|
|
- systemPartition.getId()).toURI());
|
|
|
|
- systemPartition.setSuffixDn(new Dn(ServerDNConstants.SYSTEM_DN));
|
|
|
|
- systemPartition.setSchemaManager(ds.getSchemaManager());
|
|
|
|
- ds.setSystemPartition(systemPartition);
|
|
|
|
-
|
|
|
|
- ds.getChangeLog().setEnabled(false);
|
|
|
|
- ds.setDenormalizeOpAttrsEnabled(true);
|
|
|
|
- ds.addLast(new KeyDerivationInterceptor());
|
|
|
|
-
|
|
|
|
- // create one partition
|
|
|
|
- String orgName= conf.getProperty(ORG_NAME).toLowerCase(Locale.ENGLISH);
|
|
|
|
- String orgDomain = conf.getProperty(ORG_DOMAIN).toLowerCase(Locale.ENGLISH);
|
|
|
|
-
|
|
|
|
- JdbmPartition partition = new JdbmPartition(ds.getSchemaManager());
|
|
|
|
- partition.setId(orgName);
|
|
|
|
- partition.setPartitionPath(new File(
|
|
|
|
- ds.getInstanceLayout().getPartitionsDirectory(), orgName).toURI());
|
|
|
|
- partition.setSuffixDn(new Dn("dc=" + orgName + ",dc=" + orgDomain));
|
|
|
|
- ds.addPartition(partition);
|
|
|
|
- // indexes
|
|
|
|
- Set<Index<?, ?, String>> indexedAttributes = new HashSet<Index<?, ?, String>>();
|
|
|
|
- indexedAttributes.add(new JdbmIndex<String, Entry>("objectClass", false));
|
|
|
|
- indexedAttributes.add(new JdbmIndex<String, Entry>("dc", false));
|
|
|
|
- indexedAttributes.add(new JdbmIndex<String, Entry>("ou", false));
|
|
|
|
- partition.setIndexedAttributes(indexedAttributes);
|
|
|
|
-
|
|
|
|
- // And start the ds
|
|
|
|
- ds.setInstanceId(conf.getProperty(INSTANCE));
|
|
|
|
- ds.startup();
|
|
|
|
- // context entry, after ds.startup()
|
|
|
|
- Dn dn = new Dn("dc=" + orgName + ",dc=" + orgDomain);
|
|
|
|
- Entry entry = ds.newEntry(dn);
|
|
|
|
- entry.add("objectClass", "top", "domain");
|
|
|
|
- entry.add("dc", orgName);
|
|
|
|
- ds.getAdminSession().add(entry);
|
|
|
|
|
|
+ private void resetDefaultRealm() throws IOException {
|
|
|
|
+ InputStream templateResource = new FileInputStream(
|
|
|
|
+ getKrb5conf().getAbsolutePath());
|
|
|
|
+ String content = IOUtil.readInput(templateResource);
|
|
|
|
+ content = content.replaceAll("default_realm = .*\n",
|
|
|
|
+ "default_realm = " + getRealm() + "\n");
|
|
|
|
+ IOUtil.writeFile(content, getKrb5conf());
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Convenience method that returns a resource as inputstream from the
|
|
|
|
- * classpath.
|
|
|
|
- * <p>
|
|
|
|
- * It first attempts to use the Thread's context classloader and if not
|
|
|
|
- * set it uses the class' classloader.
|
|
|
|
- *
|
|
|
|
- * @param resourceName resource to retrieve.
|
|
|
|
- *
|
|
|
|
- * @throws IOException thrown if resource cannot be loaded
|
|
|
|
- * @return inputstream with the resource.
|
|
|
|
- */
|
|
|
|
- public static InputStream getResourceAsStream(String resourceName)
|
|
|
|
- throws IOException {
|
|
|
|
- ClassLoader cl = Thread.currentThread().getContextClassLoader();
|
|
|
|
- if (cl == null) {
|
|
|
|
- cl = MiniKdc.class.getClassLoader();
|
|
|
|
- }
|
|
|
|
- InputStream is = cl.getResourceAsStream(resourceName);
|
|
|
|
- if (is == null) {
|
|
|
|
- throw new IOException("Can not read resource file '" +
|
|
|
|
- resourceName + "'");
|
|
|
|
- }
|
|
|
|
- return is;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- private void initKDCServer() throws Exception {
|
|
|
|
- String orgName= conf.getProperty(ORG_NAME);
|
|
|
|
- String orgDomain = conf.getProperty(ORG_DOMAIN);
|
|
|
|
- String bindAddress = conf.getProperty(KDC_BIND_ADDRESS);
|
|
|
|
- final Map<String, String> map = new HashMap<String, String>();
|
|
|
|
- map.put("0", orgName.toLowerCase(Locale.ENGLISH));
|
|
|
|
- map.put("1", orgDomain.toLowerCase(Locale.ENGLISH));
|
|
|
|
- map.put("2", orgName.toUpperCase(Locale.ENGLISH));
|
|
|
|
- map.put("3", orgDomain.toUpperCase(Locale.ENGLISH));
|
|
|
|
- map.put("4", bindAddress);
|
|
|
|
-
|
|
|
|
- InputStream is1 = getResourceAsStream("minikdc.ldiff");
|
|
|
|
-
|
|
|
|
- SchemaManager schemaManager = ds.getSchemaManager();
|
|
|
|
- LdifReader reader = null;
|
|
|
|
-
|
|
|
|
- try {
|
|
|
|
- final String content = StrSubstitutor.replace(IOUtils.toString(is1), map);
|
|
|
|
- reader = new LdifReader(new StringReader(content));
|
|
|
|
-
|
|
|
|
- for (LdifEntry ldifEntry : reader) {
|
|
|
|
- ds.getAdminSession().add(new DefaultEntry(schemaManager,
|
|
|
|
- ldifEntry.getEntry()));
|
|
|
|
- }
|
|
|
|
- } finally {
|
|
|
|
- IOUtils.closeQuietly(reader);
|
|
|
|
- IOUtils.closeQuietly(is1);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- KerberosConfig kerberosConfig = new KerberosConfig();
|
|
|
|
- kerberosConfig.setMaximumRenewableLifetime(Long.parseLong(conf
|
|
|
|
- .getProperty(MAX_RENEWABLE_LIFETIME)));
|
|
|
|
- kerberosConfig.setMaximumTicketLifetime(Long.parseLong(conf
|
|
|
|
- .getProperty(MAX_TICKET_LIFETIME)));
|
|
|
|
- kerberosConfig.setSearchBaseDn(String.format("dc=%s,dc=%s", orgName,
|
|
|
|
- orgDomain));
|
|
|
|
- kerberosConfig.setPaEncTimestampRequired(false);
|
|
|
|
- kdc = new KdcServer(kerberosConfig);
|
|
|
|
- kdc.setDirectoryService(ds);
|
|
|
|
-
|
|
|
|
|
|
+ private void prepareKdcServer() throws Exception {
|
|
// transport
|
|
// transport
|
|
- String transport = conf.getProperty(TRANSPORT);
|
|
|
|
- AbstractTransport absTransport;
|
|
|
|
- if (transport.trim().equals("TCP")) {
|
|
|
|
- absTransport = new TcpTransport(bindAddress, port, 3, 50);
|
|
|
|
- } else if (transport.trim().equals("UDP")) {
|
|
|
|
- absTransport = new UdpTransport(port);
|
|
|
|
- } else {
|
|
|
|
- throw new IllegalArgumentException("Invalid transport: " + transport);
|
|
|
|
|
|
+ simpleKdc.setWorkDir(workDir);
|
|
|
|
+ simpleKdc.setKdcHost(getHost());
|
|
|
|
+ simpleKdc.setKdcRealm(realm);
|
|
|
|
+ if (transport == null) {
|
|
|
|
+ transport = conf.getProperty(TRANSPORT);
|
|
}
|
|
}
|
|
- kdc.addTransports(absTransport);
|
|
|
|
- kdc.setServiceName(conf.getProperty(INSTANCE));
|
|
|
|
- kdc.start();
|
|
|
|
- // if using ephemeral port, update port number for binding
|
|
|
|
if (port == 0) {
|
|
if (port == 0) {
|
|
- InetSocketAddress addr =
|
|
|
|
- (InetSocketAddress)absTransport.getAcceptor().getLocalAddress();
|
|
|
|
- port = addr.getPort();
|
|
|
|
|
|
+ port = NetworkUtil.getServerPort();
|
|
}
|
|
}
|
|
-
|
|
|
|
- StringBuilder sb = new StringBuilder();
|
|
|
|
- InputStream is2 = getResourceAsStream("minikdc-krb5.conf");
|
|
|
|
-
|
|
|
|
- BufferedReader r = null;
|
|
|
|
-
|
|
|
|
- try {
|
|
|
|
- r = new BufferedReader(new InputStreamReader(is2, Charsets.UTF_8));
|
|
|
|
- String line = r.readLine();
|
|
|
|
-
|
|
|
|
- while (line != null) {
|
|
|
|
- sb.append(line).append("{3}");
|
|
|
|
- line = r.readLine();
|
|
|
|
|
|
+ if (transport != null) {
|
|
|
|
+ if (transport.trim().equals("TCP")) {
|
|
|
|
+ simpleKdc.setKdcTcpPort(port);
|
|
|
|
+ simpleKdc.setAllowUdp(false);
|
|
|
|
+ } else if (transport.trim().equals("UDP")) {
|
|
|
|
+ simpleKdc.setKdcUdpPort(port);
|
|
|
|
+ simpleKdc.setAllowTcp(false);
|
|
|
|
+ } else {
|
|
|
|
+ throw new IllegalArgumentException("Invalid transport: " + transport);
|
|
}
|
|
}
|
|
- } finally {
|
|
|
|
- IOUtils.closeQuietly(r);
|
|
|
|
- IOUtils.closeQuietly(is2);
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- krb5conf = new File(workDir, "krb5.conf").getAbsoluteFile();
|
|
|
|
- FileUtils.writeStringToFile(krb5conf,
|
|
|
|
- MessageFormat.format(sb.toString(), getRealm(), getHost(),
|
|
|
|
- Integer.toString(getPort()), System.getProperty("line.separator")));
|
|
|
|
- System.setProperty(JAVA_SECURITY_KRB5_CONF, krb5conf.getAbsolutePath());
|
|
|
|
-
|
|
|
|
- System.setProperty(SUN_SECURITY_KRB5_DEBUG, conf.getProperty(DEBUG,
|
|
|
|
- "false"));
|
|
|
|
-
|
|
|
|
- // refresh the config
|
|
|
|
- Class<?> classRef;
|
|
|
|
- if (System.getProperty("java.vendor").contains("IBM")) {
|
|
|
|
- classRef = Class.forName("com.ibm.security.krb5.internal.Config");
|
|
|
|
} else {
|
|
} else {
|
|
- classRef = Class.forName("sun.security.krb5.Config");
|
|
|
|
|
|
+ throw new IllegalArgumentException("Need to set transport!");
|
|
|
|
+ }
|
|
|
|
+ simpleKdc.getKdcConfig().setString(KdcConfigKey.KDC_SERVICE_NAME,
|
|
|
|
+ conf.getProperty(INSTANCE));
|
|
|
|
+ if (conf.getProperty(DEBUG) != null) {
|
|
|
|
+ krb5Debug = getAndSet(SUN_SECURITY_KRB5_DEBUG, conf.getProperty(DEBUG));
|
|
}
|
|
}
|
|
- Method refreshMethod = classRef.getMethod("refresh", new Class[0]);
|
|
|
|
- refreshMethod.invoke(classRef, new Object[0]);
|
|
|
|
-
|
|
|
|
- LOG.info("MiniKdc listening at port: {}", getPort());
|
|
|
|
- LOG.info("MiniKdc setting JVM krb5.conf to: {}",
|
|
|
|
- krb5conf.getAbsolutePath());
|
|
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
* Stops the MiniKdc
|
|
* Stops the MiniKdc
|
|
*/
|
|
*/
|
|
public synchronized void stop() {
|
|
public synchronized void stop() {
|
|
- if (kdc != null) {
|
|
|
|
- System.getProperties().remove(JAVA_SECURITY_KRB5_CONF);
|
|
|
|
- System.getProperties().remove(SUN_SECURITY_KRB5_DEBUG);
|
|
|
|
- kdc.stop();
|
|
|
|
|
|
+ if (simpleKdc != null) {
|
|
try {
|
|
try {
|
|
- ds.shutdown();
|
|
|
|
- } catch (Exception ex) {
|
|
|
|
- LOG.error("Could not shutdown ApacheDS properly: {}", ex.toString(),
|
|
|
|
- ex);
|
|
|
|
|
|
+ simpleKdc.stop();
|
|
|
|
+ } catch (KrbException e) {
|
|
|
|
+ e.printStackTrace();
|
|
|
|
+ } finally {
|
|
|
|
+ if(conf.getProperty(DEBUG) != null) {
|
|
|
|
+ System.setProperty(SUN_SECURITY_KRB5_DEBUG,
|
|
|
|
+ Boolean.toString(krb5Debug));
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
delete(workDir);
|
|
delete(workDir);
|
|
|
|
+ try {
|
|
|
|
+ // Will be fixed in next Kerby version.
|
|
|
|
+ Thread.sleep(1000);
|
|
|
|
+ } catch (InterruptedException e) {
|
|
|
|
+ e.printStackTrace();
|
|
|
|
+ }
|
|
|
|
+ LOG.info("MiniKdc stopped.");
|
|
}
|
|
}
|
|
|
|
|
|
private void delete(File f) {
|
|
private void delete(File f) {
|
|
@@ -554,55 +373,39 @@ public class MiniKdc {
|
|
*/
|
|
*/
|
|
public synchronized void createPrincipal(String principal, String password)
|
|
public synchronized void createPrincipal(String principal, String password)
|
|
throws Exception {
|
|
throws Exception {
|
|
- String orgName= conf.getProperty(ORG_NAME);
|
|
|
|
- String orgDomain = conf.getProperty(ORG_DOMAIN);
|
|
|
|
- String baseDn = "ou=users,dc=" + orgName.toLowerCase(Locale.ENGLISH)
|
|
|
|
- + ",dc=" + orgDomain.toLowerCase(Locale.ENGLISH);
|
|
|
|
- String content = "dn: uid=" + principal + "," + baseDn + "\n" +
|
|
|
|
- "objectClass: top\n" +
|
|
|
|
- "objectClass: person\n" +
|
|
|
|
- "objectClass: inetOrgPerson\n" +
|
|
|
|
- "objectClass: krb5principal\n" +
|
|
|
|
- "objectClass: krb5kdcentry\n" +
|
|
|
|
- "cn: " + principal + "\n" +
|
|
|
|
- "sn: " + principal + "\n" +
|
|
|
|
- "uid: " + principal + "\n" +
|
|
|
|
- "userPassword: " + password + "\n" +
|
|
|
|
- "krb5PrincipalName: " + principal + "@" + getRealm() + "\n" +
|
|
|
|
- "krb5KeyVersionNumber: 0";
|
|
|
|
-
|
|
|
|
- for (LdifEntry ldifEntry : new LdifReader(new StringReader(content))) {
|
|
|
|
- ds.getAdminSession().add(new DefaultEntry(ds.getSchemaManager(),
|
|
|
|
- ldifEntry.getEntry()));
|
|
|
|
- }
|
|
|
|
|
|
+ simpleKdc.createPrincipal(principal, password);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Creates multiple principals in the KDC and adds them to a keytab file.
|
|
|
|
|
|
+ * Creates multiple principals in the KDC and adds them to a keytab file.
|
|
*
|
|
*
|
|
- * @param keytabFile keytab file to add the created principal.s
|
|
|
|
|
|
+ * @param keytabFile keytab file to add the created principals.
|
|
* @param principals principals to add to the KDC, do not include the domain.
|
|
* @param principals principals to add to the KDC, do not include the domain.
|
|
* @throws Exception thrown if the principals or the keytab file could not be
|
|
* @throws Exception thrown if the principals or the keytab file could not be
|
|
* created.
|
|
* created.
|
|
*/
|
|
*/
|
|
- public void createPrincipal(File keytabFile, String ... principals)
|
|
|
|
|
|
+ public synchronized void createPrincipal(File keytabFile,
|
|
|
|
+ String ... principals)
|
|
throws Exception {
|
|
throws Exception {
|
|
- String generatedPassword = UUID.randomUUID().toString();
|
|
|
|
- Keytab keytab = new Keytab();
|
|
|
|
- List<KeytabEntry> entries = new ArrayList<KeytabEntry>();
|
|
|
|
|
|
+ simpleKdc.createPrincipals(principals);
|
|
|
|
+ if (keytabFile.exists() && !keytabFile.delete()) {
|
|
|
|
+ LOG.error("Failed to delete keytab file: " + keytabFile);
|
|
|
|
+ }
|
|
for (String principal : principals) {
|
|
for (String principal : principals) {
|
|
- createPrincipal(principal, generatedPassword);
|
|
|
|
- principal = principal + "@" + getRealm();
|
|
|
|
- KerberosTime timestamp = new KerberosTime();
|
|
|
|
- for (Map.Entry<EncryptionType, EncryptionKey> entry : KerberosKeyFactory
|
|
|
|
- .getKerberosKeys(principal, generatedPassword).entrySet()) {
|
|
|
|
- EncryptionKey ekey = entry.getValue();
|
|
|
|
- byte keyVersion = (byte) ekey.getKeyVersion();
|
|
|
|
- entries.add(new KeytabEntry(principal, 1L, timestamp, keyVersion,
|
|
|
|
- ekey));
|
|
|
|
- }
|
|
|
|
|
|
+ simpleKdc.getKadmin().exportKeytab(keytabFile, principal);
|
|
}
|
|
}
|
|
- keytab.setEntries(entries);
|
|
|
|
- keytab.write(keytabFile);
|
|
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Set the System property; return the old value for caching.
|
|
|
|
+ *
|
|
|
|
+ * @param sysprop property
|
|
|
|
+ * @param debug true or false
|
|
|
|
+ * @return the previous value
|
|
|
|
+ */
|
|
|
|
+ private boolean getAndSet(String sysprop, String debug) {
|
|
|
|
+ boolean old = Boolean.getBoolean(sysprop);
|
|
|
|
+ System.setProperty(sysprop, debug);
|
|
|
|
+ return old;
|
|
}
|
|
}
|
|
}
|
|
}
|