]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
libceph API change
authorColin Patrick McCabe <cmccabe@alumni.cmu.edu>
Wed, 20 Apr 2011 18:45:31 +0000 (11:45 -0700)
committerColin Patrick McCabe <cmccabe@alumni.cmu.edu>
Mon, 25 Apr 2011 18:05:43 +0000 (11:05 -0700)
Signed-off-by: Colin McCabe <colin.mccabe@dreamhost.com>
src/client/Client.cc
src/client/libceph.h
src/client/testceph.cc
src/include/rados/librados.h
src/libceph.cc

index 95f102ad7ef767280f692daad675a68c8530af71..ff4dd4536859d2ed7e5ac743cc0357372cd4e08e 100644 (file)
@@ -4479,7 +4479,6 @@ int Client::getdir(const char *relpath, list<string>& contents)
 
 
 /****** file i/o **********/
-
 int Client::open(const char *relpath, int flags, mode_t mode) 
 {
   dout(3) << "open enter(" << relpath << ", " << flags << "," << mode << ") = " << dendl;
index 9350593a473769e81b228b27557e535878fc04bc..a35d5da4c17b0edb9b5e27bb89166aac0a3fb8f9 100644 (file)
@@ -1,20 +1,24 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2009-2011 New Dream Network
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation.  See file COPYING.
+ *
+ */
+
 #ifndef CEPH_LIB_H
 #define CEPH_LIB_H
-#include <netinet/in.h>
-#include <sys/statvfs.h>
+
 #include <utime.h>
-#include <sys/stat.h>
-#include <stdbool.h>
 #include <sys/types.h>
-#include <unistd.h>
 #include <dirent.h>
 
-#ifdef __cplusplus
-# define CEPH_DEFAULT_ARG(v) =v
-#else
-# define CEPH_DEFAULT_ARG(v)
-#endif
-
 struct stat_precise {
   ino_t st_ino;
   dev_t st_dev;
@@ -38,87 +42,116 @@ struct stat_precise {
 extern "C" {
 #endif
 
+struct ceph_cluster_t;
+
 const char *ceph_version(int *major, int *minor, int *patch);
 
-int ceph_initialize(int argc, const char **argv);
-void ceph_deinitialize();
+/* initialization */
+int ceph_create(ceph_cluster_t **cluster, const char * const id);
+
+/* initialization with an existing configuration */
+int ceph_create_with_config(ceph_cluster_t **cluster, struct md_config_t *conf);
+
+/* Connect to the cluster */
+int ceph_connect(ceph_cluster_t *cluster);
+
+/* Destroy the cluster instance */
+void ceph_shutdown(ceph_cluster_t *cluster);
+
+/* Config
+ *
+ * Functions for manipulating the Ceph configuration at runtime.
+ */
+int ceph_conf_read_file(ceph_cluster_t *cluster, const char *path);
+
+void ceph_conf_parse_argv(ceph_cluster_t *cluster, int argc, const char **argv);
 
 /* Sets a configuration value from a string.
  * Returns 0 on success, error code otherwise. */
-int ceph_conf_set(const char *option, const char *value);
+int ceph_conf_set(ceph_cluster_t *cluster, const char *option, const char *value);
 
 /* Returns a configuration value as a string.
  * If len is positive, that is the maximum number of bytes we'll write into the
  * buffer. If len == -1, we'll call malloc() and set *buf.
  * Returns 0 on success, error code otherwise. Returns ENAMETOOLONG if the
  * buffer is too short. */
-int ceph_conf_get(const char *option, char *buf, size_t len);
-
-int ceph_mount();
-int ceph_umount();
-
-int ceph_statfs(const char *path, struct statvfs *stbuf);
-int ceph_get_local_osd();
-
-int ceph_getcwd(char *buf, int buflen);
-int ceph_chdir(const char *s);
-
-int ceph_opendir(const char *name, DIR **dirpp);
-int ceph_closedir(DIR *dirp);
-int ceph_readdir_r(DIR *dirp, struct dirent *de);
-int ceph_readdirplus_r(DIR *dirp, struct dirent *de, struct stat *st, int *stmask);
-int ceph_getdents(DIR *dirp, char *name, int buflen);
-int ceph_getdnames(DIR *dirp, char *name, int buflen);
-void ceph_rewinddir(DIR *dirp); 
-loff_t ceph_telldir(DIR *dirp);
-void ceph_seekdir(DIR *dirp, loff_t offset);
-
-int ceph_link (const char *existing, const char *newname);
-int ceph_unlink (const char *path);
-int ceph_rename(const char *from, const char *to);
+int ceph_conf_get(ceph_cluster_t *cluster, const char *option, char *buf, size_t len);
+
+int ceph_mount(ceph_cluster_t *cluster, const char *root);
+int ceph_umount(ceph_cluster_t *cluster);
+
+int ceph_statfs(ceph_cluster_t *cluster, const char *path, struct statvfs *stbuf);
+int ceph_get_local_osd(ceph_cluster_t *cluster);
+
+/* Get the current working directory.
+ *
+ * The pointer you get back from this function will continue to be valid until
+ * the *next* call you make to ceph_getcwd, at which point it will be invalidated.
+ */
+const char* ceph_getcwd(ceph_cluster_t *cluster);
+
+int ceph_chdir(ceph_cluster_t *cluster, const char *s);
+
+int ceph_opendir(ceph_cluster_t *cluster, const char *name, DIR **dirpp);
+int ceph_closedir(ceph_cluster_t *cluster, DIR *dirp);
+int ceph_readdir_r(ceph_cluster_t *cluster, DIR *dirp, struct dirent *de);
+int ceph_readdirplus_r(ceph_cluster_t *cluster, DIR *dirp, struct dirent *de,
+                      struct stat *st, int *stmask);
+int ceph_getdents(ceph_cluster_t *cluster, DIR *dirp, char *name, int buflen);
+int ceph_getdnames(ceph_cluster_t *cluster, DIR *dirp, char *name, int buflen);
+void ceph_rewinddir(ceph_cluster_t *cluster, DIR *dirp);
+loff_t ceph_telldir(ceph_cluster_t *cluster, DIR *dirp);
+void ceph_seekdir(ceph_cluster_t *cluster, DIR *dirp, loff_t offset);
+
+int ceph_link(ceph_cluster_t *cluster, const char *existing, const char *newname);
+int ceph_unlink(ceph_cluster_t *cluster, const char *path);
+int ceph_rename(ceph_cluster_t *cluster, const char *from, const char *to);
 
 // dirs
-int ceph_mkdir(const char *path, mode_t mode);
-int ceph_mkdirs(const char *path, mode_t mode);
-int ceph_rmdir(const char *path);
+int ceph_mkdir(ceph_cluster_t *cluster, const char *path, mode_t mode);
+int ceph_mkdirs(ceph_cluster_t *cluster, const char *path, mode_t mode);
+int ceph_rmdir(ceph_cluster_t *cluster, const char *path);
 
 // symlinks
-int ceph_readlink(const char *path, char *buf, loff_t size);
-int ceph_symlink(const char *existing, const char *newname);
+int ceph_readlink(ceph_cluster_t *cluster, const char *path, char *buf, loff_t size);
+int ceph_symlink(ceph_cluster_t *cluster, const char *existing, const char *newname);
 
 // inode stuff
-int ceph_lstat(const char *path, struct stat *stbuf);
-int ceph_lstat_precise(const char *path, struct stat_precise *stbuf);
+int ceph_lstat(ceph_cluster_t *cluster, const char *path, struct stat *stbuf);
+int ceph_lstat_precise(ceph_cluster_t *cluster, const char *path, struct stat_precise *stbuf);
 
-int ceph_setattr(const char *relpath, struct stat *attr, int mask);
-int ceph_setattr_precise (const char *relpath, struct stat_precise *stbuf, int mask);
-int ceph_chmod(const char *path, mode_t mode);
-int ceph_chown(const char *path, uid_t uid, gid_t gid);
-int ceph_utime(const char *path, struct utimbuf *buf);
-int ceph_truncate(const char *path, loff_t size);
+int ceph_setattr(ceph_cluster_t *cluster, const char *relpath, struct stat *attr, int mask);
+int ceph_setattr_precise (ceph_cluster_t *cluster, const char *relpath,
+                         struct stat_precise *stbuf, int mask);
+int ceph_chmod(ceph_cluster_t *cluster, const char *path, mode_t mode);
+int ceph_chown(ceph_cluster_t *cluster, const char *path, uid_t uid, gid_t gid);
+int ceph_utime(ceph_cluster_t *cluster, const char *path, struct utimbuf *buf);
+int ceph_truncate(ceph_cluster_t *cluster, const char *path, loff_t size);
 
 // file ops
-int ceph_mknod(const char *path, mode_t mode, dev_t rdev CEPH_DEFAULT_ARG(0));
-int ceph_open(const char *path, int flags, mode_t mode CEPH_DEFAULT_ARG(0));
-int ceph_close(int fd);
-loff_t ceph_lseek(int fd, loff_t offset, int whence);
-int ceph_read(int fd, char *buf, loff_t size, loff_t offset CEPH_DEFAULT_ARG(-1));
-int ceph_write(int fd, const char *buf, loff_t size, loff_t offset CEPH_DEFAULT_ARG(-1));
-int ceph_ftruncate(int fd, loff_t size);
-int ceph_fsync(int fd, bool syncdataonly);
-int ceph_fstat(int fd, struct stat *stbuf);
-
-int ceph_sync_fs();
-int ceph_get_file_stripe_unit(int fh);
-int ceph_get_file_replication(const char *path);
-int ceph_get_default_preferred_pg(int fd);
-int ceph_get_file_stripe_address(int fd, loff_t offset, char *buf, int buflen);
-int ceph_set_default_file_stripe_unit(int stripe);
-int ceph_set_default_file_stripe_count(int count);
-int ceph_set_default_object_size(int size);
-int ceph_set_default_file_replication(int replication);
-int ceph_set_default_preferred_pg(int pg);
-int ceph_localize_reads(int val);
+int ceph_mknod(ceph_cluster_t *cluster, const char *path, mode_t mode, dev_t rdev);
+int ceph_open(ceph_cluster_t *cluster, const char *path, int flags, mode_t mode);
+int ceph_close(ceph_cluster_t *cluster, int fd);
+loff_t ceph_lseek(ceph_cluster_t *cluster, int fd, loff_t offset, int whence);
+int ceph_read(ceph_cluster_t *cluster, int fd, char *buf, loff_t size, loff_t offset);
+int ceph_write(ceph_cluster_t *cluster, int fd, const char *buf, loff_t size,
+              loff_t offset);
+int ceph_ftruncate(ceph_cluster_t *cluster, int fd, loff_t size);
+int ceph_fsync(ceph_cluster_t *cluster, int fd, int syncdataonly);
+int ceph_fstat(ceph_cluster_t *cluster, int fd, struct stat *stbuf);
+
+int ceph_sync_fs(ceph_cluster_t *cluster);
+int ceph_get_file_stripe_unit(ceph_cluster_t *cluster, int fh);
+int ceph_get_file_replication(ceph_cluster_t *cluster, const char *path);
+int ceph_get_default_preferred_pg(ceph_cluster_t *cluster, int fd);
+int ceph_get_file_stripe_address(ceph_cluster_t *cluster, int fd,
+                                loff_t offset, char *buf, int buflen);
+int ceph_set_default_file_stripe_unit(ceph_cluster_t *cluster, int stripe);
+int ceph_set_default_file_stripe_count(ceph_cluster_t *cluster, int count);
+int ceph_set_default_object_size(ceph_cluster_t *cluster, int size);
+int ceph_set_default_file_replication(ceph_cluster_t *cluster, int replication);
+int ceph_set_default_preferred_pg(ceph_cluster_t *cluster, int pg);
+int ceph_localize_reads(ceph_cluster_t *cluster, int val);
 
 #ifdef __cplusplus
 }
index 2322a7f806148a811c5a465bbd5ba9234dd57d4b..2ab40ca223fab326f5eedfda7d2198cc0655a166 100644 (file)
 #include "libceph.h"
 #include <iostream>
 
-using namespace std;
+using std::cout;
+using std::cerr;
 
 int main(int argc, const char **argv)
 {
-  if (ceph_initialize(argc, argv) < 0) {
-    cerr << "error initializing\n" << std::endl;
-    return(1);
+  ceph_cluster_t *cluster;
+  int ret = ceph_create(&cluster, NULL);
+  if (ret) {
+    cerr << "ceph_create failed with error: " << ret << std::endl;
+    return 1;
   }
-  cout << "Successfully initialized Ceph!" << std::endl;
 
-  if(ceph_mount() < 0) {
-    cerr << "error mounting\n" << std::endl;
-    return(1);
+  ceph_conf_parse_argv(cluster, argc, argv);
+
+  char buf[128];
+  ret = ceph_conf_get(cluster, "log file", buf, sizeof(buf));
+  if (ret) {
+    cerr << "ceph_conf_get(\"log file\") failed with error " << ret << std::endl;
+  }
+  else {
+    cout << "log_file = \"" << buf << "\"" << std::endl;
+  }
+
+  ret = ceph_connect(cluster);
+  if (ret) {
+    cerr << "ceph_connect failed with error: " << ret << std::endl;
+    return 1;
+  }
+
+  ret = ceph_mount(cluster, NULL);
+  if (ret) {
+    cerr << "ceph_mount error: " << ret << std::endl;
+    return 1;
   }
   cout << "Successfully mounted Ceph!" << std::endl;
 
-  ceph_deinitialize();
-  cout << "Successfully deinitialized Ceph!" << std::endl;
+  ceph_shutdown(cluster);
 
   return 0;
 }
index 9fe328522fcf5a409dd0c7df416c140951c999ef..c55a0960b7b1de27dc5bdf839c421299f04d4151 100644 (file)
@@ -67,8 +67,6 @@ void rados_shutdown(rados_t cluster);
 /* Config
  *
  * Functions for manipulating the Ceph configuration at runtime.
- * After changing the Ceph configuration, you should call rados_conf_apply to
- * ensure that the changes have been applied.
  */
 int rados_conf_read_file(rados_t cluster, const char *path);
 
index a5f7bf56803cd03bb6e2582cfe275093bf327945..aeb50c93b1d0b9ae4ff2bd709cf8591e0b2289c9 100644 (file)
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2009-2011 New Dream Network
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation.  See file COPYING.
+ *
+ */
+
+#include "client/Client.h"
 #include "client/libceph.h"
+#include "common/Mutex.h"
+#include "common/ceph_argparse.h"
+#include "common/common_init.h"
+#include "common/config.h"
+#include "common/version.h"
+#include "include/str_list.h"
+#include "messages/MMonMap.h"
+#include "msg/SimpleMessenger.h"
 
-#include <string.h>
 #include <fcntl.h>
 #include <iostream>
+#include <string.h>
+#include <string>
+
+static Mutex libceph_init_mutex("libceph_init_mutex");
+static bool libceph_initialized = false; // FIXME! remove this
+static int nonce_seed = 0;
+
+class ceph_cluster_t
+{
+public:
+  ceph_cluster_t(uint64_t msgr_nonce_, md_config_t *conf)
+    : msgr_nonce(msgr_nonce_),
+      mounted(false),
+      client(NULL),
+      monclient(NULL),
+      messenger(NULL),
+      conf(conf)
+  {
+  }
 
-#include "common/ceph_argparse.h"
-#include "common/Mutex.h"
-#include "messages/MMonMap.h"
-#include "common/common_init.h"
-#include "msg/SimpleMessenger.h"
-#include "client/Client.h"
+  ~ceph_cluster_t()
+  {
+    try {
+      disconnect();
+    }
+    catch (const std::exception& e) {
+      // we shouldn't get here, but if we do, we want to know about it.
+      derr << "ceph_cluster_t::~ceph_cluster_t: caught exception: "
+          << e.what() << dendl;
+    }
+    catch (...) {
+      // ignore
+    }
+  }
 
-#include "common/version.h"
+  int connect()
+  {
+    //monmap
+    monclient = new MonClient();
+    if (monclient->build_initial_monmap() < 0) {
+      delete monclient;
+      monclient = NULL;
+      return -1000;
+    }
 
-/* ************* ************* ************* *************
- * C interface
- */
+    //network connection
+    messenger = new SimpleMessenger();
+    messenger->register_entity(entity_name_t::CLIENT());
+
+    //at last the client
+    client = new Client(messenger, monclient);
+
+    messenger->start(false, msgr_nonce); // do not daemonize
+
+    client->init();
+    return 0;
+  }
+
+  void disconnect()
+  {
+    if (mounted) {
+      client->unmount();
+      mounted = false;
+    }
+    if (client) {
+      delete client;
+      client = NULL;
+    }
+    if (messenger) {
+      messenger->wait();
+      messenger->destroy();
+      messenger = NULL;
+    }
+    if (monclient) {
+      delete monclient;
+      monclient = NULL;
+    }
+    // uncomment once conf is de-globalized
+//    if (conf) {
+//      free(conf);
+//      conf = NULL;
+//    }
+  }
+
+  int conf_read_file(const char *path)
+  {
+    if (!path)
+      path = CEPH_CONF_FILE_DEFAULT;
+
+    std::list<std::string> conf_files;
+    get_str_list(path, conf_files);
+    std::deque<std::string> parse_errors;
+    int ret = conf->parse_config_files(conf_files, &parse_errors);
+    if (ret)
+      return ret;
+    conf->parse_env(); // environment variables override
+
+    conf->apply_changes();
+    complain_about_parse_errors(&parse_errors);
+    return 0;
+  }
+
+  void conf_parse_argv(int argc, const char **argv)
+  {
+    vector<const char*> args;
+    argv_to_vec(argc, argv, args);
+    conf->parse_argv(args);
+    conf->apply_changes();
+  }
+
+  int conf_set(const char *option, const char *value)
+  {
+    int ret = conf->set_val(option, value);
+    if (ret)
+      return ret;
+    conf->apply_changes();
+    return 0;
+  }
+
+  int conf_get(const char *option, char *buf, size_t len)
+  {
+    char *tmp = buf;
+    return conf->get_val(option, &tmp, len);
+  }
+
+  int mount(const std::string &mount_root)
+  {
+    if (mounted)
+      return -EINVAL;
+     int ret = client->mount(mount_root);
+     if (ret)
+       return ret;
+    mounted = true;
+    return 0;
+  }
+
+  int umount()
+  {
+    if (!mounted)
+      return -EINVAL;
+    int ret = client->unmount();
+    if (ret)
+      return ret;
+    mounted = false;
+    return 0;
+  }
+
+  Client *get_client()
+  {
+    return client;
+  }
+
+  const char *get_cwd()
+  {
+    client->getcwd(cwd);
+    return cwd.c_str();
+  }
+
+private:
+  uint64_t msgr_nonce;
+  bool mounted;
+  Client *client;
+  MonClient *monclient;
+  SimpleMessenger *messenger;
+  md_config_t *conf;
+  std::string cwd;
+};
 
 extern "C" const char *ceph_version(int *pmajor, int *pminor, int *ppatch)
 {
   int major, minor, patch;
   const char *v = ceph_version_to_str();
-  
+
   int n = sscanf(v, "%d.%d.%d", &major, &minor, &patch);
   if (pmajor)
     *pmajor = (n >= 1) ? major : 0;
@@ -32,367 +208,370 @@ extern "C" const char *ceph_version(int *pmajor, int *pminor, int *ppatch)
   return VERSION;
 }
 
-static Mutex ceph_client_mutex("ceph_client");
-static int client_initialized = 0;
-static int client_mount = 0;
-static Client *client = NULL;
-static MonClient *monclient = NULL;
-static SimpleMessenger *messenger = NULL;
-static int instance = 0;
-
-extern "C" int ceph_initialize(int argc, const char **argv)
+static int ceph_create_with_config_impl(ceph_cluster_t **cluster, md_config_t *conf)
 {
-  ceph_client_mutex.Lock();
-  if (!client_initialized) {
-    //create everything to start a client
-    vector<const char*> args;
-    argv_to_vec(argc, argv, args);
-    // The libceph API needs to be fixed so that we don't have to call
-    // common_init here. Libraries should never call common_init.
-    common_init(args, CEPH_ENTITY_TYPE_CLIENT, CODE_ENVIRONMENT_LIBRARY, 0);
-    keyring_init(&g_conf);
+  // should hold libceph_init_mutex here
+  libceph_initialized = true;
+  uint64_t nonce = (uint64_t)++nonce_seed * 1000000ull + (uint64_t)getpid();
+  *cluster = new ceph_cluster_t(nonce, conf);
+  return 0;
+}
 
-    //monmap
-    monclient = new MonClient();
-    if (monclient->build_initial_monmap() < 0) {
-      delete monclient;
-      return -1; //error!
+extern "C" int ceph_create(ceph_cluster_t **cluster, const char * const id)
+{
+  int ret;
+  libceph_init_mutex.Lock();
+  md_config_t *conf = &g_conf;
+  if (!libceph_initialized) {
+    CephInitParameters iparams(CEPH_ENTITY_TYPE_CLIENT, CEPH_CONF_FILE_DEFAULT);
+    iparams.conf_file = "";
+    if (id) {
+      iparams.name.set(CEPH_ENTITY_TYPE_CLIENT, id);
     }
-    //network connection
-    messenger = new SimpleMessenger();
-    messenger->register_entity(entity_name_t::CLIENT());
 
-    //at last the client
-    client = new Client(messenger, monclient);
+    conf = common_preinit(iparams, CODE_ENVIRONMENT_LIBRARY, 0);
+    conf->parse_env(); // environment variables override
+    conf->apply_changes();
+  }
+  ret = ceph_create_with_config_impl(cluster, conf);
+  libceph_init_mutex.Unlock();
+  return ret;
+}
 
-    uint64_t nonce = (uint64_t)++instance * 1000000ull + (uint64_t)getpid();
-    messenger->start(false, nonce); // do not daemonize
+extern "C" int ceph_create_with_config(ceph_cluster_t **cluster, md_config_t *conf)
+{
+  int ret;
+  libceph_init_mutex.Lock();
+  ret = ceph_create_with_config_impl(cluster, conf);
+  libceph_init_mutex.Unlock();
+  return ret;
+}
 
-    client->init();
-  }
-  ++client_initialized;
-  ceph_client_mutex.Unlock();
-  return 0;
+extern "C" int ceph_connect(ceph_cluster_t *cluster)
+{
+  return cluster->connect();
 }
 
-extern "C" void ceph_deinitialize()
+extern "C" void ceph_shutdown(ceph_cluster_t *cluster)
 {
-  ceph_client_mutex.Lock();
-  --client_initialized;
-  if(!client_initialized) {
-    if(client_mount) {
-      client_mount = 0;
-      client->unmount();
-    }
-    client->shutdown();
-    delete client;
-    messenger->wait();
-    messenger->destroy();
-    delete monclient;
-  }
-  ceph_client_mutex.Unlock();
+  cluster->disconnect();
 }
 
-extern "C" int ceph_conf_set(const char *option, const char *value)
+extern "C" int ceph_conf_read_file(ceph_cluster_t *cluster, const char *path)
 {
-  return g_conf.set_val(option, value);
+  return cluster->conf_read_file(path);
 }
 
-extern "C" int ceph_conf_get(const char *option, char *buf, size_t len)
+extern "C" void ceph_conf_parse_argv(ceph_cluster_t *cluster, int argc,
+                                    const char **argv)
 {
-  char *tmp = buf;
-  return g_conf.get_val(option, &tmp, len);
+  cluster->conf_parse_argv(argc, argv);
 }
 
-extern "C" int ceph_mount()
+extern "C" int ceph_conf_set(ceph_cluster_t *cluster, const char *option,
+                            const char *value)
 {
-  int ret;
-  Mutex::Locker lock(ceph_client_mutex);
-  if(!client_mount) {
-     ret = client->mount("");
-     if (ret!=0)
-       return ret;
-  }
-  ++client_mount;
-  return 0;
+  return cluster->conf_set(option, value);
 }
 
-extern "C" int ceph_umount()
+extern "C" int ceph_conf_get(ceph_cluster_t *cluster, const char *option,
+                            char *buf, size_t len)
 {
-  Mutex::Locker lock(ceph_client_mutex);
-  --client_mount;
-  if (!client_mount)
-    return client->unmount();
-  return 0;
+  return cluster->conf_get(option, buf, len);
 }
 
-extern "C" int ceph_statfs(const char *path, struct statvfs *stbuf)
+extern "C" int ceph_mount(ceph_cluster_t *cluster, const char *root)
 {
-  return client->statfs(path, stbuf);
+  std::string mount_root;
+  if (!root)
+    mount_root = root;
+  return cluster->mount(mount_root);
 }
 
-extern "C" int ceph_get_local_osd()
+extern "C" int ceph_umount(ceph_cluster_t *cluster)
 {
-  return client->get_local_osd();
+  return cluster->umount();
 }
 
-extern "C" int ceph_getcwd(char *buf, int buflen)
+extern "C" int ceph_statfs(ceph_cluster_t *cluster, const char *path,
+                          struct statvfs *stbuf)
 {
-  string cwd;
-  client->getcwd(cwd);
-  int size = cwd.size()+1; //need space for null character
-  if (size > buflen) {
-    if (buflen == 0) return size;
-    else return -ERANGE;
-  }
-  size = cwd.copy(buf, size);
-  buf[size] = '\0'; //fill in null character
-  return 0;
+  return cluster->get_client()->statfs(path, stbuf);
+}
+
+extern "C" int ceph_get_local_osd(ceph_cluster_t *cluster)
+{
+  return cluster->get_client()->get_local_osd();
 }
 
-extern "C" int ceph_chdir (const char *s)
+extern "C" const char* ceph_getcwd(ceph_cluster_t *cluster)
 {
-  return client->chdir(s);
+  return cluster->get_cwd();
 }
 
-/*if we want to extern C this, we need to convert it to const char*,
-which will mean storing it somewhere or else making the caller
-responsible for delete-ing a c-string they didn't create*/
-void ceph_getcwd(string& cwd)
+extern "C" int ceph_chdir (ceph_cluster_t *cluster, const char *s)
 {
-  client->getcwd(cwd);
+  return cluster->get_client()->chdir(s);
 }
 
-extern "C" int ceph_opendir(const char *name, DIR **dirpp)
+extern "C" int ceph_opendir(ceph_cluster_t *cluster,
+                           const char *name, DIR **dirpp)
 {
-  return client->opendir(name, dirpp);
+  return cluster->get_client()->opendir(name, dirpp);
 }
 
-extern "C" int ceph_closedir(DIR *dirp)
+extern "C" int ceph_closedir(ceph_cluster_t *cluster, DIR *dirp)
 {
-  return client->closedir(dirp);
+  return cluster->get_client()->closedir(dirp);
 }
 
-extern "C" int ceph_readdir_r(DIR *dirp, struct dirent *de)
+extern "C" int ceph_readdir_r(ceph_cluster_t *cluster, DIR *dirp, struct dirent *de)
 {
-  return client->readdir_r(dirp, de);
+  return cluster->get_client()->readdir_r(dirp, de);
 }
 
-extern "C" int ceph_readdirplus_r(DIR *dirp, struct dirent *de, struct stat *st, int *stmask)
+extern "C" int ceph_readdirplus_r(ceph_cluster_t *cluster, DIR *dirp,
+                                 struct dirent *de, struct stat *st, int *stmask)
 {
-  return client->readdirplus_r(dirp, de, st, stmask);
+  return cluster->get_client()->readdirplus_r(dirp, de, st, stmask);
 }
 
-extern "C" int ceph_getdents(DIR *dirp, char *buf, int buflen)
+extern "C" int ceph_getdents(ceph_cluster_t *cluster, DIR *dirp,
+                            char *buf, int buflen)
 {
-  return client->getdents(dirp, buf, buflen);
+  return cluster->get_client()->getdents(dirp, buf, buflen);
 }
 
-extern "C" int ceph_getdnames(DIR *dirp, char *buf, int buflen)
+extern "C" int ceph_getdnames(ceph_cluster_t *cluster, DIR *dirp,
+                             char *buf, int buflen)
 {
-  return client->getdnames(dirp, buf, buflen);
+  return cluster->get_client()->getdnames(dirp, buf, buflen);
 }
 
-extern "C" void ceph_rewinddir(DIR *dirp)
+extern "C" void ceph_rewinddir(ceph_cluster_t *cluster, DIR *dirp)
 {
-  client->rewinddir(dirp);
+  cluster->get_client()->rewinddir(dirp);
 }
 
-extern "C" loff_t ceph_telldir(DIR *dirp)
+extern "C" loff_t ceph_telldir(ceph_cluster_t *cluster, DIR *dirp)
 {
-  return client->telldir(dirp);
+  return cluster->get_client()->telldir(dirp);
 }
 
-extern "C" void ceph_seekdir(DIR *dirp, loff_t offset)
+extern "C" void ceph_seekdir(ceph_cluster_t *cluster, DIR *dirp, loff_t offset)
 {
-  client->seekdir(dirp, offset);
+  cluster->get_client()->seekdir(dirp, offset);
 }
 
-extern "C" int ceph_link (const char *existing, const char *newname)
+extern "C" int ceph_link (ceph_cluster_t *cluster, const char *existing,
+                         const char *newname)
 {
-  return client->link(existing, newname);
+  return cluster->get_client()->link(existing, newname);
 }
 
-extern "C" int ceph_unlink (const char *path)
+extern "C" int ceph_unlink(ceph_cluster_t *cluster, const char *path)
 {
-  return client->unlink(path);
+  return cluster->get_client()->unlink(path);
 }
 
-extern "C" int ceph_rename(const char *from, const char *to)
+extern "C" int ceph_rename(ceph_cluster_t *cluster, const char *from,
+                          const char *to)
 {
-  return client->rename(from, to);
+  return cluster->get_client()->rename(from, to);
 }
 
 // dirs
-extern "C" int ceph_mkdir(const char *path, mode_t mode)
+extern "C" int ceph_mkdir(ceph_cluster_t *cluster, const char *path, mode_t mode)
 {
-  return client->mkdir(path, mode);
+  return cluster->get_client()->mkdir(path, mode);
 }
 
-extern "C" int ceph_mkdirs(const char *path, mode_t mode)
+extern "C" int ceph_mkdirs(ceph_cluster_t *cluster, const char *path, mode_t mode)
 {
-  return client->mkdirs(path, mode);
+  return cluster->get_client()->mkdirs(path, mode);
 }
 
-extern "C" int ceph_rmdir(const char *path)
+extern "C" int ceph_rmdir(ceph_cluster_t *cluster, const char *path)
 {
-  return client->rmdir(path);
+  return cluster->get_client()->rmdir(path);
 }
 
 // symlinks
-extern "C" int ceph_readlink(const char *path, char *buf, loff_t size)
+extern "C" int ceph_readlink(ceph_cluster_t *cluster, const char *path,
+                            char *buf, loff_t size)
 {
-  return client->readlink(path, buf, size);
+  return cluster->get_client()->readlink(path, buf, size);
 }
 
-extern "C" int ceph_symlink(const char *existing, const char *newname)
+extern "C" int ceph_symlink(ceph_cluster_t *cluster, const char *existing,
+                           const char *newname)
 {
-  return client->symlink(existing, newname);
+  return cluster->get_client()->symlink(existing, newname);
 }
 
 // inode stuff
-extern "C" int ceph_lstat(const char *path, struct stat *stbuf)
+extern "C" int ceph_lstat(ceph_cluster_t *cluster, const char *path,
+                         struct stat *stbuf)
 {
-  return client->lstat(path, stbuf);
+  return cluster->get_client()->lstat(path, stbuf);
 }
 
-extern "C" int ceph_lstat_precise(const char *path, stat_precise *stbuf)
+extern "C" int ceph_lstat_precise(ceph_cluster_t *cluster, const char *path,
+                                 stat_precise *stbuf)
 {
-  return client->lstat_precise(path, (Client::stat_precise*)stbuf);
+  return cluster->get_client()->lstat_precise(path, (Client::stat_precise*)stbuf);
 }
 
-extern "C" int ceph_setattr(const char *relpath, struct stat *attr, int mask)
+extern "C" int ceph_setattr(ceph_cluster_t *cluster, const char *relpath,
+                           struct stat *attr, int mask)
 {
   Client::stat_precise p_attr = Client::stat_precise(*attr);
-  return client->setattr(relpath, &p_attr, mask);
+  return cluster->get_client()->setattr(relpath, &p_attr, mask);
 }
 
-extern "C" int ceph_setattr_precise(const char *relpath,
-                                   struct stat_precise *attr, int mask) {
-  return client->setattr(relpath, (Client::stat_precise*)attr, mask);
+extern "C" int ceph_setattr_precise(ceph_cluster_t *cluster, const char *relpath,
+                                   struct stat_precise *attr, int mask)
+{
+  return cluster->get_client()->setattr(relpath, (Client::stat_precise*)attr, mask);
 }
 
-extern "C" int ceph_chmod(const char *path, mode_t mode)
+extern "C" int ceph_chmod(ceph_cluster_t *cluster, const char *path, mode_t mode)
 {
-  return client->chmod(path, mode);
+  return cluster->get_client()->chmod(path, mode);
 }
-extern "C" int ceph_chown(const char *path, uid_t uid, gid_t gid)
+extern "C" int ceph_chown(ceph_cluster_t *cluster, const char *path,
+                         uid_t uid, gid_t gid)
 {
-  return client->chown(path, uid, gid);
+  return cluster->get_client()->chown(path, uid, gid);
 }
 
-extern "C" int ceph_utime(const char *path, struct utimbuf *buf)
+extern "C" int ceph_utime(ceph_cluster_t *cluster, const char *path,
+                         struct utimbuf *buf)
 {
-  return client->utime(path, buf);
+  return cluster->get_client()->utime(path, buf);
 }
 
-extern "C" int ceph_truncate(const char *path, loff_t size)
+extern "C" int ceph_truncate(ceph_cluster_t *cluster, const char *path,
+                            loff_t size)
 {
-  return client->truncate(path, size);
+  return cluster->get_client()->truncate(path, size);
 }
 
 // file ops
-extern "C" int ceph_mknod(const char *path, mode_t mode, dev_t rdev)
+extern "C" int ceph_mknod(ceph_cluster_t *cluster, const char *path,
+                         mode_t mode, dev_t rdev)
 {
-  return client->mknod(path, mode, rdev);
+  return cluster->get_client()->mknod(path, mode, rdev);
 }
 
-extern "C" int ceph_open(const char *path, int flags, mode_t mode)
+extern "C" int ceph_open(ceph_cluster_t *cluster, const char *path,
+                        int flags, mode_t mode)
 {
-  return client->open(path, flags, mode);
+  return cluster->get_client()->open(path, flags, mode);
 }
 
-extern "C" int ceph_close(int fd)
+extern "C" int ceph_close(ceph_cluster_t *cluster, int fd)
 {
-  return client->close(fd);
+  return cluster->get_client()->close(fd);
 }
 
-extern "C" loff_t ceph_lseek(int fd, loff_t offset, int whence)
+extern "C" loff_t ceph_lseek(ceph_cluster_t *cluster, int fd,
+                            loff_t offset, int whence)
 {
-  return client->lseek(fd, offset, whence);
+  return cluster->get_client()->lseek(fd, offset, whence);
 }
 
-extern "C" int ceph_read(int fd, char *buf, loff_t size, loff_t offset)
+extern "C" int ceph_read(ceph_cluster_t *cluster, int fd, char *buf,
+                        loff_t size, loff_t offset)
 {
-  return client->read(fd, buf, size, offset);
+  return cluster->get_client()->read(fd, buf, size, offset);
 }
 
-extern "C" int ceph_write(int fd, const char *buf, loff_t size, loff_t offset)
+extern "C" int ceph_write(ceph_cluster_t *cluster, int fd, const char *buf,
+                         loff_t size, loff_t offset)
 {
-  return client->write(fd, buf, size, offset);
+  return cluster->get_client()->write(fd, buf, size, offset);
 }
 
-extern "C" int ceph_ftruncate(int fd, loff_t size)
+extern "C" int ceph_ftruncate(ceph_cluster_t *cluster, int fd, loff_t size)
 {
-  return client->ftruncate(fd, size);
+  return cluster->get_client()->ftruncate(fd, size);
 }
 
-extern "C" int ceph_fsync(int fd, bool syncdataonly)
+extern "C" int ceph_fsync(ceph_cluster_t *cluster, int fd, int syncdataonly)
 {
-  return client->fsync(fd, syncdataonly);
+  return cluster->get_client()->fsync(fd, syncdataonly);
 }
 
-extern "C" int ceph_fstat(int fd, struct stat *stbuf)
+extern "C" int ceph_fstat(ceph_cluster_t *cluster, int fd, struct stat *stbuf)
 {
-  return client->fstat(fd, stbuf);
+  return cluster->get_client()->fstat(fd, stbuf);
 }
 
-extern "C" int ceph_sync_fs()
+extern "C" int ceph_sync_fs(ceph_cluster_t *cluster)
 {
-  return client->sync_fs();
+  return cluster->get_client()->sync_fs();
 }
 
-extern "C" int ceph_get_file_stripe_unit(int fh)
+extern "C" int ceph_get_file_stripe_unit(ceph_cluster_t *cluster, int fh)
 {
-  return client->get_file_stripe_unit(fh);
+  return cluster->get_client()->get_file_stripe_unit(fh);
 }
 
-extern "C" int ceph_get_file_replication(const char *path) {
-  int fd = client->open(path, O_RDONLY);
-  int rep = client->get_file_replication(fd);
-  client->close(fd);
+extern "C" int ceph_get_file_replication(ceph_cluster_t *cluster,
+                                        const char *path)
+{
+  int fd = cluster->get_client()->open(path, O_RDONLY);
+  if (fd < 0)
+    return fd;
+  int rep = cluster->get_client()->get_file_replication(fd);
+  cluster->get_client()->close(fd);
   return rep;
 }
 
-extern "C" int ceph_get_default_preferred_pg(int fd)
+extern "C" int ceph_get_default_preferred_pg(ceph_cluster_t *cluster, int fd)
 {
-  return client->get_default_preferred_pg(fd);
+  return cluster->get_client()->get_default_preferred_pg(fd);
 }
 
-extern "C" int ceph_set_default_file_stripe_unit(int stripe)
+extern "C" int ceph_set_default_file_stripe_unit(ceph_cluster_t *cluster,
+                                                int stripe)
 {
-  client->set_default_file_stripe_unit(stripe);
+  cluster->get_client()->set_default_file_stripe_unit(stripe);
   return 0;
 }
 
-extern "C" int ceph_set_default_file_stripe_count(int count)
+extern "C" int ceph_set_default_file_stripe_count(ceph_cluster_t *cluster,
+                                                 int count)
 {
-  client->set_default_file_stripe_unit(count);
+  cluster->get_client()->set_default_file_stripe_unit(count);
   return 0;
 }
 
-extern "C" int ceph_set_default_object_size(int size)
+extern "C" int ceph_set_default_object_size(ceph_cluster_t *cluster, int size)
 {
-  client->set_default_object_size(size);
+  cluster->get_client()->set_default_object_size(size);
   return 0;
 }
 
-extern "C" int ceph_set_default_file_replication(int replication)
+extern "C" int ceph_set_default_file_replication(ceph_cluster_t *cluster,
+                                                int replication)
 {
-  client->set_default_file_replication(replication);
+  cluster->get_client()->set_default_file_replication(replication);
   return 0;
 }
 
-extern "C" int ceph_set_default_preferred_pg(int pg)
+extern "C" int ceph_set_default_preferred_pg(ceph_cluster_t *cluster, int pg)
 {
-  client->set_default_preferred_pg(pg);
+  cluster->get_client()->set_default_preferred_pg(pg);
   return 0;
 }
 
-extern "C" int ceph_get_file_stripe_address(int fh, loff_t offset, char *buf, int buflen)
+extern "C" int ceph_get_file_stripe_address(ceph_cluster_t *cluster, int fh,
+                                           loff_t offset, char *buf, int buflen)
 {
   string address;
-  int r = client->get_file_stripe_address(fh, offset, address);
+  int r = cluster->get_client()->get_file_stripe_address(fh, offset, address);
   if (r != 0) return r; //at time of writing, method ONLY returns
   // 0 or -EINVAL if there are no known osds
   int len = address.size()+1;
@@ -405,13 +584,11 @@ extern "C" int ceph_get_file_stripe_address(int fh, loff_t offset, char *buf, in
   return 0;
 }
 
-extern "C" int ceph_localize_reads(int val)
+extern "C" int ceph_localize_reads(ceph_cluster_t *cluster, int val)
 {
-  if (!client)
-    return -ENOENT;
   if (!val)
-    client->clear_filer_flags(CEPH_OSD_FLAG_LOCALIZE_READS);
+    cluster->get_client()->clear_filer_flags(CEPH_OSD_FLAG_LOCALIZE_READS);
   else
-    client->set_filer_flags(CEPH_OSD_FLAG_LOCALIZE_READS);
+    cluster->get_client()->set_filer_flags(CEPH_OSD_FLAG_LOCALIZE_READS);
   return 0;
 }