Browse Source

HDFS-10450b: libhdfs++: Add Cyrus SASL support (engines). Contributed by Don Davis.

Bob Hansen 8 years ago
parent
commit
d22e4b2eb7

+ 413 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/cyrus_sasl_engine.cc

@@ -0,0 +1,413 @@
+/**
+ * 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 <sys/types.h>
+#include "sasl/sasl.h"
+#include "sasl/saslutil.h"
+#include <string.h>
+#include <string>
+#include <sstream>
+#include <unistd.h>    // getpass() ( deprecated)
+
+#include "common/logging.h"
+
+#include       "sasl_engine.h"
+#include "cyrus_sasl_engine.h"
+
+namespace hdfs {
+
+
+// Forward decls of sasl callback functions
+typedef int (*sasl_callback_ft)(void);
+int get_name(void        *context,
+             int          id,
+             const char **result,
+             unsigned    *len);
+
+int getrealm(void *context,
+             int   id,
+             const char **availrealms,
+             const char **result);
+
+// This should be constructed once per process, and destroyed once per process
+
+class CyrusPerProcessData
+{
+public:
+  static Status Init(); // Can be called many times
+private:
+  CyrusPerProcessData();
+  ~CyrusPerProcessData();
+  Status init_status_;
+
+  static CyrusPerProcessData & GetInstance();
+};
+
+
+/*****************************************************************************
+ *              CYRUS UTILITY FUNCTIONS
+ */
+
+// Cyrus-specific error messages:
+// errStr() is the non-method version, to
+//          be called by utility routines.
+std::string errStr( int rc) {
+  switch (rc) {
+    case SASL_NOTINIT:  /* -12 */ return "SASL library not initialized";
+    case SASL_WRONGMECH:/* -11 */ return "mechanism doesn't support requested feature";
+    case SASL_BADSERV:  /* -10 */ return "server failed mutual authentication step";
+    case SASL_BADMAC:   /*  -9 */ return "integrity check failed";
+    case SASL_TRYAGAIN: /*  -8 */ return "transient failure (e.g., weak key)";
+    case SASL_BADPARAM: /*  -7 */ return "invalid parameter supplied";
+    case SASL_NOTDONE:  /*  -6 */ return "can't request info until later in exchange";
+    case SASL_BADPROT:  /*  -5 */ return "bad protocol / cancel";
+    case SASL_NOMECH:   /*  -4 */ return "mechanism not supported";
+    case SASL_BUFOVER:  /*  -3 */ return "overflowed buffer";
+    case SASL_NOMEM:    /*  -2 */ return "memory shortage failure";
+    case SASL_FAIL:     /*  -1 */ return "generic failure";
+    case SASL_OK:       /*   0 */ return "successful result";
+    case SASL_CONTINUE: /*   1 */ return "another step is needed in authentication";
+    case SASL_INTERACT: /*   2 */ return "needs user interaction";
+    default:                      return "unknown error";
+  } // switch(rc)
+} // errStr()
+
+Status make_status(int rc) {
+  if (rc != SASL_OK &&
+      rc != SASL_CONTINUE &&
+      rc != SASL_INTERACT) {
+     return Status::Error(errStr(rc).c_str());
+  }
+  return Status::OK();
+}
+
+// SaslError() method:  Use this when a method needs
+//                   to update the engine's state.
+Status CySaslEngine::SaslError( int rc) {
+  Status status = make_status(rc);
+  if (!status.ok())
+      state_ = kErrorState;
+
+  return status;
+}
+
+
+/*****************************************************************************
+*                     Cyrus SASL ENGINE
+*/
+
+  CySaslEngine::CySaslEngine() : SaslEngine(), conn_(nullptr)
+  {
+    // Create an array of callbacks that embed a pointer to this
+    //   so we can call methods of the engine
+    per_connection_callbacks_ = {
+      { SASL_CB_USER,     (sasl_callback_ft) & get_name, this}, // userid for authZ
+      { SASL_CB_AUTHNAME, (sasl_callback_ft) & get_name, this}, // authid for authT
+      { SASL_CB_GETREALM, (sasl_callback_ft) & getrealm, this}, // krb/gssapi realm
+      //  { SASL_CB_PASS,        (sasl_callback_ft)&getsecret,  this
+      { SASL_CB_LIST_END, (sasl_callback_ft) NULL, NULL}
+    };
+  }
+
+CySaslEngine::~CySaslEngine()
+{
+  if (conn_) {
+      sasl_dispose( &conn_); // undo sasl_client_new()
+  }
+} // destructor
+
+// initialize some cyrus sasl context stuff:
+
+Status CySaslEngine::InitCyrusSasl()
+{
+  int rc = SASL_OK;
+
+  // set up some callbacks once per process:
+  Status init_status = CyrusPerProcessData::Init();
+  if (!init_status.ok())
+    return init_status;
+
+  // Initialize the sasl_li  brary with per-connection configuration:
+  const char * fqdn = chosen_mech_.serverid.c_str();
+  const char * proto = chosen_mech_.protocol.c_str();
+
+  rc = sasl_client_new(proto, fqdn, NULL, NULL, &per_connection_callbacks_[0], 0, &conn_);
+  if (rc != SASL_OK) return SaslError(rc);
+
+  return Status::OK();
+} // cysasl_new()
+
+// start() method:  Ask the Sasl ibrary, "How do we
+//                  ask the hdfs server for service?
+std::pair<Status, std::string>
+CySaslEngine::Start()
+{
+  int    rc;
+  Status status;
+
+  if (state_ != kUnstarted)
+    LOG_WARN(kRPC, << "CySaslEngine::start() when state is " << state_);
+
+  status = InitCyrusSasl();
+
+  if ( !status.ok()) {
+    state_ = kErrorState;
+    return std::make_pair( status, "");
+  }
+
+  sasl_interact_t * client_interact = NULL;
+  char            * buf;
+  unsigned int      buflen;
+  const char      * chosen_mech;
+  std::string       token;
+
+  rc = sasl_client_start(conn_, chosen_mech_.mechanism.c_str(), &client_interact,
+            (const char **) &buf, &buflen, &chosen_mech);
+
+  switch (rc) {
+  case SASL_OK:        state_ = kSuccess;
+                       break;
+  case SASL_CONTINUE:  state_ = kWaitingForData;
+                       break;
+  default:             state_ = kFailure;
+                       return std::make_pair( SaslError(rc), "");
+                       break;
+  } // switch( rc)
+
+  // Cyrus will free this buffer when the connection is shut down
+  token = std::string( buf, buflen);
+  return std::make_pair( Status::OK(), token);
+} // start() method
+
+std::pair<Status, std::string> CySaslEngine::Step(const std::string data)
+{
+  char            * output = NULL;
+  unsigned int      outlen = 0;
+  sasl_interact_t * client_interact = NULL;
+
+  if (state_ != kWaitingForData)
+    LOG_WARN(kRPC, << "CySaslEngine::step when state is " << state_);
+
+  int rc = sasl_client_step(conn_, data.c_str(), data.size(), &client_interact,
+                        (const char **) &output, &outlen);
+
+  // right now, state_ == kWaitingForData,
+  // so update  state_, to reflect _step()'s result:
+  switch (rc) {
+  case SASL_OK:        state_ = kSuccess;        break;
+  case SASL_CONTINUE:  state_ = kWaitingForData; break;
+  default:             state_ = kFailure;
+               return std::make_pair(SaslError(rc), "");
+               break;
+  } // switch( rc)
+  return std::make_pair(Status::OK(), std::string( output,outlen));
+} // step() method
+
+Status CySaslEngine::Finish()
+{
+  if (state_ != kSuccess && state_ != kFailure && state_ != kErrorState )
+    LOG_WARN(kRPC, << "CySaslEngine::finish when state is " << state_);
+
+  if (conn_ != nullptr) {
+      sasl_dispose( &conn_);
+      conn_ = NULL;
+  }
+
+  return Status::OK();
+}
+
+//////////////////////////////////////////////////
+// Internal callbacks, for sasl_init_client().  //
+// Mostly lifted from cyrus' sample_client.c .  //
+//////////////////////////////////////////////////
+
+static int
+sasl_my_log(void *context __attribute__((unused)),
+        int   priority,
+        const char *message)
+{
+  if (! message)
+    return SASL_BADPARAM;
+
+  //TODO: get client, connection ID in here
+  switch (priority) {
+  case SASL_LOG_NONE: return SASL_OK; // no-op
+  case SASL_LOG_ERR:  // fall through to FAIL
+  case SASL_LOG_FAIL:
+    LOG_ERROR(kRPC, << "SASL Error: " << message);
+    break;
+  case SASL_LOG_WARN:
+    LOG_ERROR(kRPC, << message);
+    break;
+  case SASL_LOG_NOTE:
+    LOG_INFO(kRPC, << message);
+    break;
+  case SASL_LOG_DEBUG:
+    LOG_DEBUG(kRPC, << message);
+    break;
+  case SASL_LOG_TRACE:
+    LOG_TRACE(kRPC, << message);
+    break;
+  case SASL_LOG_PASS: return SASL_OK; // don't log password-info
+  default:
+    LOG_WARN(kRPC, << "Unknown SASL log level(" << priority << "): " << message);
+    break;
+  }
+
+  return SASL_OK;
+} // sasl_my_log() callback
+
+static int
+sasl_getopt(void *context, const char *plugin_name,
+               const char *option,
+               const char **result, unsigned *len)
+{
+  if (plugin_name) {
+    LOG_WARN(kRPC, << "CySaslEngine: Unexpected plugin_name " << plugin_name);
+    return SASL_OK;
+  }                   //   123456789012345678
+  if (! strncmp( option,  "canon_user_plugin", 18)) {
+    // TODO: maybe write a canon_user_plugin to do user-to-principal mapping
+    *result = "INTERNAL";
+    if (len) *len = strlen( *result);
+    return SASL_OK;
+  }                   //  12345678901234567
+  if (! strncmp( option, "client_mech_list", 17)) {
+    *result = "GSSAPI";
+    if (len) *len = strlen( *result);
+    return SASL_OK;
+  }
+
+  (void) context;   // unused
+  return SASL_OK; }
+
+#define PLUGINDIR "/usr/local/lib/sasl2" // where the mechanisms are
+
+static int
+get_path(void *context, const char ** path)
+{
+  const char *searchpath = (const char *) context;
+
+  if (! path)
+    return SASL_BADPARAM;
+
+  // TODO: check the SASL_PATH environment, or will Cyrus pass that in in the context?
+  if (searchpath) {
+      *path = searchpath;
+  } else {
+      *path = PLUGINDIR;
+  }
+
+  return SASL_OK;
+} // getpath() callback
+
+int get_name(void *context,
+             int id,
+             const char **result,
+             unsigned *len)
+{
+  const CySaslEngine * pThis = (const CySaslEngine *) context;
+
+  if (!result)
+    return SASL_BADPARAM;
+
+  switch (id) {
+    case SASL_CB_AUTHNAME:
+      if (!pThis->id_)
+        break;
+      if (len)
+        *len = pThis->id_->size();
+      *result = pThis->id_->c_str();
+      break;
+    case SASL_CB_USER:
+      if (!pThis->principal_)
+        break;
+      if (len)
+        *len = pThis->principal_->size();
+      *result = pThis->principal_->c_str();
+      break;
+    case SASL_CB_LANGUAGE:
+      *result = NULL;
+      if (len)
+        *len = 0;
+      break;
+    default:
+      return SASL_BADPARAM;
+  }
+
+  LOG_DEBUG(kRPC, << "Cyrus::get_name: returning " << *result);
+
+  return SASL_OK;
+} // simple() callback
+
+int getrealm(void *context,
+             int id,
+             const char **availrealms,
+             const char **result)
+{
+  (void)availrealms; // unused
+  const CySaslEngine * pThis = (const CySaslEngine *) context;
+
+  if (!result)
+    return SASL_BADPARAM;
+
+  if (id != SASL_CB_GETREALM) return SASL_FAIL;
+  if (pThis->realm_)
+    *result = pThis->realm_->c_str();
+
+  return SASL_OK;
+} // getrealm() callback
+
+
+/*****************************************************************************
+*        CYRUS PER-PROCESS INITIALIZATION
+*/
+
+
+const sasl_callback_t per_process_callbacks[] = {
+    { SASL_CB_LOG, (sasl_callback_ft) & sasl_my_log, NULL},
+    { SASL_CB_GETOPT, (sasl_callback_ft) & sasl_getopt, NULL},
+    { SASL_CB_GETPATH, (sasl_callback_ft) & get_path, NULL}, // to find th mechanisms
+    { SASL_CB_LIST_END, (sasl_callback_ft) NULL, NULL}
+  }; // callbacks_ array
+
+CyrusPerProcessData::CyrusPerProcessData()
+{
+  int init_rc = sasl_client_init(per_process_callbacks);
+  init_status_ = make_status(init_rc);
+}
+
+CyrusPerProcessData::~CyrusPerProcessData()
+{
+  // Undo sasl_client_init())
+  sasl_done();
+}
+
+Status CyrusPerProcessData::Init()
+{
+  return GetInstance().init_status_;
+}
+
+CyrusPerProcessData & CyrusPerProcessData::GetInstance()
+{
+  static CyrusPerProcessData per_process_data;
+  return per_process_data;
+}
+
+
+} // namespace hdfs

