You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by da...@apache.org on 2016/10/24 22:02:07 UTC

kudu git commit: MiniKdc for Java

Repository: kudu
Updated Branches:
  refs/heads/master 4b5425aa3 -> 1a8ce4269


MiniKdc for Java

Change-Id: Ie24eaa94fae14ca91fb4fdd2deae1f9aec58438b
Reviewed-on: http://gerrit.cloudera.org:8080/4788
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin <as...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/1a8ce426
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/1a8ce426
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/1a8ce426

Branch: refs/heads/master
Commit: 1a8ce4269870acaf758e9140e719a496d0246631
Parents: 4b5425a
Author: Dan Burkert <da...@apache.org>
Authored: Fri Oct 21 14:05:54 2016 -0700
Committer: Dan Burkert <da...@apache.org>
Committed: Mon Oct 24 22:01:31 2016 +0000

----------------------------------------------------------------------
 .../java/org/apache/kudu/client/MiniKdc.java    | 357 +++++++++++++++++++
 .../org/apache/kudu/client/TestMiniKdc.java     |  48 +++
 src/kudu/security/mini_kdc.cc                   |  14 +-
 3 files changed, 411 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/1a8ce426/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKdc.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKdc.java b/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKdc.java
new file mode 100644
index 0000000..ecd7041
--- /dev/null
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKdc.java
@@ -0,0 +1,357 @@
+// 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.kudu.client;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Joiner;
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.io.CharStreams;
+
+import org.apache.commons.io.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A managed Kerberos Key Distribution Center.
+ *
+ * Provides utility functions to create users and services which can authenticate
+ * to the KDC.
+ *
+ * The KDC is managed as an external process, using the krb5 binaries installed on the system.
+ */
+public class MiniKdc implements Closeable {
+
+  // The KDC port will be assigned starting from this value.
+  private static final int PORT_START = 64530;
+
+  private static final Logger LOG = LoggerFactory.getLogger(MiniKuduCluster.class);
+
+  private final Options options;
+
+  Process kdcProcess;
+
+  /**
+   * Options for the MiniKdc.
+   */
+  public static class Options {
+    private final String realm;
+    private final Path dataRoot;
+    private final int port;
+
+    public Options(String realm, Path dataRoot, int port) {
+      Preconditions.checkArgument(port > 0);
+      this.realm = realm;
+      this.dataRoot = dataRoot;
+      this.port = port;
+    }
+
+    public String getRealm() {
+      return realm;
+    }
+
+    public Path getDataRoot() {
+      return dataRoot;
+    }
+
+    public int getPort() {
+      return port;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public String toString() {
+      return MoreObjects.toStringHelper(this)
+                        .add("realm", realm)
+                        .add("dataRoot", dataRoot)
+                        .add("port", port)
+                        .toString();
+    }
+  }
+
+  /**
+   * Creates a MiniKdc with explicit options.
+   */
+  public MiniKdc(Options options) {
+    this.options = options;
+  }
+
+  /**
+   * Creates a MiniKdc with default options.
+   */
+  public static MiniKdc withDefaults() throws IOException {
+    return new MiniKdc(
+        new Options("KRBTEST.COM",
+                    Paths.get(TestUtils.getBaseDir(), "krb5kdc-" + System.currentTimeMillis()),
+                    TestUtils.findFreePort(PORT_START)));
+  }
+
+  /**
+   * Start the MiniKdc.
+   */
+  public void start() throws IOException {
+    Preconditions.checkState(kdcProcess == null);
+    LOG.debug("starting KDC {}", options);
+
+    File dataRootDir = options.dataRoot.toFile();
+    if (!dataRootDir.exists()) {
+      if (!dataRootDir.mkdir()) {
+        throw new RuntimeException(String.format("unable to create krb5 state directory: %s",
+                                                 dataRootDir));
+      }
+
+      File credentialCacheDir = options.dataRoot.resolve("krb5cc").toFile();
+      if (!credentialCacheDir.mkdir()) {
+        throw new RuntimeException(String.format("unable to create credential cache directory: %s",
+                                                 credentialCacheDir));
+      }
+
+      createKdcConf();
+      createKrb5Conf();
+
+      // Create the KDC database using the kdb5_util tool.
+      checkReturnCode(
+          startProcessWithKrbEnv(
+              getBinaryPath("kdb5_util"),
+              "create",
+              "-s", // Stash the master password.
+              "-P", "masterpw", // Set a password.
+              "-W" // Use weak entropy (since we don't need real security).
+          ), "kdb5_util");
+    }
+
+    kdcProcess = startProcessWithKrbEnv(getBinaryPath("krb5kdc"),
+                                        "-n"); // Do not daemonize.
+
+    // The C++ MiniKdc defaults to binding the KDC to an ephemeral port, which
+    // it then finds using lsof at this point. Java is unable to do that since
+    // the Process API does not expose the subprocess PID. As a result, this
+    // MiniKdc doesn't support binding to an ephemeral port, and we use the
+    // race-prone TestUtils.findFreePort instead. The upside is that we
+    // don't have to rewrite the config files.
+  }
+
+  /**
+   * Creates a new Kerberos user with the given username.
+   * @param username the new user
+   */
+  void createUserPrincipal(String username) throws IOException {
+    checkReturnCode(
+        startProcessWithKrbEnv(
+            getBinaryPath("kadmin.local"),
+            "-q",
+            String.format("add_principal -pw %s %s", username, username)
+        ), "kadmin.local");
+  }
+
+  /**
+   * Kinit a user with the mini KDC.
+   * @param username the user to kinit
+   */
+  void kinit(String username) throws IOException {
+    Process proc = startProcessWithKrbEnv(getBinaryPath("kinit"), username);
+    proc.getOutputStream().write(username.getBytes());
+    proc.getOutputStream().close();
+    checkReturnCode(proc, "kinit");
+  }
+
+  /**
+   * Returns the output from the 'klist' utility. This is useful for logging the
+   * local ticket cache state.
+   */
+  String klist() throws IOException {
+    Process proc = buildProcessWithKrbEnv(getBinaryPath("klist"), "-A")
+                      .redirectOutput(ProcessBuilder.Redirect.PIPE).start();
+    checkReturnCode(proc, "klist");
+    return CharStreams.toString(new InputStreamReader(proc.getInputStream()));
+  }
+
+  /**
+   * Creates a new service principal and associated keytab, returning its path.
+   * @param spn the desired service principal name (e.g. "kudu/foo.example.com").
+   *            If the principal already exists, its key will be reset and a new
+   *            keytab will be generated.
+   * @return the path to the new services' keytab file.
+   */
+  Path createServiceKeytab(String spn) throws IOException {
+    Path kt_path = options.dataRoot.resolve(spn.replace('/', '_') + ".keytab");
+    String kadmin = getBinaryPath("kadmin.local");
+    checkReturnCode(startProcessWithKrbEnv(kadmin,
+                                           "-q",
+                                           String.format("add_principal -randkey %s", spn)),
+                    "kadmin.local");
+
+    checkReturnCode(startProcessWithKrbEnv(kadmin,
+                                           "-q",
+                                           String.format("ktadd -k %s %s", kt_path, spn)),
+                    "kadmin.local");
+    return kt_path;
+  }
+
+  private void createKrb5Conf() throws IOException {
+    List<String> contents = ImmutableList.of(
+        "[logging]",
+        "   kdc = STDERR",
+
+        "[libdefaults]",
+        "   default_ccache_name = " + "DIR:" + options.dataRoot.resolve("krb5cc"),
+        "   default_realm = " + options.realm,
+        "   dns_lookup_kdc = false",
+        "   dns_lookup_realm = false",
+        "   forwardable = true",
+        "   renew_lifetime = 7d",
+        "   ticket_lifetime = 24h",
+
+        // The KDC is configured to only use TCP, so the client should not prefer UDP.
+        "   udp_preference_limit = 0",
+
+        "[realms]",
+        options.realm + " = {",
+        "   kdc = 127.0.0.1:" + options.port,
+        "}");
+
+    Files.write(options.dataRoot.resolve("krb5.conf"), contents, Charsets.UTF_8);
+  }
+
+  private void createKdcConf() throws IOException {
+    List<String> contents = ImmutableList.of(
+        "[kdcdefaults]",
+        "   kdc_ports = \"\"",
+        "   kdc_tcp_ports = " + options.port,
+
+        "[realms]",
+        options.realm + " = {",
+        "   acl_file = " + options.dataRoot.resolve("kadm5.acl"),
+        "   admin_keytab = " + options.dataRoot.resolve("kadm5.keytab"),
+        "   database_name = " + options.dataRoot.resolve("principal"),
+        "   key_stash_file = " + options.dataRoot.resolve(".k5." + options.realm),
+        "   max_renewable_life = 7d 0h 0m 0s",
+        "}");
+
+    Files.write(options.dataRoot.resolve("kdc.conf"), contents, Charsets.UTF_8);
+  }
+
+  /**
+   * Stop the MiniKdc.
+   */
+  public void stop() throws IOException {
+    Preconditions.checkState(kdcProcess != null);
+    LOG.debug("stopping KDC {}", options);
+    try {
+      kdcProcess.destroy();
+      kdcProcess.waitFor();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    } finally {
+      kdcProcess = null;
+    }
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void close() throws IOException {
+    LOG.debug("closing KDC {}", options);
+    try {
+      if (kdcProcess != null) {
+        stop();
+      }
+    } finally {
+      FileUtils.deleteDirectory(options.dataRoot.toFile());
+    }
+  }
+
+  private static final List<String> KRB5_BINARY_PATHS = ImmutableList.of(
+      "/usr/local/opt/krb5/sbin", // Homebrew
+      "/usr/local/opt/krb5/bin", // Homebrew
+      "/opt/local/sbin", // Macports
+      "/opt/local/bin", // Macports
+      "/usr/lib/mit/sbin", // SLES
+      "/usr/sbin" // Linux
+  );
+
+  private Map<String, String> getEnvVars() {
+    return ImmutableMap.of(
+        "KRB5_CONFIG", options.dataRoot.resolve("krb5.conf").toString(),
+        "KRB5_KDC_PROFILE", options.dataRoot.resolve("kdc.conf").toString(),
+        "KRB5CCNAME", "DIR:" + options.dataRoot.resolve("krb5cc").toString()
+    );
+  }
+
+  private static String getBinaryPath(String executable) throws IOException {
+    return getBinaryPath(executable, KRB5_BINARY_PATHS);
+  }
+
+  private ProcessBuilder buildProcessWithKrbEnv(String... argv) throws IOException {
+    List<String> args = new ArrayList<>();
+    args.add("env");
+    for (Map.Entry<String, String> entry : getEnvVars().entrySet()) {
+      args.add(String.format("%s=%s", entry.getKey(), entry.getValue()));
+    }
+    args.addAll(Arrays.asList(argv));
+
+    LOG.trace("executing {}: {}", Paths.get(argv[0]).getFileName(), Joiner.on(' ').join(args));
+
+    return new ProcessBuilder(args).redirectOutput(ProcessBuilder.Redirect.INHERIT)
+                                   .redirectError(ProcessBuilder.Redirect.INHERIT)
+                                   .redirectInput(ProcessBuilder.Redirect.PIPE);
+  }
+
+  private Process startProcessWithKrbEnv(String... argv) throws IOException {
+    return buildProcessWithKrbEnv(argv).start();
+  }
+
+  private static void checkReturnCode(Process process, String name) throws IOException {
+    int ret;
+    try {
+      ret = process.waitFor();
+    } catch (InterruptedException e) {
+      Thread.interrupted();
+      throw new IOException(String.format("process '%s' interrupted", name));
+    }
+    if (ret != 0) {
+      throw new IOException(String.format("process '%s' failed: %s", name, ret));
+    }
+  }
+
+  private static String getBinaryPath(String executable,
+                                      List<String> searchPaths) throws IOException {
+    for (String path : searchPaths) {
+      File f = Paths.get(path).resolve(executable).toFile();
+      if (f.exists() && f.canExecute()) {
+        return f.getPath();
+      }
+    }
+
+    Process which = new ProcessBuilder().command("which", executable).start();
+    checkReturnCode(which, "which");
+    return CharStreams.toString(new InputStreamReader(which.getInputStream())).trim();
+  }
+}

http://git-wip-us.apache.org/repos/asf/kudu/blob/1a8ce426/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKdc.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKdc.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKdc.java
new file mode 100644
index 0000000..e1628dd
--- /dev/null
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKdc.java
@@ -0,0 +1,48 @@
+// 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.kudu.client;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class TestMiniKdc {
+
+  @Test
+  public void testBasicFunctionality() throws Exception {
+    try (MiniKdc kdc = MiniKdc.withDefaults()) {
+      kdc.start();
+
+      kdc.createUserPrincipal("alice");
+      kdc.kinit("alice");
+
+      kdc.stop();
+      kdc.start();
+
+      kdc.createUserPrincipal("bob");
+      kdc.kinit("bob");
+
+      kdc.createServiceKeytab("kudu/KRBTEST.COM");
+
+      String klist = kdc.klist();
+
+      assertTrue(klist.contains("alice@KRBTEST.COM"));
+      assertTrue(klist.contains("bob@KRBTEST.COM"));
+      assertTrue(klist.contains("krbtgt/KRBTEST.COM@KRBTEST.COM"));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/kudu/blob/1a8ce426/src/kudu/security/mini_kdc.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/mini_kdc.cc b/src/kudu/security/mini_kdc.cc
index 2546e16..25373fd 100644
--- a/src/kudu/security/mini_kdc.cc
+++ b/src/kudu/security/mini_kdc.cc
@@ -184,13 +184,11 @@ kdc_tcp_ports = $2
 
 [realms]
 $1 = {
-        max_renewable_life = 7d 0h 0m 0s
         acl_file = $0/kadm5.acl
         admin_keytab = $0/kadm5.keytab
-
         database_name = $0/principal
         key_stash_file = $0/.k5.$1
-        acl_file = $0/kadm5.acl
+        max_renewable_life = 7d 0h 0m 0s
 }
   )";
   string file_contents = strings::Substitute(kFileTemplate, options_.data_root,
@@ -206,13 +204,13 @@ Status MiniKdc::CreateKrb5Conf() const {
     kdc = STDERR
 
 [libdefaults]
+    default_ccache_name = $2
     default_realm = $1
-    dns_lookup_realm = false
     dns_lookup_kdc = false
-    ticket_lifetime = 24h
-    renew_lifetime = 7d
+    dns_lookup_realm = false
     forwardable = true
-    default_ccache_name = $2
+    renew_lifetime = 7d
+    ticket_lifetime = 24h
 
     # The KDC is configured to only use TCP, so the client should not prefer UDP.
     udp_preference_limit = 0
@@ -283,7 +281,7 @@ Status MiniKdc::CreateUserPrincipal(const string& username) {
   string kadmin;
   RETURN_NOT_OK(GetBinaryPath("kadmin.local", &kadmin));
   RETURN_NOT_OK(Subprocess::Call(MakeArgv({
-          kadmin, "-q", strings::Substitute("add_principal -pw $0 $0", username, username)})));
+          kadmin, "-q", strings::Substitute("add_principal -pw $0 $0", username)})));
   return Status::OK();
 }