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/12/10 00:35:49 UTC

kudu git commit: MiniKuduClusterBuilder.build should clean up state when initialization fails

Repository: kudu
Updated Branches:
  refs/heads/master 251860b36 -> e018a837c


MiniKuduClusterBuilder.build should clean up state when initialization fails

The MiniKuduCluster starts processes and does many other potentially
failing operations as part of the constructor. This change moves these
potentially failing operations out of the constructor so that the
MiniKuduCluster can be shutdown cleanly on partial-initialization
failure. This change should be completely transparent to users of the
public API.

https://lists.apache.org/thread.html/922614e80f3f64be885f67380bf44730132896dc21480b28d0a93108@%3Cdev.kudu.apache.org%3E
contains more discussion.

Change-Id: I024872cddf352a9b75dab87af685ca3af50099c7
Reviewed-on: http://gerrit.cloudera.org:8080/5448
Reviewed-by: Adar Dembo <ad...@cloudera.com>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: e018a837ce4808dfeb828ed3ac268fa4cac93a8f
Parents: 251860b
Author: Dan Burkert <da...@apache.org>
Authored: Fri Dec 9 15:03:03 2016 -0800
Committer: Dan Burkert <da...@apache.org>
Committed: Sat Dec 10 00:35:24 2016 +0000

----------------------------------------------------------------------
 .../org/apache/kudu/client/MiniKuduCluster.java | 131 ++++++++++---------
 1 file changed, 71 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/e018a837/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java b/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
index ed49a1c..3a05ccc 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
@@ -35,7 +35,6 @@ import javax.security.auth.login.Configuration;
 import javax.security.auth.login.LoginContext;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Splitter;
 import com.google.common.base.Stopwatch;
