You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by me...@apache.org on 2015/06/21 10:46:18 UTC

[1/4] mesos git commit: Moved cram-md5 authenticatee process definition into implementation file.

Repository: mesos
Updated Branches:
  refs/heads/master 89dbd3d22 -> effc3636d


Moved cram-md5 authenticatee process definition into implementation file.

Removing the process from the header is much cleaner and also fixes the
linked clang 3.4.2 JIRA. Apart from that moving, no code is changed.

Review: https://reviews.apache.org/r/32850


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

Branch: refs/heads/master
Commit: 362bf1702fe1854bd0f086f7e3c7d6d1e90676ac
Parents: 89dbd3d
Author: Till Toenshoff <to...@me.com>
Authored: Sun Jun 21 08:16:12 2015 +0000
Committer: Adam B <ad...@mesosphere.io>
Committed: Sun Jun 21 08:17:05 2015 +0000

----------------------------------------------------------------------
 src/Makefile.am                               |   1 +
 src/authentication/cram_md5/authenticatee.cpp | 401 +++++++++++++++++++++
 src/authentication/cram_md5/authenticatee.hpp | 375 +------------------
 3 files changed, 407 insertions(+), 370 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/362bf170/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index e7de0f3..dfebd2b 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -347,6 +347,7 @@ nodist_libmesos_no_3rdparty_la_SOURCES =				\
 # TODO(tillt): Remove authentication/cram_md5/* which will enable us to
 # lose the immediate cyrus-sasl2 dependency.
 libmesos_no_3rdparty_la_SOURCES =					\
+	authentication/cram_md5/authenticatee.cpp			\
 	authentication/cram_md5/authenticator.cpp			\
 	authentication/cram_md5/auxprop.cpp				\
 	authorizer/authorizer.cpp					\