+ 50 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/cyrus_sasl_engine.h

@@ -0,0 +1,50 @@
+/**
+ * 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.
+ */
+
+#ifndef LIB_RPC_CYRUS_SASLENGINE_H
+#define LIB_RPC_CYRUS_SASLENGINE_H
+
+#include "sasl/sasl.h"
+#include "sasl_engine.h"
+
+namespace hdfs
+{
+
+class CySaslEngine : public SaslEngine
+{
+public:
+  CySaslEngine();
+  virtual ~CySaslEngine();
+
+  virtual std::pair<Status, std::string> Start();
+  virtual std::pair<Status, std::string> Step(const std::string data);
+  virtual Status Finish();
+private:
+  Status InitCyrusSasl();
+  Status SaslError(int rc);
+
+  friend int get_name(void *, int, const char **, unsigned *);
+  friend int getrealm(void *, int, const char **availrealms, const char **);
+
+  sasl_conn_t * conn_;
+  std::vector<sasl_callback_t> per_connection_callbacks_;
+}; //class CySaslEngine
+
+} // namespace hdfs
+
+#endif /* LIB_RPC_CYRUS_SASLENGINE_H */

+ 182 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/gsasl_engine.cc

@@ -0,0 +1,182 @@
+/**
+ * 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 <sstream>
+#include <gsasl.h>
+#include  "sasl_engine.h"
+#include "gsasl_engine.h"
+#include "common/logging.h"
+
+namespace hdfs {
+
+/*****************************************************************************
+ *               GSASL UTILITY FUNCTIONS
+ */
+
+static Status rc_to_status(int rc)
+{
+  if (rc == GSASL_OK) {
+    return Status::OK();
+  } else {
+    std::ostringstream ss;
+    ss << "Cannot initialize client (" << rc << "): " << gsasl_strerror(rc);
+    return Status::Error(ss.str().c_str());
+  }
+}
+
+static
+std::pair<Status, std::string> base64_encode(const std::string & in) {
+    char * temp;
+    size_t len;
+    std::string retval;
+    (void)base64_encode;
+
+    int rc = gsasl_base64_to(in.c_str(), in.size(), &temp, &len);
+
+    if (rc != GSASL_OK) {
+      return std::make_pair(rc_to_status(rc), "");
+    }
+
+    if (temp) {
+        retval = temp;
+        free(temp);
+    }
+
+    if (!temp || retval.length() != len) {
+        return std::make_pair(Status::Error("SaslEngine: Failed to encode string to base64"), "");
+    }
+
+    return std::make_pair(Status::OK(), retval);
+}
+
+/*****************************************************************************
+ *                     GSASL ENGINE
+ */
+
+GSaslEngine::~GSaslEngine()
+{
+  if (session_ != nullptr) {
+      gsasl_finish(session_);
+  }
+
+  if (ctx_ != nullptr) {
+      gsasl_done(ctx_);
+  }
+}
+
+Status GSaslEngine::gsasl_new() {
+   int status = GSASL_OK;
+
+   if (ctx_) return Status::OK();
+
+   status = gsasl_init( & ctx_);
+
+   switch ( status) {
+   case GSASL_OK:
+      return Status::OK();
+   case GSASL_MALLOC_ERROR:
+      LOG_WARN(kRPC, <<   "GSaslEngine: Out of memory.");
+      return Status::Error("SaslEngine: Out of memory.");
+   default:
+      LOG_WARN(kRPC, <<   "GSaslEngine: Unexpected error." << status);
+      return Status::Error("SaslEngine: Unexpected error.");
+   }
+} // gsasl_new()
+
+std::pair<Status, std::string>
+GSaslEngine::Start()
+{
+  int    rc;
+  Status status;
+
+  this->gsasl_new();
+
+  /* Create new authentication session. */
+  rc = gsasl_client_start(ctx_, chosen_mech_.mechanism.c_str(), &session_);
+  if (rc != GSASL_OK) {
+    state_ = kErrorState;
+    return std::make_pair( rc_to_status( rc), std::string(""));
+  }
+  init_kerberos();
+
+  state_ = kWaitingForData;
+
+  // get from the sasl library the initial token
+  // that we'll send to the application server:
+  return this->Step( chosen_mech_.challenge.c_str());
+} // start() method
+
+Status GSaslEngine::init_kerberos() {
+
+  //TODO: check that we have a principal
+
+  gsasl_property_set(session_, GSASL_AUTHID, principal_.value().c_str());
+  gsasl_property_set(session_, GSASL_HOSTNAME,   chosen_mech_.serverid.c_str());
+  gsasl_property_set(session_, GSASL_SERVICE,    chosen_mech_.protocol.c_str());
+  return Status::OK();
+  }
+
+std::pair<Status, std::string> GSaslEngine::Step(const std::string data) {
+  if (state_ != kWaitingForData)
+    LOG_WARN(kRPC, << "GSaslEngine::step when state is " << state_);
+
+  char * output = NULL;
+  size_t outputSize;
+  int rc = gsasl_step(session_, data.c_str(), data.size(), &output,
+          &outputSize);
+
+  if (rc == GSASL_NEEDS_MORE || rc == GSASL_OK) {
+    std::string retval(output, output ? outputSize : 0);
+    if (output) {
+      free(output);
+    }
+
+    if (rc == GSASL_OK) {
+      state_ = kSuccess;
+    }
+
+    return std::make_pair(Status::OK(), retval);
+  }
+  else {
+    if (output) {
+      free(output);
+    }
+    state_ = kFailure;
+    return std::make_pair(rc_to_status(rc), "");
+  }
+}
+
+Status GSaslEngine::Finish()
+{
+  if (state_ != kSuccess && state_ != kFailure && state_ != kErrorState )
+    LOG_WARN(kRPC, << "GSaslEngine::finish when state is " << state_);
+
+  if (session_ != nullptr) {
+      gsasl_finish(session_);
+      session_ = NULL;
+  }
+
+  if (ctx_ != nullptr) {
+      gsasl_done(ctx_);
+      ctx_ = nullptr;
+  }
+
+  return Status::OK();
+} // finish() method
+
+} // namespace hdfs

+ 47 - 0
hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/rpc/gsasl_engine.h

@@ -0,0 +1,47 @@
+/**
+ * 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.
+ */
+
+#ifndef LIB_RPC_GSASLENGINE_H
+#define LIB_RPC_GSASLENGINE_H
+
+#include  <gsasl.h>
+
+#include "sasl_engine.h"
+
+namespace hdfs {
+
+class GSaslEngine : public SaslEngine
+{
+public:
+  GSaslEngine() : SaslEngine(), ctx_(nullptr), session_(nullptr) {}
+  virtual ~GSaslEngine();
+
+  virtual std::pair<Status,std::string>  Start();
+  virtual std::pair<Status,std::string> Step(const std::string data);
+  virtual Status Finish();
+private:
+  Status gsasl_new();
+  Gsasl * ctx_;
+  Gsasl_session * session_;
+
+  Status init_kerberos();
+};
+
+} // namespace hdfs
+
+#endif /* LIB_RPC_GSASLENGINE_H */