@@ -75,72 +74,76 @@ public class MiniKuduCluster implements AutoCloseable {
 
   private final List<String> pathsToDelete = new ArrayList<>();
   private final List<HostAndPort> masterHostPorts = new ArrayList<>();
-  private List<Integer> tserverPorts = new ArrayList<>();
-  private ImmutableList<String> extraTserverFlags;
-  private ImmutableList<String> extraMasterFlags;
+  private final List<Integer> tserverPorts = new ArrayList<>();
+  private final ImmutableList<String> extraTserverFlags;
+  private final ImmutableList<String> extraMasterFlags;
 
   // Client we can use for common operations.
-  private final KuduClient syncClient;
+  private KuduClient syncClient;
   private final int defaultTimeoutMs;
 
   private String masterAddresses;
 
   private final String bindHost = TestUtils.getUniqueLocalhost();
-  private final Path keytab;
-  private final MiniKdc miniKdc;
-  private final Subject subject;
-
-  private MiniKuduCluster(int numMasters,
-                          int numTservers,
-                          final int defaultTimeoutMs,
-                          boolean enableKerberos,
+  private Path keytab;
+  private MiniKdc miniKdc;
+  private Subject subject;
+
+  private MiniKuduCluster(final int defaultTimeoutMs,
                           final List<String> extraTserverFlags,
-                          final List<String> extraMasterFlags) throws Exception {
+                          final List<String> extraMasterFlags) {
     this.defaultTimeoutMs = defaultTimeoutMs;
     this.extraTserverFlags = ImmutableList.copyOf(extraTserverFlags);
     this.extraMasterFlags = ImmutableList.copyOf(extraMasterFlags);
+  }
 
-    if (enableKerberos) {
-      miniKdc = MiniKdc.withDefaults();
-      miniKdc.start();
-
-      keytab = miniKdc.createServiceKeytab("kudu/" + bindHost);
-
-      miniKdc.createUserPrincipal("testuser");
-      miniKdc.kinit("testuser");
-      System.setProperty("java.security.krb5.conf", miniKdc.getEnvVars().get("KRB5_CONFIG"));
-
-      Configuration conf = new Configuration() {
-        @Override
-        public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
-          Map<String, String> options = new HashMap<>();
-          options.put("useKeyTab", "true");
-          options.put("useTicketCache", "true");
-          options.put("ticketCache", miniKdc.getEnvVars().get("KRB5CCNAME"));
-          options.put("principal", "testuser");
-          options.put("doNotPrompt", "true");
-          options.put("renewTGT", "true");
-          options.put("debug", "true");
-
-          return new AppConfigurationEntry[] {
-            new AppConfigurationEntry("com.sun.security.auth.module.Krb5LoginModule",
-                                      AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
-                                      options)
-          };
-        }
-      };
+  /**
+   * Enable Kerberos security for this cluster, start the MiniKdc, and log in
+   * the required subjects.
+   */
+  private void startKerberos() throws Exception {
+    miniKdc = MiniKdc.withDefaults();
+    miniKdc.start();
 
-      LoginContext context = new LoginContext("com.sun.security.auth.module.Krb5LoginModule",
-                                              new Subject(), null, conf);
-      context.login();
-      context.getSubject();
-      subject = context.getSubject();
-    } else {
-      miniKdc = null;
-      keytab = null;
-      subject = null;
-    }
+    keytab = miniKdc.createServiceKeytab("kudu/" + bindHost);
+
+    miniKdc.createUserPrincipal("testuser");
+    miniKdc.kinit("testuser");
+    System.setProperty("java.security.krb5.conf", miniKdc.getEnvVars().get("KRB5_CONFIG"));
+
+    Configuration conf = new Configuration() {
+      @Override
+      public AppConfigurationEntry[] getAppConfigurationEntry(String name) {
+        Map<String, String> options = new HashMap<>();
+        options.put("useKeyTab", "true");
+        options.put("useTicketCache", "true");
+        options.put("ticketCache", miniKdc.getEnvVars().get("KRB5CCNAME"));
+        options.put("principal", "testuser");
+        options.put("doNotPrompt", "true");
+        options.put("renewTGT", "true");
+        options.put("debug", "true");
+
+        return new AppConfigurationEntry[] {
+          new AppConfigurationEntry("com.sun.security.auth.module.Krb5LoginModule",
+                                    AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
+                                    options)
+        };
+      }
+    };
+
+    LoginContext context = new LoginContext("com.sun.security.auth.module.Krb5LoginModule",
+                                            new Subject(), null, conf);
+    context.login();
+    context.getSubject();
+    subject = context.getSubject();
+  }
 
+  /**
+   * Start the master and tablet server processes.
+   * @param numMasters the number of masters to start.
+   * @param numTservers the number of tablet servers to start.
+   */
+  private void start(int numMasters, int numTservers) throws Exception {
     startCluster(numMasters, numTservers);
 
     PrivilegedAction<KuduClient> createClient = new PrivilegedAction<KuduClient>() {
@@ -430,12 +433,9 @@ public class MiniKuduCluster implements AutoCloseable {
     destroyAndWaitForProcess(master);
   }
 
-  /**
-   * See {@link #shutdown()}.
-   * @throws Exception never thrown, exceptions are logged
-   */
+  /** {@override} */
   @Override
-  public void close() throws Exception {
+  public void close() {
     shutdown();
   }
 
@@ -627,8 +627,19 @@ public class MiniKuduCluster implements AutoCloseable {
     }
 
     public MiniKuduCluster build() throws Exception {
-      return new MiniKuduCluster(numMasters, numTservers, defaultTimeoutMs,
-          enableKerberos, extraTserverFlags, extraMasterFlags);
+      MiniKuduCluster cluster =
+          new MiniKuduCluster(defaultTimeoutMs, extraTserverFlags, extraMasterFlags);
+      try {
+        if (enableKerberos) {
+          cluster.startKerberos();
+        }
+        cluster.start(numMasters, numTservers);
+      } catch (Exception e) {
+        // MiniKuduCluster.close should not throw, so no need for a nested try/catch.
+        cluster.close();
+        throw e;
+      }
+      return cluster;
     }
   }
 }