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 2017/12/07 20:56:28 UTC

[1/2] kudu git commit: tablet_copy_client-test: have servers use same number of dirs

Repository: kudu
Updated Branches:
  refs/heads/master 84d019e61 -> fbed0d683


tablet_copy_client-test: have servers use same number of dirs

The TestDownloadBlock copies a block from one server to another. Before
doing so, the copy client ensures that the block it's about to copy over
does not exist in the local FS, and doing so entails doing a block
lookup.

The problem is that, since 8a81f4ff18ee30951a25bab247fa5a3c20058e49, the
client's local FS has two data directories, while the source tablet
server has three.

When running with --block_manager=file, block IDs are prefixed with the
index of the directory they are in. This is problematic because for
safety, block lookups DCHECK that the index of the block makes sense
with the number of data directories. Given the mentioned configurations,
when looking up a block in the third directory from the source tserver,
the local FS would hit the DCHECK:

F1205 10:17:50.160812 1999118336 data_dirs.cc:953] Check failed: uuid_idx < data_dirs_.size() (2 vs. 2)
*** Check failure stack trace: ***
*** Aborted at 1512497870 (unix time) try "date -d @1512497870" if you are using GNU date ***
PC: @     0x7fff99d2df06 __pthread_kill
*** SIGABRT (@0x7fff99d2df06) received by PID 53106 (TID 0x7fff77282000) stack trace: ***
    @     0x7fff86cea52a _sigtramp
    @                0x2 (unknown)
    @     0x7fff8acd86df abort
    @        0x108b3d049 google::logging_fail()
    @        0x108b3d03a google::LogMessage::Fail()
    @        0x108b3c129 google::LogMessage::SendToLog()
    @        0x108b3c7cd google::LogMessage::Flush()
    @        0x108b405ef google::LogMessageFatal::~LogMessageFatal()
    @        0x108b3d3d9 google::LogMessageFatal::~LogMessageFatal()
    @        0x10719e0b1 kudu::fs::DataDirManager::FindDataDirByUuidIndex()
    @        0x1071dbc5f kudu::fs::FileBlockManager::FindBlockPath()
    @        0x1071e234a kudu::fs::FileBlockManager::OpenBlock()
    @        0x1072019dc kudu::FsManager::OpenBlock()
    @        0x104666ee1 kudu::tserver::TabletCopyTest::ReadLocalBlockFile()
    @        0x104665eda kudu::tserver::TabletCopyClientTest_TestDownloadBlock_Test::TestBody()
    @        0x10817ea5a testing::internal::HandleExceptionsInMethodIfSupported<>()
    @        0x10817e94f testing::Test::Run()
    @        0x10817f94e testing::TestInfo::Run()
    @        0x108180303 testing::TestCase::Run()
    @        0x10818774b testing::internal::UnitTestImpl::RunAllTests()
    @        0x108187303 testing::internal::HandleExceptionsInMethodIfSupported<>()
    @        0x10818721e testing::UnitTest::Run()
    @        0x1047939e1 RUN_ALL_TESTS()
    @        0x1047935fd main
    @     0x7fff913265ad start
    @                0x1 (unknown)

The solution in this patch is to make the number of directories in the
local FS equal the number of directories in the remote tserver.

A slight change to TestDownloadAllBlocks has also been made to
accommodate the change in directories.

Change-Id: I4c65a10a2a0f32ad0ecb467073dbec9965d2b093
Reviewed-on: http://gerrit.cloudera.org:8080/8767
Tested-by: Kudu Jenkins
Reviewed-by: Hao Hao <ha...@cloudera.com>
Reviewed-by: Mike Percy <mp...@apache.org>


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

Branch: refs/heads/master
Commit: 9baebd6dab18c378551fcdd9048cb519db21e644
Parents: 84d019e
Author: Andrew Wong <aw...@cloudera.com>
Authored: Tue Dec 5 11:29:47 2017 -0800
Committer: Andrew Wong <aw...@cloudera.com>
Committed: Wed Dec 6 06:28:50 2017 +0000

----------------------------------------------------------------------
 src/kudu/tserver/tablet_copy-test-base.h    |  5 ++++-
 src/kudu/tserver/tablet_copy_client-test.cc | 16 +++++++++++-----
 2 files changed, 15 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/9baebd6d/src/kudu/tserver/tablet_copy-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy-test-base.h b/src/kudu/tserver/tablet_copy-test-base.h