http://git-wip-us.apache.org/repos/asf/mesos/blob/362bf170/src/authentication/cram_md5/authenticatee.cpp
----------------------------------------------------------------------
diff --git a/src/authentication/cram_md5/authenticatee.cpp b/src/authentication/cram_md5/authenticatee.cpp
new file mode 100644
index 0000000..7143ef6
--- /dev/null
+++ b/src/authentication/cram_md5/authenticatee.cpp
@@ -0,0 +1,401 @@
+/**
+ * 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.
+ */
+
+#include "authentication/cram_md5/authenticatee.hpp"
+
+#include <stddef.h>   // For size_t needed by sasl.h.
+
+#include <sasl/sasl.h>
+
+#include <string>
+
+#include <process/defer.hpp>
+#include <process/dispatch.hpp>
+#include <process/once.hpp>
+#include <process/process.hpp>
+#include <process/protobuf.hpp>
+
+#include <stout/strings.hpp>
+
+#include "logging/logging.hpp"
+
+#include "messages/messages.hpp"
+
+namespace mesos {
+namespace internal {
+namespace cram_md5 {
+
+using namespace process;
+using std::string;
+
+class CRAMMD5AuthenticateeProcess
+  : public ProtobufProcess<CRAMMD5AuthenticateeProcess>
+{
+public:
+  CRAMMD5AuthenticateeProcess(
+      const Credential& _credential,
+      const UPID& _client)
+    : ProcessBase(ID::generate("crammd5_authenticatee")),
+      credential(_credential),
+      client(_client),
+      status(READY),
+      connection(NULL)
+  {
+    const char* data = credential.secret().data();
+    size_t length = credential.secret().length();
+
+    // Need to allocate the secret via 'malloc' because SASL is
+    // expecting the data appended to the end of the struct. *sigh*
+    secret = (sasl_secret_t*) malloc(sizeof(sasl_secret_t) + length);
+
+    CHECK(secret != NULL) << "Failed to allocate memory for secret";
+
+    memcpy(secret->data, data, length);
+    secret->len = length;
+  }
+
+  virtual ~CRAMMD5AuthenticateeProcess()
+  {
+    if (connection != NULL) {
+      sasl_dispose(&connection);
+    }
+    free(secret);
+  }
+
+  virtual void finalize()
+  {
+    discarded(); // Fail the promise.
+  }
+
+  Future<bool> authenticate(const UPID& pid)
+  {
+    static Once* initialize = new Once();
+    static bool initialized = false;
+
+    if (!initialize->once()) {
+      LOG(INFO) << "Initializing client SASL";
+      int result = sasl_client_init(NULL);
+      if (result != SASL_OK) {
+        status = ERROR;
+        string error(sasl_errstring(result, NULL, NULL));
+        promise.fail("Failed to initialize SASL: " + error);
+        initialize->done();
+        return promise.future();
+      }
+
+      initialized = true;
+
+      initialize->done();
+    }
+
+    if (!initialized) {
+      promise.fail("Failed to initialize SASL");
+      return promise.future();
+    }
+
+    if (status != READY) {
+      return promise.future();
+    }
+
+    LOG(INFO) << "Creating new client SASL connection";
+
+    callbacks[0].id = SASL_CB_GETREALM;
+    callbacks[0].proc = NULL;
+    callbacks[0].context = NULL;
+
+    callbacks[1].id = SASL_CB_USER;
+    callbacks[1].proc = (int(*)()) &user;
+    callbacks[1].context = (void*) credential.principal().c_str();
+
+    // NOTE: Some SASL mechanisms do not allow/enable "proxying",
+    // i.e., authorization. Therefore, some mechanisms send _only_ the
+    // authoriation name rather than both the user (authentication
+    // name) and authorization name. Thus, for now, we assume
+    // authorization is handled out of band. Consider the
+    // SASL_NEED_PROXY flag if we want to reconsider this in the
+    // future.
+    callbacks[2].id = SASL_CB_AUTHNAME;
+    callbacks[2].proc = (int(*)()) &user;
+    callbacks[2].context = (void*) credential.principal().c_str();
+
+    callbacks[3].id = SASL_CB_PASS;
+    callbacks[3].proc = (int(*)()) &pass;
+    callbacks[3].context = (void*) secret;
+
+    callbacks[4].id = SASL_CB_LIST_END;
+    callbacks[4].proc = NULL;
+    callbacks[4].context = NULL;
+
+    int result = sasl_client_new(
+        "mesos",    // Registered name of service.
+        NULL,       // Server's FQDN.
+        NULL, NULL, // IP Address information strings.
+        callbacks,  // Callbacks supported only for this connection.
+        0,          // Security flags (security layers are enabled
+                    // using security properties, separately).
+        &connection);
+
+    if (result != SASL_OK) {
+      status = ERROR;
+      string error(sasl_errstring(result, NULL, NULL));
+      promise.fail("Failed to create client SASL connection: " + error);
+      return promise.future();
+    }
+
+    AuthenticateMessage message;
+    message.set_pid(client);
+    send(pid, message);
+
+    status = STARTING;
+
+    // Stop authenticating if nobody cares.
+    promise.future().onDiscard(defer(self(), &Self::discarded));
+
+    return promise.future();
+  }
+
+protected:
+  virtual void initialize()
+  {
+    // Anticipate mechanisms and steps from the server.
+    install<AuthenticationMechanismsMessage>(
+        &CRAMMD5AuthenticateeProcess::mechanisms,
+        &AuthenticationMechanismsMessage::mechanisms);
+
+    install<AuthenticationStepMessage>(
+        &CRAMMD5AuthenticateeProcess::step,
+        &AuthenticationStepMessage::data);
+
+    install<AuthenticationCompletedMessage>(
+        &CRAMMD5AuthenticateeProcess::completed);
+
+    install<AuthenticationFailedMessage>(
+        &CRAMMD5AuthenticateeProcess::failed);
+
+    install<AuthenticationErrorMessage>(
+        &CRAMMD5AuthenticateeProcess::error,
+        &AuthenticationErrorMessage::error);
+  }
+
+  void mechanisms(const std::vector<string>& mechanisms)
+  {
+    if (status != STARTING) {
+      status = ERROR;
+      promise.fail("Unexpected authentication 'mechanisms' received");
+      return;
+    }
+
+    // TODO(benh): Store 'from' in order to ensure we only communicate
+    // with the same Authenticator.
+
+    LOG(INFO) << "Received SASL authentication mechanisms: "
+              << strings::join(",", mechanisms);
+
+    sasl_interact_t* interact = NULL;
+    const char* output = NULL;
+    unsigned length = 0;
+    const char* mechanism = NULL;
+
+    int result = sasl_client_start(
+        connection,
+        strings::join(" ", mechanisms).c_str(),
+        &interact,     // Set if an interaction is needed.
+        &output,       // The output string (to send to server).
+        &length,       // The length of the output string.
+        &mechanism);   // The chosen mechanism.
+
+    CHECK_NE(SASL_INTERACT, result)
+      << "Not expecting an interaction (ID: " << interact->id << ")";
+
+    if (result != SASL_OK && result != SASL_CONTINUE) {
+      string error(sasl_errdetail(connection));
+      status = ERROR;
+      promise.fail("Failed to start the SASL client: " + error);
+      return;
+    }
+
+    LOG(INFO) << "Attempting to authenticate with mechanism '"
+              << mechanism << "'";
+
+    AuthenticationStartMessage message;
+    message.set_mechanism(mechanism);
+    message.set_data(output, length);
+
+    reply(message);
+
+    status = STEPPING;
+  }
+
+  void step(const string& data)
+  {
+    if (status != STEPPING) {
+      status = ERROR;
+      promise.fail("Unexpected authentication 'step' received");
+      return;
+    }
+
+    LOG(INFO) << "Received SASL authentication step";
+
+    sasl_interact_t* interact = NULL;
+    const char* output = NULL;
+    unsigned length = 0;
+
+    int result = sasl_client_step(
+        connection,
+        data.length() == 0 ? NULL : data.data(),
+        data.length(),
+        &interact,
+        &output,
+        &length);
+
+    CHECK_NE(SASL_INTERACT, result)
+      << "Not expecting an interaction (ID: " << interact->id << ")";
+
+    if (result == SASL_OK || result == SASL_CONTINUE) {
+      // We don't start the client with SASL_SUCCESS_DATA so we may
+      // need to send one more "empty" message to the server.
+      AuthenticationStepMessage message;
+      if (output != NULL && length > 0) {
+        message.set_data(output, length);
+      }
+      reply(message);
+    } else {
+      status = ERROR;
+      string error(sasl_errdetail(connection));
+      promise.fail("Failed to perform authentication step: " + error);
+    }
+  }
+
+  void completed()
+  {
+    if (status != STEPPING) {
+      status = ERROR;
+      promise.fail("Unexpected authentication 'completed' received");
+      return;
+    }
+
+    LOG(INFO) << "Authentication success";
+
+    status = COMPLETED;
+    promise.set(true);
+  }
+
+  void failed()
+  {
+    status = FAILED;
+    promise.set(false);
+  }
+
+  void error(const string& error)
+  {
+    status = ERROR;
+    promise.fail("Authentication error: " + error);
+  }
+
+  void discarded()
+  {
+    status = DISCARDED;
+    promise.fail("Authentication discarded");
+  }
+
+private:
+  static int user(
+      void* context,
+      int id,
+      const char** result,
+      unsigned* length)
+  {
+    CHECK(SASL_CB_USER == id || SASL_CB_AUTHNAME == id);
+    *result = static_cast<const char*>(context);
+    if (length != NULL) {
+      *length = strlen(*result);
+    }
+    return SASL_OK;
+  }
+
+  static int pass(
+      sasl_conn_t* connection,
+      void* context,
+      int id,
+      sasl_secret_t** secret)
+  {
+    CHECK_EQ(SASL_CB_PASS, id);
+    *secret = static_cast<sasl_secret_t*>(context);
+    return SASL_OK;
+  }
+
+  const Credential credential;
+
+  // PID of the client that needs to be authenticated.
+  const UPID client;
+
+  sasl_secret_t* secret;
+
+  sasl_callback_t callbacks[5];
+
+  enum {
+    READY,
+    STARTING,
+    STEPPING,
+    COMPLETED,
+    FAILED,
+    ERROR,
+    DISCARDED
+  } status;
+
+  sasl_conn_t* connection;
+
+  Promise<bool> promise;
+};
+
+
+Try<Authenticatee*> CRAMMD5Authenticatee::create()
+{
+  return new CRAMMD5Authenticatee();
+}
+
+
+CRAMMD5Authenticatee::CRAMMD5Authenticatee() : process(NULL) {}
+
+
+CRAMMD5Authenticatee::~CRAMMD5Authenticatee()
+{
+  if (process != NULL) {
+    terminate(process);
+    wait(process);
+    delete process;
+  }
+}
+
+
+Future<bool> CRAMMD5Authenticatee::authenticate(
+  const UPID& pid,
+  const UPID& client,
+  const mesos::Credential& credential)
+{
+  CHECK(process == NULL);
+  process = new CRAMMD5AuthenticateeProcess(credential, client);
+  spawn(process);
+
+  return dispatch(
+      process, &CRAMMD5AuthenticateeProcess::authenticate, pid);
+}
+
+} // namespace cram_md5 {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/362bf170/src/authentication/cram_md5/authenticatee.hpp
----------------------------------------------------------------------
diff --git a/src/authentication/cram_md5/authenticatee.hpp b/src/authentication/cram_md5/authenticatee.hpp
index 55fac68..9d6293c 100644
--- a/src/authentication/cram_md5/authenticatee.hpp
+++ b/src/authentication/cram_md5/authenticatee.hpp
@@ -19,27 +19,14 @@
 #ifndef __AUTHENTICATION_CRAM_MD5_AUTHENTICATEE_HPP__
 #define __AUTHENTICATION_CRAM_MD5_AUTHENTICATEE_HPP__
 
-#include <stddef.h>   // For size_t needed by sasl.h.
-
-#include <sasl/sasl.h>
-
-#include <string>
-
 #include <mesos/mesos.hpp>
 
 #include <mesos/module/authenticatee.hpp>
 
-#include <process/defer.hpp>
 #include <process/future.hpp>
 #include <process/id.hpp>
-#include <process/once.hpp>
-#include <process/process.hpp>
-#include <process/protobuf.hpp>
 
-#include <stout/strings.hpp>
-
-
-#include "messages/messages.hpp"
+#include <stout/try.hpp>
 
 namespace mesos {
 namespace internal {
@@ -59,367 +46,15 @@ public:
 
   virtual ~CRAMMD5Authenticatee();
 
-  process::Future<bool> authenticate(const process::UPID& pid,
-                                     const process::UPID& client,
-                                     const Credential& credential);
+  process::Future<bool> authenticate(
+      const process::UPID& pid,
+      const process::UPID& client,
+      const Credential& credential);
 
 private:
   CRAMMD5AuthenticateeProcess* process;
 };
 
-
-class CRAMMD5AuthenticateeProcess
-  : public ProtobufProcess<CRAMMD5AuthenticateeProcess>
-{
-public:
-  CRAMMD5AuthenticateeProcess(const Credential& _credential,
-                              const process::UPID& _client)
-    : ProcessBase(process::ID::generate("crammd5_authenticatee")),
-      credential(_credential),
-      client(_client),
-      status(READY),
-      connection(NULL)
-  {
-    const char* data = credential.secret().data();
-    size_t length = credential.secret().length();
-
-    // Need to allocate the secret via 'malloc' because SASL is
-    // expecting the data appended to the end of the struct. *sigh*
-    secret = (sasl_secret_t*) malloc(sizeof(sasl_secret_t) + length);
-
-    CHECK(secret != NULL) << "Failed to allocate memory for secret";
-
-    memcpy(secret->data, data, length);
-    secret->len = length;
-  }
-
-  virtual ~CRAMMD5AuthenticateeProcess()
-  {
-    if (connection != NULL) {
-      sasl_dispose(&connection);
-    }
-    free(secret);
-  }
-
-  virtual void finalize()
-  {
-    discarded(); // Fail the promise.
-  }
-
-  process::Future<bool> authenticate(const process::UPID& pid)
-  {
-    static process::Once* initialize = new process::Once();
-    static bool initialized = false;
-
-    if (!initialize->once()) {
-      LOG(INFO) << "Initializing client SASL";
-      int result = sasl_client_init(NULL);
-      if (result != SASL_OK) {
-        status = ERROR;
-        std::string error(sasl_errstring(result, NULL, NULL));
-        promise.fail("Failed to initialize SASL: " + error);
-        initialize->done();
-        return promise.future();
-      }
-
-      initialized = true;
-
-      initialize->done();
-    }
-
-    if (!initialized) {
-      promise.fail("Failed to initialize SASL");
-      return promise.future();
-    }
-
-    if (status != READY) {
-      return promise.future();
-    }
-
-    LOG(INFO) << "Creating new client SASL connection";
-
-    callbacks[0].id = SASL_CB_GETREALM;
-    callbacks[0].proc = NULL;
-    callbacks[0].context = NULL;
-
-    callbacks[1].id = SASL_CB_USER;
-    callbacks[1].proc = (int(*)()) &user;
-    callbacks[1].context = (void*) credential.principal().c_str();
-
-    // NOTE: Some SASL mechanisms do not allow/enable "proxying",
-    // i.e., authorization. Therefore, some mechanisms send _only_ the
-    // authoriation name rather than both the user (authentication
-    // name) and authorization name. Thus, for now, we assume
-    // authorization is handled out of band. Consider the
-    // SASL_NEED_PROXY flag if we want to reconsider this in the
-    // future.
-    callbacks[2].id = SASL_CB_AUTHNAME;
-    callbacks[2].proc = (int(*)()) &user;
-    callbacks[2].context = (void*) credential.principal().c_str();
-
-    callbacks[3].id = SASL_CB_PASS;
-    callbacks[3].proc = (int(*)()) &pass;
-    callbacks[3].context = (void*) secret;
-
-    callbacks[4].id = SASL_CB_LIST_END;
-    callbacks[4].proc = NULL;
-    callbacks[4].context = NULL;
-
-    int result = sasl_client_new(
-        "mesos",    // Registered name of service.
-        NULL,       // Server's FQDN.
-        NULL, NULL, // IP Address information strings.
-        callbacks,  // Callbacks supported only for this connection.
-        0,          // Security flags (security layers are enabled
-                    // using security properties, separately).
-        &connection);
-
-    if (result != SASL_OK) {
-      status = ERROR;
-      std::string error(sasl_errstring(result, NULL, NULL));
-      promise.fail("Failed to create client SASL connection: " + error);
-      return promise.future();
-    }
-
-    AuthenticateMessage message;
-    message.set_pid(client);
-    send(pid, message);
-
-    status = STARTING;
-
-    // Stop authenticating if nobody cares.
-    promise.future().onDiscard(defer(self(), &Self::discarded));
-
-    return promise.future();
-  }
-
-protected:
-  virtual void initialize()
-  {
-    // Anticipate mechanisms and steps from the server.
-    install<AuthenticationMechanismsMessage>(
-        &CRAMMD5AuthenticateeProcess::mechanisms,
-        &AuthenticationMechanismsMessage::mechanisms);
-
-    install<AuthenticationStepMessage>(
-        &CRAMMD5AuthenticateeProcess::step,
-        &AuthenticationStepMessage::data);
-
-    install<AuthenticationCompletedMessage>(
-        &CRAMMD5AuthenticateeProcess::completed);
-
-    install<AuthenticationFailedMessage>(
-        &CRAMMD5AuthenticateeProcess::failed);
-
-    install<AuthenticationErrorMessage>(
-        &CRAMMD5AuthenticateeProcess::error,
-        &AuthenticationErrorMessage::error);
-  }
-
-  void mechanisms(const std::vector<std::string>& mechanisms)
-  {
-    if (status != STARTING) {
-      status = ERROR;
-      promise.fail("Unexpected authentication 'mechanisms' received");
-      return;
-    }
-
-    // TODO(benh): Store 'from' in order to ensure we only communicate
-    // with the same Authenticator.
-
-    LOG(INFO) << "Received SASL authentication mechanisms: "
-              << strings::join(",", mechanisms);
-
-    sasl_interact_t* interact = NULL;
-    const char* output = NULL;
-    unsigned length = 0;
-    const char* mechanism = NULL;
-
-    int result = sasl_client_start(
-        connection,
-        strings::join(" ", mechanisms).c_str(),
-        &interact,     // Set if an interaction is needed.
-        &output,       // The output string (to send to server).
-        &length,       // The length of the output string.
-        &mechanism);   // The chosen mechanism.
-
-    CHECK_NE(SASL_INTERACT, result)
-      << "Not expecting an interaction (ID: " << interact->id << ")";
-
-    if (result != SASL_OK && result != SASL_CONTINUE) {
-      std::string error(sasl_errdetail(connection));
-      status = ERROR;
-      promise.fail("Failed to start the SASL client: " + error);
-      return;
-    }
-
-    LOG(INFO) << "Attempting to authenticate with mechanism '"
-              << mechanism << "'";
-
-    AuthenticationStartMessage message;
-    message.set_mechanism(mechanism);
-    message.set_data(output, length);
-
-    reply(message);
-
-    status = STEPPING;
-  }
-
-  void step(const std::string& data)
-  {
-    if (status != STEPPING) {
-      status = ERROR;
-      promise.fail("Unexpected authentication 'step' received");
-      return;
-    }
-
-    LOG(INFO) << "Received SASL authentication step";
-
-    sasl_interact_t* interact = NULL;
-    const char* output = NULL;
-    unsigned length = 0;
-
-    int result = sasl_client_step(
-        connection,
-        data.length() == 0 ? NULL : data.data(),
-        data.length(),
-        &interact,
-        &output,
-        &length);
-
-    CHECK_NE(SASL_INTERACT, result)
-      << "Not expecting an interaction (ID: " << interact->id << ")";
-
-    if (result == SASL_OK || result == SASL_CONTINUE) {
-      // We don't start the client with SASL_SUCCESS_DATA so we may
-      // need to send one more "empty" message to the server.
-      AuthenticationStepMessage message;
-      if (output != NULL && length > 0) {
-        message.set_data(output, length);
-      }
-      reply(message);
-    } else {
-      status = ERROR;
-      std::string error(sasl_errdetail(connection));
-      promise.fail("Failed to perform authentication step: " + error);
-    }
-  }
-
-  void completed()
-  {
-    if (status != STEPPING) {
-      status = ERROR;
-      promise.fail("Unexpected authentication 'completed' received");
-      return;
-    }
-
-    LOG(INFO) << "Authentication success";
-
-    status = COMPLETED;
-    promise.set(true);
-  }
-
-  void failed()
-  {
-    status = FAILED;
-    promise.set(false);
-  }
-
-  void error(const std::string& error)
-  {
-    status = ERROR;
-    promise.fail("Authentication error: " + error);
-  }
-
-  void discarded()
-  {
-    status = DISCARDED;
-    promise.fail("Authentication discarded");
-  }
-
-private:
-  static int user(
-      void* context,
-      int id,
-      const char** result,
-      unsigned* length)
-  {
-    CHECK(SASL_CB_USER == id || SASL_CB_AUTHNAME == id);
-    *result = static_cast<const char*>(context);
-    if (length != NULL) {
-      *length = strlen(*result);
-    }
-    return SASL_OK;
-  }
-
-  static int pass(
-      sasl_conn_t* connection,
-      void* context,
-      int id,
-      sasl_secret_t** secret)
-  {
-    CHECK_EQ(SASL_CB_PASS, id);
-    *secret = static_cast<sasl_secret_t*>(context);
-    return SASL_OK;
-  }
-
-  const Credential credential;
-
-  // PID of the client that needs to be authenticated.
-  const process::UPID client;
-
-  sasl_secret_t* secret;
-
-  sasl_callback_t callbacks[5];
-
-  enum {
-    READY,
-    STARTING,
-    STEPPING,
-    COMPLETED,
-    FAILED,
-    ERROR,
-    DISCARDED
-  } status;
-
-  sasl_conn_t* connection;
-
-  process::Promise<bool> promise;
-};
-
-
-inline Try<Authenticatee*> CRAMMD5Authenticatee::create()
-{
-  return new CRAMMD5Authenticatee();
-}
-
-
-inline CRAMMD5Authenticatee::CRAMMD5Authenticatee() : process(NULL) {}
-
-
-inline CRAMMD5Authenticatee::~CRAMMD5Authenticatee()
-{
-  if (process != NULL) {
-    process::terminate(process);
-    process::wait(process);
-    delete process;
-  }
-}
-
-
-inline process::Future<bool> CRAMMD5Authenticatee::authenticate(
-  const process::UPID& pid,
-  const process::UPID& client,
-  const mesos::Credential& credential)
-{
-  CHECK(process == NULL);
-  process = new CRAMMD5AuthenticateeProcess(credential, client);
-  process::spawn(process);
-
-  return process::dispatch(
-      process, &CRAMMD5AuthenticateeProcess::authenticate, pid);
-}
-
 } // namespace cram_md5 {
 } // namespace internal {
 } // namespace mesos {


[3/4] mesos git commit: Fixed authentication failure triggered slave crash.

Posted by me...@apache.org.
Fixed authentication failure triggered slave crash.

Review: https://reviews.apache.org/r/31838


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

Branch: refs/heads/master
Commit: 6d9f9b7ac6c690ed3a1690a108f95577375d6f68
Parents: b3a3d53
Author: Till Toenshoff <to...@me.com>
Authored: Sun Jun 21 08:28:14 2015 +0000
Committer: Adam B <ad...@mesosphere.io>
Committed: Sun Jun 21 08:32:50 2015 +0000

----------------------------------------------------------------------
 src/slave/slave.cpp | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/6d9f9b7a/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index e73913b..08ff50b 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -601,12 +601,14 @@ void Slave::shutdown(const UPID& from, const string& message)
   if (from) {
     LOG(INFO) << "Slave asked to shut down by " << from
               << (message.empty() ? "" : " because '" + message + "'");
-  } else {
+  } else if (info.has_id()) {
     LOG(INFO) << message << "; unregistering and shutting down";
 
     UnregisterSlaveMessage message_;
     message_.mutable_slave_id()->MergeFrom(info.id());
     send(master.get(), message_);
+  } else {
+    LOG(INFO) << message << "; shutting down";
   }
 
   state = TERMINATING;