index 638435d..99624b5 100644
--- a/src/kudu/tserver/tablet_copy-test-base.h
+++ b/src/kudu/tserver/tablet_copy-test-base.h
@@ -46,7 +46,7 @@ class TabletCopyTest : public TabletServerTestBase {
     // Create a tablet server with multiple data dirs. In most cases, this is
     // unimportant, but in some cases can be helpful to test multi-disk
     // behavior and disk failures.
-    NO_FATALS(StartTabletServer(/* num_data_dirs */ 3));
+    NO_FATALS(StartTabletServer(kNumDataDirs));
     // Prevent logs from being deleted out from under us until / unless we want
     // to test that we are anchoring correctly. Since GenerateTestData() does a
     // Flush(), Log GC is allowed to eat the logs before we get around to
@@ -62,6 +62,9 @@ class TabletCopyTest : public TabletServerTestBase {
   }
 
  protected:
+  // Number of data directories on the copying server.
+  const int kNumDataDirs = 3;
+
   // Grab the first column block we find in the SuperBlock.
   static BlockId FirstColumnBlockId(const tablet::TabletSuperBlockPB& superblock) {
     DCHECK_GT(superblock.rowsets_size(), 0);

http://git-wip-us.apache.org/repos/asf/kudu/blob/9baebd6d/src/kudu/tserver/tablet_copy_client-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_client-test.cc b/src/kudu/tserver/tablet_copy_client-test.cc
index 7ac12cd..cc5b695 100644
--- a/src/kudu/tserver/tablet_copy_client-test.cc
+++ b/src/kudu/tserver/tablet_copy_client-test.cc
@@ -84,6 +84,7 @@ using consensus::RaftPeerPB;
 using fs::DataDirManager;
 using std::tuple;
 using std::unique_ptr;
+using strings::Substitute;
 using tablet::TabletMetadata;
 
 class TabletCopyClientTest : public TabletCopyTest {
@@ -96,8 +97,9 @@ class TabletCopyClientTest : public TabletCopyTest {
     const string kTestDataDirPrefix = GetTestPath("client_tablet_data");
     FsManagerOpts opts;
     opts.wal_root = kTestWalDir;
-    opts.data_roots.emplace_back(kTestDataDirPrefix + "A");
-    opts.data_roots.emplace_back(kTestDataDirPrefix + "B");
+    for (int dir = 0; dir < kNumDataDirs; dir++) {
+      opts.data_roots.emplace_back(Substitute("$0-$1", kTestDataDirPrefix, dir));
+    }
 
     metric_entity_ = METRIC_ENTITY_server.Instantiate(&metric_registry_, "test");
     opts.metric_entity = metric_entity_;
@@ -147,7 +149,7 @@ Status TabletCopyClientTest::CompareFileContents(const string& path1, const stri
   RETURN_NOT_OK(file2->Size(&size2));
   if (size1 != size2) {
     return Status::Corruption("Sizes of files don't match",
-                              strings::Substitute("$0 vs $1 bytes", size1, size2));
+                              Substitute("$0 vs $1 bytes", size1, size2));
   }
 
   faststring scratch1, scratch2;
@@ -259,11 +261,15 @@ TEST_F(TabletCopyClientTest, TestDownloadAllBlocks) {
   ASSERT_OK(client_->transaction_->CommitCreatedBlocks());
 
   // Verify the disk synchronization count.
+  // TODO(awong): These values have been determined to be safe empirically.
+  // If kNumDataDirs changes, these values may also change. The point of this
+  // test is to exemplify the difference in syncs between the log and file
+  // block managers, but it would be nice to formulate a bound here.
   if (FLAGS_block_manager == "log") {
-    ASSERT_EQ(6, down_cast<Counter*>(
+    ASSERT_GE(9, down_cast<Counter*>(
         metric_entity_->FindOrNull(METRIC_block_manager_total_disk_sync).get())->value());
   } else {
-    ASSERT_EQ(18, down_cast<Counter*>(
+    ASSERT_GE(22, down_cast<Counter*>(
         metric_entity_->FindOrNull(METRIC_block_manager_total_disk_sync).get())->value());
   }
 


[2/2] kudu git commit: KUDU-2121: fix SASL PLAIN fallback with rpc-authentication=optional

Posted by da...@apache.org.
KUDU-2121: fix SASL PLAIN fallback with rpc-authentication=optional

The original issue was filed for certain Java edge cases, however
Sailesh and David who are working with KRPC in Impala pointed out that
the C++ side never does SASL PLAIN fallback when the server has Kerberos
enabled. This commit fixes both clients to correctly fall back to SASL
PLAIN when the server is Kerberized and authentication is optional.

Detecting whether the client has Kerberos credentials requires
using the GSSAPI directly instead of the SASL api. As a result, we now
find GSSAPI while building. Since GSSAPI is a direct dependency of Cyrus
SASL, this shouldn't cause any issues or require updating installation
docs.

Change-Id: I3f42f4b7a8ac767ccae439feb1dcd49080827276
Reviewed-on: http://gerrit.cloudera.org:8080/8755
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/fbed0d68
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/fbed0d68
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/fbed0d68

Branch: refs/heads/master
Commit: fbed0d683f634dd92e1a02874e705ecce66bfbc2
Parents: 9baebd6
Author: Dan Burkert <da...@apache.org>
Authored: Fri Dec 1 09:40:43 2017 -0800
Committer: Dan Burkert <da...@apache.org>
Committed: Thu Dec 7 20:55:30 2017 +0000

----------------------------------------------------------------------
 CMakeLists.txt                                  |   7 +
 cmake_modules/FindGSSAPI.cmake                  |  34 +++++
 .../java/org/apache/kudu/client/Negotiator.java |  59 +++++---
 .../org/apache/kudu/client/MiniKuduCluster.java |  10 ++
 .../apache/kudu/client/TestMiniKuduCluster.java |   1 +
 .../org/apache/kudu/client/TestNegotiation.java |  61 ++++++++
 .../org/apache/kudu/client/TestSecurity.java    |   4 +-
 .../integration-tests/security-faults-itest.cc  |  20 +--
 src/kudu/integration-tests/security-itest.cc    |  30 ++--
 .../mini-cluster/external_mini_cluster-test.cc  |   7 +-
 src/kudu/mini-cluster/external_mini_cluster.cc  |  17 ++-
 src/kudu/rpc/CMakeLists.txt                     |   1 +
 src/kudu/rpc/client_negotiation.cc              | 143 ++++++++++++++-----
 src/kudu/rpc/client_negotiation.h               |   3 +
 src/kudu/rpc/negotiation-test.cc                |   8 +-
 src/kudu/security/tls_handshake.cc              |  10 +-
 src/kudu/tools/tool.proto                       |   4 +
 src/kudu/tools/tool_action_test.cc              |   8 ++
 18 files changed, 331 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 8afae55..f246455 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -818,6 +818,13 @@ ADD_THIRDPARTY_LIB(cyrus_sasl
   SHARED_LIB "${CYRUS_SASL_SHARED_LIB}"
   DEPS ${CYRUS_SASL_LIB_DEPS})
 
+## GSSAPI
+find_package(GSSAPI REQUIRED)
+include_directories(SYSTEM ${GSSAPI_INCLUDE_DIR})
+ADD_THIRDPARTY_LIB(gssapi_krb5
+  SHARED_LIB "${GSSAPI_SHARED_LIB}"
+  DEPS ${GSSAPI_LIB_DEPS})
+
 ## GLog
 find_package(GLog REQUIRED)
 include_directories(SYSTEM ${GLOG_INCLUDE_DIR})

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/cmake_modules/FindGSSAPI.cmake
----------------------------------------------------------------------
diff --git a/cmake_modules/FindGSSAPI.cmake b/cmake_modules/FindGSSAPI.cmake
new file mode 100644
index 0000000..19c0691
--- /dev/null
+++ b/cmake_modules/FindGSSAPI.cmake
@@ -0,0 +1,34 @@
+# 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.
+#
+# - Find GSSAPI SASL (gssapi.h, gssapi_krb5.so)
+#
+# This module defines
+#  GSSAPI_INCLUDE_DIR, directory containing headers
+#  GSSAPI_SHARED_LIB, path to GSSAPI's shared library
+#  GSSAPI_FOUND, whether GSSAPI has been found
+#
+# N.B: we do _not_ include GSSAPI in thirdparty. In practice, GSSAPI (like
+# Cyrus-SASL) is so commonly used and generally non-ABI-breaking that we should
+# be OK to depend on the host installation.
+
+find_path(GSSAPI_INCLUDE_DIR gssapi/gssapi.h)
+find_library(GSSAPI_SHARED_LIB gssapi_krb5)
+
+include(FindPackageHandleStandardArgs)
+find_package_handle_standard_args(GSSAPI_VARS
+  GSSAPI_SHARED_LIB GSSAPI_INCLUDE_DIR)

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java b/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
index ad62b9f..029c516 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
@@ -46,6 +46,7 @@ import javax.security.auth.callback.CallbackHandler;
 import javax.security.auth.callback.NameCallback;
 import javax.security.auth.callback.PasswordCallback;
 import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.kerberos.KerberosTicket;
 import javax.security.sasl.Sasl;
 import javax.security.sasl.SaslClient;
 import javax.security.sasl.SaslException;
@@ -98,11 +99,14 @@ public class Negotiator extends SimpleChannelUpstreamHandler {
           RpcHeader.RpcFeatureFlag.TLS);
 
   /**
-   * List of SASL mechanisms supported by the client, in descending priority order.
+   * Set of SASL mechanisms supported by the client, in descending priority order.
    * The client will pick the first of these mechanisms that is supported by
    * the server and also succeeds to initialize.
    */
-  private static final String[] PRIORITIZED_MECHS = new String[] { "GSSAPI", "PLAIN" };
+  private enum SaslMechanism {
+    GSSAPI,
+    PLAIN,
+  }
 
   static final int CONNECTION_CTX_CALL_ID = -3;
   static final int SASL_CALL_ID = -33;
@@ -131,7 +135,7 @@ public class Negotiator extends SimpleChannelUpstreamHandler {
   private SaslClient saslClient;
 
   /** The negotiated mechanism, set after NEGOTIATE stage. */
-  private String chosenMech;
+  private SaslMechanism chosenMech;
 
   /** The negotiated authentication type, set after NEGOTIATE state. */
   private AuthenticationTypePB.TypeCase chosenAuthnType;
@@ -343,31 +347,49 @@ public class Negotiator extends SimpleChannelUpstreamHandler {
 
   private void chooseAndInitializeSaslMech(NegotiatePB response) throws NonRecoverableException {
     // Gather the set of server-supported mechanisms.
-    Set<String> serverMechs = Sets.newHashSet();
+    Map<String, String> errorsByMech = Maps.newHashMap();
+    Set<SaslMechanism> serverMechs = Sets.newHashSet();
     for (RpcHeader.NegotiatePB.SaslMechanism mech : response.getSaslMechanismsList()) {
-      serverMechs.add(mech.getMechanism());
+      switch (mech.getMechanism().toUpperCase()) {
+        case "GSSAPI":
+          serverMechs.add(SaslMechanism.GSSAPI);
+          break;
+        case "PLAIN":
+          serverMechs.add(SaslMechanism.PLAIN);
+          break;
+        default:
+          errorsByMech.put(mech.getMechanism(), "unrecognized mechanism");
+          break;
+      }
     }
 
     // For each of our own mechanisms, in descending priority, check if
     // the server also supports them. If so, try to initialize saslClient.
     // If we find a common mechanism that also can be successfully initialized,
     // choose that mech.
-    Map<String, String> errorsByMech = Maps.newHashMap();
-    for (String clientMech : PRIORITIZED_MECHS) {
+    for (SaslMechanism clientMech : SaslMechanism.values()) {
+
+      if (clientMech.equals(SaslMechanism.GSSAPI) &&
+          (securityContext.getSubject() == null ||
+           securityContext.getSubject().getPrivateCredentials(KerberosTicket.class).isEmpty())) {
+        errorsByMech.put(clientMech.name(), "Client does not have Kerberos credentials (tgt)");
+        continue;
+      }
+
       if (!serverMechs.contains(clientMech)) {
-        errorsByMech.put(clientMech, "not advertised by server");
+        errorsByMech.put(clientMech.name(), "not advertised by server");
         continue;
       }
       Map<String, String> props = Maps.newHashMap();
       // If the negotiated mechanism is GSSAPI (Kerberos), configure SASL to use
       // integrity protection so that the channel bindings and nonce can be
       // verified.
-      if ("GSSAPI".equals(clientMech)) {
+      if (clientMech == SaslMechanism.GSSAPI) {
         props.put(Sasl.QOP, "auth-int");
       }
 
       try {
-        saslClient = Sasl.createSaslClient(new String[]{ clientMech },
+        saslClient = Sasl.createSaslClient(new String[]{ clientMech.name() },
                                            null,
                                            "kudu",
                                            remoteHostname,
@@ -376,27 +398,28 @@ public class Negotiator extends SimpleChannelUpstreamHandler {
         chosenMech = clientMech;
         break;
       } catch (SaslException e) {
-        errorsByMech.put(clientMech, e.getMessage());
+        errorsByMech.put(clientMech.name(), e.getMessage());
       }
     }
 
     if (chosenMech != null) {
-      LOG.debug("SASL mechanism {} chosen for peer {}", chosenMech, remoteHostname);
+      LOG.debug("SASL mechanism {} chosen for peer {}", chosenMech.name(), remoteHostname);
       return;
     }
 
     // TODO(KUDU-1948): when the Java client has an option to require security, detect the case
     // where the server is configured without Kerberos and the client requires it.
     String message;
-    if (serverMechs.size() == 1 && serverMechs.contains("GSSAPI")) {
+    if (serverMechs.size() == 1 && serverMechs.contains(SaslMechanism.GSSAPI)) {
       // Give a better error diagnostic for common case of an unauthenticated client connecting
       // to a secure server.
-      message = "Server requires Kerberos, but this client is not authenticated (kinit)";
+      message = "server requires authentication, " +
+                "but client does not have Kerberos credentials available";
     } else {
-      message = "Unable to negotiate a matching mechanism. Errors: [" +
+      message = "client/server supported SASL mechanism mismatch: [" +
                 Joiner.on(", ").withKeyValueSeparator(": ").join(errorsByMech) + "]";
     }
-    throw new NonRecoverableException(Status.ConfigurationError(message));
+    throw new NonRecoverableException(Status.NotAuthorized(message));
   }
 
   private AuthenticationTypePB.TypeCase chooseAuthenticationType(NegotiatePB response) {
@@ -582,7 +605,7 @@ public class Negotiator extends SimpleChannelUpstreamHandler {
       builder.setToken(UnsafeByteOperations.unsafeWrap(initialResponse));
     }
     builder.setStep(RpcHeader.NegotiatePB.NegotiateStep.SASL_INITIATE);
-    builder.addSaslMechanismsBuilder().setMechanism(chosenMech);
+    builder.addSaslMechanismsBuilder().setMechanism(chosenMech.name());
     state = State.AWAIT_SASL;
     sendSaslMessage(chan, builder.build());
   }
@@ -625,7 +648,7 @@ public class Negotiator extends SimpleChannelUpstreamHandler {
   private void handleSuccessResponse(Channel chan, NegotiatePB response) throws IOException {
     Preconditions.checkState(saslClient.isComplete(),
                              "server sent SASL_SUCCESS step, but SASL negotiation is not complete");
-    if (chosenMech.equals("GSSAPI")) {
+    if (chosenMech == SaslMechanism.GSSAPI) {
       if (response.hasNonce()) {
         // Grab the nonce from the server, if it has sent one. We'll send it back
         // later with SASL integrity protection as part of the connection context.

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/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 962bf46..5d5a498 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
@@ -40,6 +40,7 @@ import org.apache.kudu.tools.Tool.DaemonInfoPB;
 import org.apache.kudu.tools.Tool.GetKDCEnvVarsRequestPB;
 import org.apache.kudu.tools.Tool.GetMastersRequestPB;
 import org.apache.kudu.tools.Tool.GetTServersRequestPB;
+import org.apache.kudu.tools.Tool.KdestroyRequestPB;
 import org.apache.kudu.tools.Tool.StartClusterRequestPB;
 import org.apache.kudu.tools.Tool.StartDaemonRequestPB;
 import org.apache.kudu.tools.Tool.StopDaemonRequestPB;
@@ -354,6 +355,15 @@ public class MiniKuduCluster implements AutoCloseable {
     }
   }
 
+  /**
+   * Removes all credentials for all principals from the KDC credential cache.
+   */
+  public void kdestroy() throws IOException {
+    sendRequestToCluster(ControlShellRequestPB.newBuilder()
+                                              .setKdestroy(KdestroyRequestPB.getDefaultInstance())
+                                              .build());
+  }
+
   /** {@override} */
   @Override
   public void close() {

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKuduCluster.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKuduCluster.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKuduCluster.java
index 6f7640e..0adafee 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKuduCluster.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestMiniKuduCluster.java
@@ -73,6 +73,7 @@ public class TestMiniKuduCluster {
 
   @Test(timeout = 50000)
   public void testKerberos() throws Exception {
+    FakeDNS.getInstance().install();
     try (MiniKuduCluster cluster = new MiniKuduCluster.MiniKuduClusterBuilder()
                                                       .numMasters(NUM_MASTERS)
                                                       .numTservers(NUM_TABLET_SERVERS)

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiation.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiation.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiation.java
new file mode 100644
index 0000000..633c4d9
--- /dev/null
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiation.java
@@ -0,0 +1,61 @@
+// 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.apache.kudu.util.CapturingLogAppender;
+import org.junit.Test;
+
+import java.io.Closeable;
+
+import static junit.framework.TestCase.assertTrue;
+
+public class TestNegotiation {
+
+  /**
+   * Test that a non-Kerberized client will use SASL PLAIN to connect to a
+   * Kerberized server which doesn't require authentication. Regression test for
+   * KUDU-2121.
+   */
+  @Test
+  public void testSaslPlainFallback() throws Exception {
+    FakeDNS.getInstance().install();
+    MiniKuduCluster.MiniKuduClusterBuilder clusterBuilder =
+        new MiniKuduCluster.MiniKuduClusterBuilder();
+
+    clusterBuilder.numMasters(1)
+                  .numTservers(0)
+                  .enableKerberos()
+                  .addMasterFlag("--rpc-authentication=optional")
+                  .addMasterFlag("--rpc-trace-negotiation")
+                  .addMasterFlag("--user-acl=*");
+
+    CapturingLogAppender cla = new CapturingLogAppender();
+    try (MiniKuduCluster cluster = clusterBuilder.build()) {
+      cluster.kdestroy();
+      try (Closeable c = cla.attach();
+           KuduClient client = new KuduClient.KuduClientBuilder(cluster.getMasterAddresses())
+                                             .build()
+      ) {
+        assertTrue(client.getTablesList().getTablesList().isEmpty());
+      }
+    }
+
+    assertTrue(cla.getAppendedText(),
+               cla.getAppendedText().contains("Client requested to use mechanism: PLAIN"));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java
index 50b566a..d2f8fcb 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestSecurity.java
@@ -53,8 +53,8 @@ public class TestSecurity extends BaseKuduTest {
         Assert.fail("should not have been able to connect to a secure cluster " +
             "with no credentials");
       } catch (NonRecoverableException e) {
-        Assert.assertTrue(e.getMessage().contains(
-            "Server requires Kerberos, but this client is not authenticated"));
+        Assert.assertTrue(e.getMessage().contains("server requires authentication, " +
+            "but client does not have Kerberos credentials available"));
       }
 
       // If we import the authentication data from the old authenticated client,

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/src/kudu/integration-tests/security-faults-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/security-faults-itest.cc b/src/kudu/integration-tests/security-faults-itest.cc
index ecfa84f..3873039 100644
--- a/src/kudu/integration-tests/security-faults-itest.cc
+++ b/src/kudu/integration-tests/security-faults-itest.cc
@@ -229,28 +229,12 @@ TEST_F(SecurityComponentsFaultsITest, KdcRestartsInTheMiddle) {
   // Wait for Kerberos tickets to expire.
   SleepFor(MonoDelta::FromSeconds(krb_lifetime_seconds_));
 
-#ifndef __APPLE__
-  // For some reason (may be caching negative responses?), calling this on OS X
-  // causes the next call to SmokeTestCluster() fail. Not sure that's expected
-  // or correct, so disabling this for OS X until it's clarified.
-
-  // It seems different version of krb5 library handles the error differently:
-  // in some cases, the error is about ticket expiration, in other -- failure
-  // to contact KDC.
-  //
-  // Also, different versions of krb5 library have different error messages
-  // for the same error.
   const Status s = SmokeTestCluster();
   ASSERT_TRUE(s.IsNotAuthorized()) << s.ToString();
   ASSERT_STR_MATCHES(s.ToString(),
       "Not authorized: Could not connect to the cluster: "
-      "Client connection negotiation failed: client connection to .* ("
-      "Cannot contact any KDC for realm .*|"
-      "Ticket expired.*|"
-      "GSSAPI Error:  The context has expire.*|"
-      "GSSAPI Error: The referenced context has expired .*|"
-      "GSSAPI Error: The referenced credential has expired .*)");
-#endif
+      "Client connection negotiation failed: client connection to .*: "
+      "server requires authentication, but client does not have Kerberos credentials available");
 
   ASSERT_OK(cluster_->kdc()->Start());
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/src/kudu/integration-tests/security-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/security-itest.cc b/src/kudu/integration-tests/security-itest.cc
index 74a3ba9..16c0175 100644
--- a/src/kudu/integration-tests/security-itest.cc
+++ b/src/kudu/integration-tests/security-itest.cc
@@ -181,26 +181,38 @@ TEST_F(SecurityITest, SmokeTestAsAuthorizedUser) {
             s.ToString());
 }
 
-#ifndef __APPLE__
 // Test trying to access the cluster with no Kerberos credentials at all.
-// This test is ignored on macOS because the system Kerberos implementation
-// (Heimdal) caches the non-existence of client credentials, which causes
-// subsequent tests to fail.
 TEST_F(SecurityITest, TestNoKerberosCredentials) {
   ASSERT_OK(StartCluster());
   ASSERT_OK(cluster_->kdc()->Kdestroy());
 
   client::sp::shared_ptr<KuduClient> client;
   Status s = cluster_->CreateClient(nullptr, &client);
-  // The error message differs on el6 from newer krb5 implementations,
-  // so we'll check for either one.
   ASSERT_STR_MATCHES(s.ToString(),
                      "Not authorized: Could not connect to the cluster: "
                      "Client connection negotiation failed: client connection "
-                     "to .*: (No Kerberos credentials available|"
-                     "Credentials cache file.*not found)");
+                     "to .*: server requires authentication, "
+                     "but client does not have Kerberos credentials available");
+}
+
+// Regression test for KUDU-2121. Set up a Kerberized cluster with optional
+// authentication. An un-Kerberized client should be able to connect with SASL
+// PLAIN authentication.
+TEST_F(SecurityITest, SaslPlainFallback) {
+  cluster_opts_.num_masters = 1;
+  cluster_opts_.num_tablet_servers = 0;
+  cluster_opts_.extra_master_flags.emplace_back("--rpc-authentication=optional");
+  cluster_opts_.extra_master_flags.emplace_back("--user-acl=*");
+  ASSERT_OK(StartCluster());
+  ASSERT_OK(cluster_->kdc()->Kdestroy());
+
+  client::sp::shared_ptr<KuduClient> client;
+  ASSERT_OK(cluster_->CreateClient(nullptr, &client));
+
+  // Check client can successfully call ListTables().
+  vector<string> tables;
+  ASSERT_OK(client->ListTables(&tables));
 }
-#endif
 
 // Test cluster access by a user who is not authorized as a client.
 TEST_F(SecurityITest, TestUnauthorizedClientKerberosCredentials) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/src/kudu/mini-cluster/external_mini_cluster-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/mini-cluster/external_mini_cluster-test.cc b/src/kudu/mini-cluster/external_mini_cluster-test.cc
index cb1bef3..288e5df 100644
--- a/src/kudu/mini-cluster/external_mini_cluster-test.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster-test.cc
@@ -197,10 +197,9 @@ TEST_P(ExternalMiniClusterTest, TestBasicOperation) {
     Status s = cluster.SetFlag(ts, "foo", "bar");
     // The error differs depending on the version of Kerberos, so we match
     // either message.
-    ASSERT_STR_MATCHES(s.ToString(), "Not authorized.*"
-                       "(Credentials cache file.*not found|"
-                        "No Kerberos credentials|"
-                        ".*No such file or directory)");
+    ASSERT_STR_CONTAINS(s.ToString(),
+                        "server requires authentication, "
+                        "but client does not have Kerberos credentials available");
   }
 
   // Verify that the HMS is reachable.

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/src/kudu/mini-cluster/external_mini_cluster.cc
----------------------------------------------------------------------
diff --git a/src/kudu/mini-cluster/external_mini_cluster.cc b/src/kudu/mini-cluster/external_mini_cluster.cc
index e827e5d..1495b7b 100644
--- a/src/kudu/mini-cluster/external_mini_cluster.cc
+++ b/src/kudu/mini-cluster/external_mini_cluster.cc
@@ -174,6 +174,7 @@ Status ExternalMiniCluster::Start() {
 
     RETURN_NOT_OK_PREPEND(kdc_->Kinit("test-admin"),
                           "could not kinit as admin");
+
     RETURN_NOT_OK_PREPEND(kdc_->SetKrb5Environment(),
                           "could not set krb5 client env");
   }
@@ -690,11 +691,17 @@ Status ExternalDaemon::EnableKerberos(MiniKdc* kdc, const string& bind_host) {
   RETURN_NOT_OK_PREPEND(kdc->CreateServiceKeytab(spn, &ktpath),
                         "could not create keytab");
   extra_env_ = kdc->GetEnvVars();
-  opts_.extra_flags.push_back(Substitute("--keytab_file=$0", ktpath));
-  opts_.extra_flags.push_back(Substitute("--principal=$0", spn));
-  opts_.extra_flags.emplace_back("--rpc_authentication=required");
-  opts_.extra_flags.emplace_back("--superuser_acl=test-admin");
-  opts_.extra_flags.emplace_back("--user_acl=test-user");
+
+  // Insert Kerberos flags at the front of extra_flags, so that user specified
+  // flags will override them.
+  opts_.extra_flags.insert(opts_.extra_flags.begin(), {
+    Substitute("--keytab_file=$0", ktpath),
+    Substitute("--principal=$0", spn),
+    "--rpc_authentication=required",
+    "--superuser_acl=test-admin",
+    "--user_acl=test-user",
+  });
+
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/src/kudu/rpc/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/CMakeLists.txt b/src/kudu/rpc/CMakeLists.txt
index 4555702..5a3a2b0 100644
--- a/src/kudu/rpc/CMakeLists.txt
+++ b/src/kudu/rpc/CMakeLists.txt
@@ -76,6 +76,7 @@ set(KRPC_SRCS
 
 set(KRPC_LIBS
   cyrus_sasl
+  gssapi_krb5
   gutil
   kudu_util
   libev

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/src/kudu/rpc/client_negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/client_negotiation.cc b/src/kudu/rpc/client_negotiation.cc
index 0a4a372..219ad22 100644
--- a/src/kudu/rpc/client_negotiation.cc
+++ b/src/kudu/rpc/client_negotiation.cc
@@ -17,8 +17,8 @@
 
 #include "kudu/rpc/client_negotiation.h"
 
-#include <cstring>
 #include <cstdint>
+#include <cstring>
 #include <map>
 #include <memory>
 #include <ostream>
@@ -27,6 +27,8 @@
 
 #include <gflags/gflags_declare.h>
 #include <glog/logging.h>
+#include <gssapi/gssapi.h>
+#include <gssapi/gssapi_krb5.h>
 #include <sasl/sasl.h>
 
 #include "kudu/gutil/basictypes.h"
@@ -293,7 +295,7 @@ Status ClientNegotiation::InitSaslClient() {
           &callbacks_[0],               // Connection-specific callbacks.
           flags,
           &sasl_conn);
-    }), "Unable to create new SASL client");
+    }), Substitute("unable to create new SASL $0 client", sasl_proto_name_));
   sasl_conn_.reset(sasl_conn);
   return Status::OK();
 }
@@ -399,7 +401,7 @@ Status ClientNegotiation::HandleNegotiate(const NegotiatePB& response) {
   DCHECK_EQ(negotiated_authn_, AuthenticationType::SASL);
 
   // Build a map of the SASL mechanisms offered by the server.
-  const set<SaslMechanism::Type>& client_mechs = helper_.EnabledMechs();
+  set<SaslMechanism::Type> client_mechs(helper_.EnabledMechs());
   set<SaslMechanism::Type> server_mechs;
   for (const NegotiatePB::SaslMechanism& sasl_mech : response.sasl_mechanisms()) {
     auto mech = SaslMechanism::value_of(sasl_mech.mechanism());
@@ -414,40 +416,50 @@ Status ClientNegotiation::HandleNegotiate(const NegotiatePB& response) {
   // The preference list in order of most to least preferred:
   //  * GSSAPI
   //  * PLAIN
-  set<SaslMechanism::Type> common_mechs = STLSetIntersection(client_mechs, server_mechs);
-
-  if (common_mechs.empty()) {
-    if (ContainsKey(server_mechs, SaslMechanism::GSSAPI) &&
-        !ContainsKey(client_mechs, SaslMechanism::GSSAPI)) {
-      return Status::NotAuthorized("server requires authentication, "
-                                   "but client does not have Kerberos enabled");
-    }
-    if (!ContainsKey(server_mechs, SaslMechanism::GSSAPI) &&
-        ContainsKey(client_mechs, SaslMechanism::GSSAPI)) {
-      return Status::NotAuthorized("client requires authentication, "
-                                   "but server does not have Kerberos enabled");
+  //
+  // TODO(KUDU-1921): allow the client to require authentication.
+  if (ContainsKey(client_mechs, SaslMechanism::GSSAPI) &&
+      ContainsKey(server_mechs, SaslMechanism::GSSAPI)) {
+
+    // Check that the client has local Kerberos credentials, and if not fall
+    // back to an alternate mechanism.
+    Status s = CheckGSSAPI();
+    if (s.ok()) {
+      negotiated_mech_ = SaslMechanism::GSSAPI;
+      return Status::OK();
     }
-    string msg = Substitute("client/server supported SASL mechanism mismatch; "
-                            "client mechanisms: [$0], server mechanisms: [$1]",
-                            JoinMapped(client_mechs, SaslMechanism::name_of, ", "),
-                            JoinMapped(server_mechs, SaslMechanism::name_of, ", "));
 
-    // For now, there should never be a SASL mechanism mismatch that isn't due
-    // to one of the sides requiring Kerberos and the other not having it, so
-    // lets sanity check that.
-    DLOG(FATAL) << msg;
-    return Status::NotAuthorized(msg);
+    TRACE("Kerberos authentication credentials are not available: $0", s.ToString());
+    client_mechs.erase(SaslMechanism::GSSAPI);
   }
 
-  // TODO(KUDU-1921): allow the client to require authentication.
-  if (ContainsKey(common_mechs, SaslMechanism::GSSAPI)) {
-    negotiated_mech_ = SaslMechanism::GSSAPI;
-  } else {
-    DCHECK(ContainsKey(common_mechs, SaslMechanism::PLAIN));
+  if (ContainsKey(client_mechs, SaslMechanism::PLAIN) &&
+      ContainsKey(server_mechs, SaslMechanism::PLAIN)) {
     negotiated_mech_ = SaslMechanism::PLAIN;
-  }
-
-  return Status::OK();
+    return Status::OK();
+  }
+
+  // There are no mechanisms in common.
+  if (ContainsKey(server_mechs, SaslMechanism::GSSAPI) &&
+      !ContainsKey(client_mechs, SaslMechanism::GSSAPI)) {
+    return Status::NotAuthorized("server requires authentication, "
+                                  "but client does not have Kerberos credentials available");
+  }
+  if (!ContainsKey(server_mechs, SaslMechanism::GSSAPI) &&
+      ContainsKey(client_mechs, SaslMechanism::GSSAPI)) {
+    return Status::NotAuthorized("client requires authentication, "
+                                  "but server does not have Kerberos enabled");
+  }
+  string msg = Substitute("client/server supported SASL mechanism mismatch; "
+                          "client mechanisms: [$0], server mechanisms: [$1]",
+                          JoinMapped(client_mechs, SaslMechanism::name_of, ", "),
+                          JoinMapped(server_mechs, SaslMechanism::name_of, ", "));
+
+  // For now, there should never be a SASL mechanism mismatch that isn't due
+  // to one of the sides requiring Kerberos and the other not having it, so
+  // lets sanity check that.
+  DCHECK(STLSetIntersection(client_mechs, server_mechs).empty()) << msg;
+  return Status::NotAuthorized(msg);
 }
 
 Status ClientNegotiation::SendTlsHandshake(string tls_token) {
@@ -770,5 +782,72 @@ int ClientNegotiation::SecretCb(sasl_conn_t* conn, int id, sasl_secret_t** psecr
   return SASL_OK;
 }
 
+namespace {
+// Retrieve the GSSAPI error description for an error code and type.
+string gss_error_description(OM_uint32 code, int type) {
+  string description;
+  OM_uint32 message_context = 0;
+
+  do {
+    if (!description.empty()) {
+      description.append(": ");
+    }
+    OM_uint32 minor = 0;
+    gss_buffer_desc buf;
+    gss_display_status(&minor, code, type, GSS_C_NULL_OID, &message_context, &buf);
+    description.append(static_cast<const char*>(buf.value), buf.length);
+    gss_release_buffer(&minor, &buf);
+  } while (message_context != 0);
+
+  return description;
+}
+
+// Transforms a GSSAPI major and minor error code into a Kudu Status.
+Status check_gss_error(OM_uint32 major, OM_uint32 minor) {
+    if (GSS_ERROR(major)) {
+      return Status::NotAuthorized(gss_error_description(major, GSS_C_GSS_CODE),
+                                   gss_error_description(minor, GSS_C_MECH_CODE));
+    }
+    return Status::OK();
+}
+} // anonymous namespace
+
+Status ClientNegotiation::CheckGSSAPI() {
+  OM_uint32 major, minor;
+  gss_cred_id_t cred = GSS_C_NO_CREDENTIAL;
+
+  // Acquire the Kerberos credential. This will fail if the client does not have
+  // a Kerberos tgt ticket. In theory it should be sufficient to call
+  // gss_inquire_cred_by_mech, but that causes a memory leak on RHEL 7.
+  major = gss_acquire_cred(&minor,
+                           GSS_C_NO_NAME,
+                           GSS_C_INDEFINITE,
+                           const_cast<gss_OID_set>(gss_mech_set_krb5),
+                           GSS_C_INITIATE,
+                           &cred,
+                           nullptr,
+                           nullptr);
+  Status s = check_gss_error(major, minor);
+
+  // Inspect the Kerberos credential to determine if it is expired. The lifetime
+  // returned from gss_acquire_cred in the RHEL 6 version of krb5 is always 0,
+  // so it has to be done with a separate call to gss_inquire_cred. The lifetime
+  // holds the remaining validity of the tgt in seconds.
+  OM_uint32 lifetime;
+  if (s.ok()) {
+    major = gss_inquire_cred(&minor, cred, nullptr, &lifetime, nullptr, nullptr);
+    s = check_gss_error(major, minor);
+  }
+
+  // Release the credential even if gss_inquire_cred fails.
+  gss_release_cred(&minor, &cred);
+  RETURN_NOT_OK(s);
+
+  if (lifetime == 0) {
+    return Status::NotAuthorized("Kerberos ticket expired");
+  }
+  return Status::OK();
+}
+
 } // namespace rpc
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/src/kudu/rpc/client_negotiation.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/client_negotiation.h b/src/kudu/rpc/client_negotiation.h
index 541798f..19a155c 100644
--- a/src/kudu/rpc/client_negotiation.h
+++ b/src/kudu/rpc/client_negotiation.h
@@ -139,6 +139,9 @@ class ClientNegotiation {
   // SASL callback for SASL_CB_PASS
   int SecretCb(sasl_conn_t* conn, int id, sasl_secret_t** psecret);
 
+  // Check that GSSAPI/Kerberos credentials are available.
+  static Status CheckGSSAPI() WARN_UNUSED_RESULT;
+
  private:
 
   // Encode and send the specified negotiate request message to the server.

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/src/kudu/rpc/negotiation-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/negotiation-test.cc b/src/kudu/rpc/negotiation-test.cc
index 4158ffc..67f93a5 100644
--- a/src/kudu/rpc/negotiation-test.cc
+++ b/src/kudu/rpc/negotiation-test.cc
@@ -552,7 +552,7 @@ INSTANTIATE_TEST_CASE_P(NegotiationCombinations,
           },
           false,
           false,
-          Status::NotAuthorized(".*client does not have Kerberos enabled"),
+          Status::NotAuthorized(".*client does not have Kerberos credentials available"),
           Status::NetworkError(""),
           AuthenticationType::INVALID,
           SaslMechanism::INVALID,
@@ -1075,7 +1075,6 @@ TEST_F(TestNegotiation, TestGSSAPIInvalidNegotiation) {
 #endif
                 }));
 
-
   // Create the server principal and keytab.
   string kt_path;
   ASSERT_OK(kdc.CreateServiceKeytab("kudu/127.0.0.1", &kt_path));
@@ -1094,10 +1093,9 @@ TEST_F(TestNegotiation, TestGSSAPIInvalidNegotiation) {
       std::bind(RunGSSAPINegotiationClient, std::placeholders::_1,
                 [](const Status& s, ClientNegotiation& client) {
                   CHECK(s.IsNotAuthorized());
-#ifndef KRB5_VERSION_LE_1_10
                   ASSERT_STR_MATCHES(s.ToString(),
-                                     "Not authorized: No Kerberos credentials available.*");
-#endif
+                                     "Not authorized: server requires authentication, "
+                                     "but client does not have Kerberos credentials available");
                 }));
 
   // Create and kinit as a client user.

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/src/kudu/security/tls_handshake.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_handshake.cc b/src/kudu/security/tls_handshake.cc
index 254b553..51b74fe 100644
--- a/src/kudu/security/tls_handshake.cc
+++ b/src/kudu/security/tls_handshake.cc
@@ -24,6 +24,7 @@
 #include <openssl/x509.h>
 #include <openssl/x509v3.h>
 
+#include "kudu/gutil/strings/strip.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/security/cert.h"
 #include "kudu/security/tls_socket.h"
@@ -255,14 +256,17 @@ string TlsHandshake::GetCipherDescription() const {
   SCOPED_OPENSSL_NO_PENDING_ERRORS;
   CHECK(has_started_);
   const SSL_CIPHER* cipher = SSL_get_current_cipher(ssl_.get());
-  if (cipher == nullptr) {
+  if (!cipher) {
     return "NONE";
   }
   char buf[512];
-  const char* ret = SSL_CIPHER_description(cipher, buf, sizeof(buf));
-  if (ret == nullptr) {
+  const char* description = SSL_CIPHER_description(cipher, buf, sizeof(buf));
+  if (!description) {
     return "NONE";
   }
+  string ret(description);
+  StripTrailingNewline(&ret);
+  StripDupCharacters(&ret, ' ', 0);
   return ret;
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/src/kudu/tools/tool.proto
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool.proto b/src/kudu/tools/tool.proto
index 6107c26..1c1e5a3 100644
--- a/src/kudu/tools/tool.proto
+++ b/src/kudu/tools/tool.proto
@@ -140,6 +140,9 @@ message GetKDCEnvVarsResponsePB {
 // It is an error to call this on a non-Kerberized cluster.
 message GetKDCEnvVarsRequestPB {}
 
+// Removes all credentials for all principals from the KDC credential cache.
+message KdestroyRequestPB {};
+
 // Sent by the control shell in response to a control shell command request.
 message ControlShellResponsePB {
 
@@ -172,5 +175,6 @@ message ControlShellRequestPB {
     GetMastersRequestPB get_masters = 7;
     GetTServersRequestPB get_tservers = 8;
     GetKDCEnvVarsRequestPB get_kdc_env_vars = 9;
+    KdestroyRequestPB kdestroy = 10;
   }
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/fbed0d68/src/kudu/tools/tool_action_test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tools/tool_action_test.cc b/src/kudu/tools/tool_action_test.cc
index 504b4e1..5e01ae9 100644
--- a/src/kudu/tools/tool_action_test.cc
+++ b/src/kudu/tools/tool_action_test.cc
@@ -270,6 +270,14 @@ Status ProcessRequest(const ControlShellRequestPB& req,
           env_vars.begin(), env_vars.end());
       break;
     }
+    case ControlShellRequestPB::kKdestroy:
+    {
+      if (!(*cluster)->kdc()) {
+        RETURN_NOT_OK(Status::NotFound("kdc not found"));
+      }
+      RETURN_NOT_OK((*cluster)->kdc()->Kdestroy());
+      break;
+    }
     default:
       RETURN_NOT_OK(Status::InvalidArgument("unknown cluster control request"));
   }