[4/4] mesos git commit: Updated slave to exit on authentication refusal.

Posted by me...@apache.org.
Updated slave to exit on authentication refusal.

Review: https://reviews.apache.org/r/35583


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

Branch: refs/heads/master
Commit: effc3636df64af42c4d1af36014bc5e01408458a
Parents: 6d9f9b7
Author: Till Toenshoff <to...@me.com>
Authored: Sun Jun 21 08:25:49 2015 +0000
Committer: Adam B <ad...@mesosphere.io>
Committed: Sun Jun 21 08:32:50 2015 +0000

----------------------------------------------------------------------
 src/slave/slave.cpp | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/effc3636/src/slave/slave.cpp
----------------------------------------------------------------------
diff --git a/src/slave/slave.cpp b/src/slave/slave.cpp
index 08ff50b..40c0c33 100644
--- a/src/slave/slave.cpp
+++ b/src/slave/slave.cpp
@@ -803,9 +803,9 @@ void Slave::_authenticate()
   }
 
   if (!future.get()) {
-    LOG(ERROR) << "Master " << master.get() << " refused authentication";
-    shutdown(UPID(), "Master refused authentication");
-    return;
+    // For refused authentication, we exit instead of doing a shutdown
+    // to keep possibly active executors running.
+    EXIT(1) << "Master " << master.get() << " refused authentication";
   }
 
   LOG(INFO) << "Successfully authenticated with master " << master.get();


[2/4] mesos git commit: Added secret check to CRAM-MD5 authenticatee.

Posted by me...@apache.org.
Added secret check to CRAM-MD5 authenticatee.

Updating authenticatee to check for secret within credential. Adding a
test verifying immediate authenticatee failure when secret is missing.

Review: https://reviews.apache.org/r/33057


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

Branch: refs/heads/master
Commit: b3a3d53377304de361393de6af9ab7e239a4b1bb
Parents: 362bf17
Author: Till Toenshoff <to...@me.com>
Authored: Sun Jun 21 08:23:37 2015 +0000
Committer: Adam B <ad...@mesosphere.io>
Committed: Sun Jun 21 08:24:32 2015 +0000

----------------------------------------------------------------------
 src/authentication/cram_md5/authenticatee.cpp |  6 ++++++
 src/tests/cram_md5_authentication_tests.cpp   | 18 ++++++++++++++++++
 2 files changed, 24 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/b3a3d533/src/authentication/cram_md5/authenticatee.cpp
----------------------------------------------------------------------
diff --git a/src/authentication/cram_md5/authenticatee.cpp b/src/authentication/cram_md5/authenticatee.cpp
index 7143ef6..63ae17e 100644
--- a/src/authentication/cram_md5/authenticatee.cpp
+++ b/src/authentication/cram_md5/authenticatee.cpp
@@ -388,6 +388,12 @@ Future<bool> CRAMMD5Authenticatee::authenticate(
   const UPID& client,
   const mesos::Credential& credential)
 {
+  if (!credential.has_secret()) {
+    LOG(WARNING) << "Authentication failed; secret needed by CRAM-MD5 "
+                 << "authenticatee";
+    return false;
+  }
+
   CHECK(process == NULL);
   process = new CRAMMD5AuthenticateeProcess(credential, client);
   spawn(process);

http://git-wip-us.apache.org/repos/asf/mesos/blob/b3a3d533/src/tests/cram_md5_authentication_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/cram_md5_authentication_tests.cpp b/src/tests/cram_md5_authentication_tests.cpp
index 9923023..9d15b55 100644
--- a/src/tests/cram_md5_authentication_tests.cpp
+++ b/src/tests/cram_md5_authentication_tests.cpp
@@ -270,6 +270,24 @@ TYPED_TEST(CRAMMD5Authentication, AuthenticatorDestructionRace)
   delete authenticatee.get();
 }
 
+
+// This test verifies that a missing secret fails the authenticatee.
+TYPED_TEST(CRAMMD5Authentication, AuthenticateeSecretMissing)
+{
+  Credential credential;
+  credential.set_principal("benh");
+
+  Try<Authenticatee*> authenticatee = TypeParam::TypeAuthenticatee::create();
+  CHECK_SOME(authenticatee);
+
+  Future<bool> future =
+    authenticatee.get()->authenticate(UPID(), UPID(), credential);
+
+  AWAIT_EQ(false, future);
+
+  delete authenticatee.get();
+}
+
 } // namespace cram_md5 {
 } // namespace internal {
 } // namespace mesos {