]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
Rework ProfLogger
authorColin Patrick McCabe <cmccabe@alumni.cmu.edu>
Mon, 18 Jul 2011 20:53:03 +0000 (13:53 -0700)
committerColin Patrick McCabe <cmccabe@alumni.cmu.edu>
Mon, 18 Jul 2011 20:53:03 +0000 (13:53 -0700)
* Replace existing proflogger config options with "profiling_logger_uri".
This option controls profiling logger sinks.

* ProfLogger: replace file-writing code with code that sends the
information over a UNIX domain socket.

* handle_conf_change is now fully and correctly implemented. We never
read from the md_config_t structure except in this function, so there
are no races. We re-create the thread when the settings change (no need
for SIGHUP, etc.)

* Replace the single big lock with a lock per Proflogger.

* No need for favg any more; just use fset everywhere for floating-point
variables.

Signed-off-by: Colin McCabe <colin.mccabe@dreamhost.com>
24 files changed:
src/Makefile.am
src/ceph.conf.twoosds
src/client/Client.cc
src/client/SyntheticClient.cc
src/common/ProfLogType.h [deleted file]
src/common/ProfLogger.cc
src/common/ProfLogger.h
src/common/ceph_context.cc
src/common/config.cc
src/common/config.h
src/mds/MDBalancer.cc
src/mds/MDCache.cc
src/mds/MDLog.cc
src/mds/MDLog.h
src/mds/MDS.cc
src/mds/MDS.h
src/mds/Server.cc
src/mds/Server.h
src/os/FileStore.cc
src/osd/OSD.cc
src/osd/OSD.h
src/osd/ReplicatedPG.cc
src/osdc/Journaler.cc
src/vstart.sh

index cf2dea99d3a5b0f6e8833dee7303c8dfc4938984..2d8de1ce462182ea4a9c0ba978cb32b0c9064f58 100644 (file)
@@ -873,7 +873,6 @@ noinst_HEADERS = \
         common/ConfUtils.h\
         common/DecayCounter.h\
         common/Finisher.h\
-        common/ProfLogType.h\
         common/ProfLogger.h\
         common/MemoryModel.h\
         common/Mutex.h\
index e796911fc3bb197cbad025573e318c0cb0241d1c..c0cfc68f1a0c1450e66c97caa2d4751aa1268bce 100644 (file)
@@ -11,7 +11,7 @@
 ; ---------------------
 [global]
        pid file = /home/sage/ceph/src/out/$name.pid
-       profiling logger dir = /home/sage/ceph/src/log
+       profiling logger uri = /home/sage/ceph/src/profiling_logger.$name
        log sym dir = /home/sage/ceph/src/out
        chdir = /home/sage/ceph/src
        restart on core dump = false
index 3543031c450993a522be2739792c20aa6784c36f..90ef8c597307d02b759de6d2b6d0e357b62dbb62 100644 (file)
@@ -1104,8 +1104,8 @@ int Client::make_request(MetaRequest *request,
     utime_t lat = ceph_clock_now(cct);
     lat -= request->sent_stamp;
     ldout(cct, 20) << "lat " << lat << dendl;
-    client_logger->favg(l_c_lat,(double)lat);
-    client_logger->favg(l_c_reply,(double)lat);
+    client_logger->fset(l_c_lat,(double)lat);
+    client_logger->fset(l_c_reply,(double)lat);
   }
 
   request->put();
@@ -5203,7 +5203,7 @@ int Client::_write(Fh *f, int64_t offset, uint64_t size, const char *buf)
   utime_t lat = ceph_clock_now(cct);
   lat -= start;
   if (client_logger)
-    client_logger->favg(l_c_wrlat,(double)lat);
+    client_logger->fset(l_c_wrlat,(double)lat);
     
   // assume success for now.  FIXME.
   uint64_t totalwritten = size;
index 2b5b36410b6c24b8fd6f9bc8ceca602a1996a71f..6189db12c910a35efc5d81ede56151e748c8bad4 100644 (file)
@@ -2350,9 +2350,9 @@ int SyntheticClient::object_rw(int nobj, int osize, int wrpc,
     lat -= start;
     if (client_logger) {
       if (write) 
-       client_logger->favg(l_c_owrlat, lat);
+       client_logger->fset(l_c_owrlat, lat);
       else 
-       client_logger->favg(l_c_ordlat, lat);
+       client_logger->fset(l_c_ordlat, lat);
     }
   }
 
diff --git a/src/common/ProfLogType.h b/src/common/ProfLogType.h
deleted file mode 100644 (file)
index 34d88d4..0000000
+++ /dev/null
@@ -1,70 +0,0 @@
-// -*- 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) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * 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_LOGTYPE_H
-#define CEPH_LOGTYPE_H
-
-#include "include/types.h"
-
-#include <vector>
-using std::vector;
-
-class ProfLogType {
- protected:
-  int first_key, num_keys;
-  vector<const char*> key_name;
-  vector<bool> inc_keys, avg_keys;
-
-  friend class ProfLogger;
-
- public:
-  ProfLogType(int first, int tail) :
-    first_key(first), num_keys(tail-first - 1),
-    key_name(num_keys), inc_keys(num_keys), avg_keys(num_keys) {
-    for (int i=0; i<num_keys; i++) {
-      key_name[i] = 0;
-      inc_keys[i] = 0;
-      avg_keys[i] = 0;
-    }
-  }
-  int lookup_key(int key, bool isnew=false) {
-    int i = key - first_key - 1;
-    assert(i >= 0 && i < num_keys);
-    assert(isnew || key_name[i]);
-    return i;
-  }
-  void add_key(int key, const char *name, bool is_inc, bool is_avg) {
-    int i = lookup_key(key, true);
-    assert(!key_name[i]);  // only register each type once!
-    key_name[i] = name;
-    inc_keys[i] = is_inc;
-    avg_keys[i] = is_avg;
-  }
-  void add_inc(int key, const char *name) {
-    return add_key(key, name, true, false);
-  }
-  void add_set(int key, const char *name) {
-    return add_key(key, name, false, false);
-  }
-  void add_avg(int key, const char *name) {
-    return add_key(key, name, true, true);
-  }
-  void validate() {
-    for (int i=0; i<num_keys; i++)
-      assert(key_name[i]);
-  }
-};
-
-#endif
index 58397157224e6b996a474f71e3b20e4af072ef60..0c856ae32424893fa64055b4634aa06468e05134 100644 (file)
@@ -3,7 +3,7 @@
 /*
  * Ceph - scalable distributed file system
  *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ * Copyright (C) 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
  *
  */
 
+#define __STDC_FORMAT_MACROS // for PRId64, etc.
 
-
+#include "common/ProfLogger.h"
+#include "common/Thread.h"
+#include "common/config.h"
+#include "common/config_obs.h"
+#include "common/dout.h"
+#include "common/errno.h"
+#include "common/safe_io.h"
+
+#include <errno.h>
+#include <fcntl.h>
+#include <inttypes.h>
+#include <map>
+#include <poll.h>
+#include <sstream>
+#include <stdint.h>
+#include <string.h>
 #include <string>
+#include <sys/socket.h>
+#include <sys/types.h>
+#include <sys/un.h>
+#include <unistd.h>
 
-#include "ProfLogType.h"
-#include "ProfLogger.h"
-
-#include <iostream>
-#include <memory>
-#include "Clock.h"
-
-#include "common/config.h"
+#define PFL_SUCCESS ((void*)(intptr_t)0)
+#define PFL_FAIL ((void*)(intptr_t)1)
+#define COUNT_DISABLED ((uint64_t)(int64_t)-1)
 
-#include <sys/stat.h>
-#include <sys/types.h>
+using std::ostringstream;
 
-#include "common/Timer.h"
+enum prof_log_data_any_t {
+  PROF_LOG_DATA_ANY_NONE,
+  PROF_LOG_DATA_ANY_U64,
+  PROF_LOG_DATA_ANY_DOUBLE
+};
 
-//////////////// C_FlushProfLoggers ////////////////
-class C_FlushProfLoggers : public Context
+class ProfLogThread : public Thread
 {
 public:
-  C_FlushProfLoggers(ProfLoggerCollection *coll_)
-    : coll(coll_)
+
+  static std::string create_shutdown_pipe(int *pipe_rd, int *pipe_wr)
   {
+    int pipefd[2];
+    int ret = pipe2(pipefd, O_CLOEXEC);
+    if (ret < 0) {
+      int err = errno;
+      ostringstream oss;
+      oss << "ProfLogThread::create_shutdown_pipe error: "
+         << cpp_strerror(err);
+      return oss.str();
+    }
+
+    *pipe_rd = pipefd[0];
+    *pipe_wr = pipefd[1];
+    return "";
   }
 
-  void finish(int r) {
-    coll->flush_all_loggers();
+  static std::string bind_and_listen(const std::string &sock_path, int *fd)
+  {
+    int sock_fd = socket(PF_UNIX, SOCK_STREAM, 0);
+    if (sock_fd < 0) {
+      int err = errno;
+      ostringstream oss;
+      oss << "ProfLogThread::bind_and_listen: "
+         << "failed to create socket: " << cpp_strerror(err);
+      return oss.str();
+    }
+    struct sockaddr_un address;
+    memset(&address, 0, sizeof(struct sockaddr_un));
+    address.sun_family = AF_UNIX;
+    snprintf(address.sun_path, sizeof(address.sun_path), sock_path.c_str());
+    if (bind(sock_fd, (struct sockaddr*)&address,
+            sizeof(struct sockaddr_un)) != 0) {
+      int err = errno;
+      ostringstream oss;
+      oss << "ProfLogThread::bind_and_listen: "
+         << "failed to bind socket: " << cpp_strerror(err);
+      close(sock_fd);
+      return oss.str();
+    }
+    if (listen(sock_fd, 5) != 0) {
+      int err = errno;
+      ostringstream oss;
+      oss << "ProfLogThread::bind_and_listen: "
+         << "failed to listen to socket: " << cpp_strerror(err);
+      close(sock_fd);
+      return oss.str();
+    }
+    *fd = sock_fd;
+    return "";
+  }
+
+  ProfLogThread(int sock_fd, int shutdown_fd, ProfLoggerCollection *parent)
+    : m_sock_fd(sock_fd),
+      m_shutdown_fd(shutdown_fd),
+      m_parent(parent)
+  {
+  }
+
+  virtual ~ProfLogThread()
+  {
+    if (m_sock_fd != -1)
+      close(m_sock_fd);
+    if (m_shutdown_fd != -1)
+      close(m_shutdown_fd);
+  }
+
+  virtual void* entry()
+  {
+    while (true) {
+      struct pollfd fds[2];
+      memset(fds, 0, sizeof(fds));
+      fds[0].fd = m_sock_fd;
+      fds[0].events = POLLOUT | POLLWRBAND;
+      fds[1].fd = m_shutdown_fd;
+      fds[1].events = POLLIN | POLLRDBAND;
+
+      int ret = poll(fds, 2, NULL);
+      if (ret < 0) {
+       if (ret == -EINTR) {
+         continue;
+       }
+       int err = errno;
+       lderr(m_parent->m_cct) << "ProfLogThread: poll(2) error: '"
+           << cpp_strerror(err) << dendl;
+       return PFL_FAIL;
+      }
+
+      if (fds[0].revents & POLLOUT) {
+       // Send out some data
+       if (!do_accept())
+         return PFL_FAIL;
+      }
+      if (fds[1].revents & POLLIN) {
+       // Parent wants us to shut down
+       return PFL_SUCCESS;
+      }
+    }
   }
 
 private:
-  ProfLoggerCollection *coll;
+  const static int MAX_PFL_RETRIES = 10;
+
+  bool do_accept()
+  {
+    struct sockaddr_un address;
+    socklen_t address_length;
+    int connection_fd = accept(m_sock_fd, (struct sockaddr*) &address,
+                                  &address_length);
+    if (connection_fd < 0) {
+      int err = errno;
+      lderr(m_parent->m_cct) << "ProfLogThread: do_accept error: '"
+         << cpp_strerror(err) << dendl;
+      return false;
+    }
+    FILE *fp = fdopen(m_sock_fd, "w");
+    if (!fp) {
+      int err = errno;
+      lderr(m_parent->m_cct) << "ProfLogThread: failed to fdopen '"
+         << m_sock_fd << "'. error " << cpp_strerror(err) << dendl;
+      close(connection_fd);
+      return false;
+    }
+    fprintf(fp, "{");
+
+    {
+      Mutex::Locker lck(m_parent->m_lock); // Take lock to access m_loggers
+      for (std::set <ProfLogger*>::iterator log = m_parent->m_loggers.begin();
+          log != m_parent->m_loggers.end(); ++log)
+      {
+       // This will take the logger's lock for short period of time,
+       // then release it.
+       (*log)->write_json_to_fp(fp);
+      }
+    }
+
+    fprintf(fp, "}");
+    fflush(fp);
+    fclose(fp); // calls close(connection_fd)
+    return true;
+  }
+
+  ProfLogThread(ProfLogThread &rhs);
+  const ProfLogThread &operator=(const ProfLogThread &rhs);
+
+  int m_sock_fd;
+  int m_shutdown_fd;
+  ProfLoggerCollection *m_parent;
 };
 
-//////////////// ProfLoggerCollection ////////////////
 ProfLoggerCollection::
-ProfLoggerCollection(CephContext *cct_)
-  : lock("ProfLoggerCollection::lock"),
-    logger_timer(cct_, lock), logger_event(NULL),
-    last_flush(0), need_reopen(true), need_reset(false), cct(cct_)
+ProfLoggerCollection(CephContext *cct)
+  : m_cct(cct),
+    m_thread(NULL),
+    m_lock("ProfLoggerCollection"),
+    m_shutdown_fd(-1)
 {
 }
 
 ProfLoggerCollection::
 ~ProfLoggerCollection()
 {
-  try {
-    lock.Lock();
-    logger_timer.shutdown();
-    lock.Unlock();
-  }
-  catch (...) {
+  Mutex::Locker lck(m_lock);
+  shutdown();
+  for (std::set <ProfLogger*>::iterator l = m_loggers.begin();
+       l != m_loggers.end(); ++l) {
+    delete *l;
   }
+  m_loggers.clear();
 }
 
-void ProfLoggerCollection::
-logger_reopen_all()
+const char** ProfLoggerCollection::
+get_tracked_conf_keys() const
 {
-  Mutex::Locker l(lock);
-  need_reopen = true;
+  static const char *KEYS[] =
+       { "profiling_logger_uri", NULL };
+  return KEYS;
 }
 
 void ProfLoggerCollection::
-logger_reset_all()
+handle_conf_change(const md_config_t *conf,
+                  const std::set <std::string> &changed)
 {
-  Mutex::Locker l(lock);
-  need_reopen = true;
-  need_reset = true;
+  Mutex::Locker lck(m_lock);
+  if (conf->profiling_logger_uri.empty()) {
+    shutdown();
+  }
+  else {
+    if (!init(conf->profiling_logger_uri)) {
+      lderr(m_cct) << "Initializing profiling logger failed!" << dendl;
+    }
+  }
 }
 
 void ProfLoggerCollection::
-logger_start()
+logger_add(class ProfLogger *l)
 {
-  Mutex::Locker l(lock);
-  logger_timer.init();
-  if (!logger_event)
-    flush_all_loggers();
+  Mutex::Locker lck(m_lock);
+  std::set<ProfLogger*>::iterator i = m_loggers.find(l);
+  assert(i == m_loggers.end());
+  m_loggers.insert(l);
 }
 
 void ProfLoggerCollection::
-logger_tare(utime_t s)
+logger_remove(class ProfLogger *l)
 {
-  Mutex::Locker l(lock);
-
-  ldout(cct, 10) << "logger_tare " << s << dendl;
-
-  start = s;
-
-  utime_t fromstart = ceph_clock_now(cct);
-  if (fromstart < start) {
-    lderr(cct) << "logger_tare time jumped backwards from "
-        << start << " to " << fromstart << dendl;
-    fromstart = start;
-  }
-  fromstart -= start;
-  last_flush = fromstart.sec();
+  Mutex::Locker lck(m_lock);
+  std::set<ProfLogger*>::iterator i = m_loggers.find(l);
+  assert(i != m_loggers.end());
+  m_loggers.erase(i);
 }
 
-void ProfLoggerCollection::
-logger_add(ProfLogger *logger)
+bool ProfLoggerCollection::
+init(const std::string &uri)
 {
-  Mutex::Locker l(lock);
+  /* Shut down old thread, if it exists.  */
+  shutdown();
+
+  /* Set up things for the new thread */
+  std::string err;
+  int pipe_rd, pipe_wr;
+  err = ProfLogThread::create_shutdown_pipe(&pipe_rd, &pipe_wr);
+  if (!err.empty()) {
+    lderr(m_cct) << "ProfLoggerCollection::init: error: " << err << dendl;
+    return false;
+  }
+  int sock_fd;
+  err = ProfLogThread::bind_and_listen(uri, &sock_fd);
+  if (!err.empty()) {
+    lderr(m_cct) << "ProfLoggerCollection::init: failed: " << err << dendl;
+    close(pipe_rd);
+    close(pipe_wr);
+    return false;
+  }
 
-  if (logger_list.empty()) {
-    if (start == utime_t())
-      start = ceph_clock_now(cct);
-    last_flush = 0;
+  /* Create new thread */
+  m_thread = new (std::nothrow) ProfLogThread(sock_fd, pipe_rd, this);
+  if (!m_thread) {
+    close(sock_fd);
+    close(pipe_rd);
+    close(pipe_wr);
+    return false;
   }
-  logger_list.push_back(logger);
-  logger->lock = &lock;
+  m_thread->create();
+  m_shutdown_fd = pipe_wr;
+  return 0;
 }
 
 void ProfLoggerCollection::
-logger_remove(ProfLogger *logger)
+shutdown()
 {
-  Mutex::Locker l(lock);
-
-  for (list<ProfLogger*>::iterator p = logger_list.begin();
-       p != logger_list.end();
-       p++) {
-    if (*p == logger) {
-      logger_list.erase(p);
-      delete logger;
-      if (logger_list.empty() && logger_event) {
-       // If there are no timers, stop the timer events.
-       logger_timer.cancel_event(logger_event);
-       logger_event = 0;
-      }
-      return;
+  if (m_thread) {
+    // Send a byte to the shutdown pipe that the thread is listening to
+    char buf[1] = { 0x0 };
+    int ret = safe_write(m_shutdown_fd, buf, sizeof(buf));
+    m_shutdown_fd = -1;
+
+    if (ret == 0) {
+      // Join and delete the thread
+      m_thread->join();
+      delete m_thread;
+    }
+    else {
+      lderr(m_cct) << "ProfLoggerCollection::shutdown: failed to write "
+             "to thread shutdown pipe: error " << ret << dendl;
     }
+    m_thread = NULL;
   }
 }
 
-void ProfLoggerCollection::
-flush_all_loggers()
+ProfLogger::
+~ProfLogger()
 {
-  // ProfLoggerCollection lock must be held here.
-  ldout(cct, 20) << "flush_all_loggers" << dendl;
+}
 
-  if (!cct->_conf->profiling_logger)
+void ProfLogger::
+inc(int idx, uint64_t amt)
+{
+  Mutex::Locker lck(m_lock);
+  assert(idx > m_lower_bound);
+  assert(idx < m_upper_bound);
+  prof_log_data_any_d& data(m_data[idx - m_lower_bound - 1]);
+  if (data.type != PROF_LOG_DATA_ANY_U64)
     return;
-
-  utime_t now = ceph_clock_now(cct);
-  utime_t fromstart = now;
-  if (fromstart < start) {
-    lderr(cct) << "logger time jumped backwards from " << start << " to "
-        << fromstart << dendl;
-    //assert(0);
-    start = fromstart;
-  }
-  fromstart -= start;
-  int now_sec = fromstart.sec();
-
-  // do any catching up we need to
-  bool twice = now_sec - last_flush >= 2 * cct->_conf->profiling_logger_interval;
- again:
-  ldout(cct, 20) << "fromstart " << fromstart << " last_flush " << last_flush << " flushing" << dendl;
-
-  // This logic seems unecessary. We're holding the mutex the whole time here,
-  // so need_reopen and need_reset can't change unless we change them.
-  // TODO: clean this up slightly
-  bool reopen = need_reopen;
-  bool reset = need_reset;
-
-  for (list<ProfLogger*>::iterator p = logger_list.begin();
-       p != logger_list.end();
-       ++p)
-    (*p)->_flush(need_reopen, need_reset, last_flush);
-
-  // did full pass while true?
-  if (reopen && need_reopen)
-    need_reopen = false;
-  if (reset && need_reset)
-    need_reset = false;
-
-  last_flush = now_sec - (now_sec % cct->_conf->profiling_logger_interval);
-  if (twice) {
-    twice = false;
-    goto again;
-  }
-
-  // schedule next flush event
-  utime_t next;
-  next.sec_ref() = start.sec() + last_flush + cct->_conf->profiling_logger_interval;
-  next.nsec_ref() = start.nsec();
-  ldout(cct, 20) << "logger now=" << now
-                  << "  start=" << start
-                  << "  next=" << next
-                  << dendl;
-  logger_event = new C_FlushProfLoggers(this);
-  logger_timer.add_event_at(next, logger_event);
+  data.u.u64 += amt;
+  if (data.count != COUNT_DISABLED)
+    data.count++;
 }
 
-//////////////// ProfLoggerConfObs ////////////////
-ProfLoggerConfObs::ProfLoggerConfObs(ProfLoggerCollection *coll_)
-  : coll(coll_)
+void ProfLogger::
+set(int idx, uint64_t amt)
 {
+  Mutex::Locker lck(m_lock);
+  assert(idx > m_lower_bound);
+  assert(idx < m_upper_bound);
+  prof_log_data_any_d& data(m_data[idx - m_lower_bound - 1]);
+  if (data.type != PROF_LOG_DATA_ANY_U64)
+    return;
+  data.u.u64 = amt;
+  if (data.count != COUNT_DISABLED)
+    data.count++;
 }
 
-ProfLoggerConfObs::~ProfLoggerConfObs()
+uint64_t ProfLogger::
+get(int idx)
 {
+  Mutex::Locker lck(m_lock);
+  assert(idx > m_lower_bound);
+  assert(idx < m_upper_bound);
+  prof_log_data_any_d& data(m_data[idx - m_lower_bound - 1]);
+  if (data.type != PROF_LOG_DATA_ANY_DOUBLE)
+    return 0;
+  return data.u.u64;
 }
 
-const char **ProfLoggerConfObs::get_tracked_conf_keys() const
+void ProfLogger::
+finc(int idx, double amt)
 {
-  static const char *KEYS[] = {
-    "profiling_logger", "profiling_logger_interval", "profiling_logger_calc_variance",
-    "profiling_logger_subdir", "profiling_logger_dir", NULL
-  };
-  return KEYS;
+  Mutex::Locker lck(m_lock);
+  assert(idx > m_lower_bound);
+  assert(idx < m_upper_bound);
+  prof_log_data_any_d& data(m_data[idx - m_lower_bound - 1]);
+  if (data.type != PROF_LOG_DATA_ANY_DOUBLE)
+    return;
+  data.u.dbl += amt;
+  if (data.count != COUNT_DISABLED)
+    data.count++;
 }
 
-void ProfLoggerConfObs::handle_conf_change(const md_config_t *conf,
-                         const std::set <std::string> &changed)
+void ProfLogger::
+fset(int idx, double amt)
 {
-  // This could be done a *lot* smarter, if anyone cares to spend time
-  // fixing this up.
-  // We could probably just take the mutex and call _open_log from here.
-  coll->logger_reopen_all();
+  Mutex::Locker lck(m_lock);
+  assert(idx > m_lower_bound);
+  assert(idx < m_upper_bound);
+  prof_log_data_any_d& data(m_data[idx - m_lower_bound - 1]);
+  if (data.type != PROF_LOG_DATA_ANY_DOUBLE)
+    return;
+  data.u.dbl = amt;
+  if (data.count != COUNT_DISABLED)
+    data.count++;
 }
 
-//////////////// ProfLogger ////////////////
-void ProfLogger::_open_log()
+double ProfLogger::
+fget(int idx)
 {
-  struct stat st;
-
-  filename = "";
-  if ((!cct->_conf->chdir.empty()) &&
-      (cct->_conf->profiling_logger_dir.substr(0,1) != "/")) {
-    char cwd[PATH_MAX];
-    char *c = getcwd(cwd, sizeof(cwd));
-    assert(c);
-    filename = c;
-    filename += "/";
-  }
-
-  filename = cct->_conf->profiling_logger_dir;
-
-  // make (feeble) attempt to create logger_dir
-  if (::stat(filename.c_str(), &st))
-    ::mkdir(filename.c_str(), 0750);
-
-  filename += "/";
-  if (!cct->_conf->profiling_logger_subdir.empty()) {
-    filename += cct->_conf->profiling_logger_subdir;
-    ::mkdir( filename.c_str(), 0755 );   // make sure dir exists
-    filename += "/";
-  }
-  filename += name;
-
-  ldout(cct, 10) << "ProfLogger::_open " << filename << dendl;
-  if (out.is_open())
-    out.close();
-  out.open(filename.c_str(),
-          (need_reset || need_reset) ? ofstream::out : ofstream::out|ofstream::app);
-  if (!out.is_open()) {
-    ldout(cct, 10) << "failed to open '" << filename << "'" << dendl;
-    return; // we fail
-  }
-
-  // success
-  need_open = false;
+  Mutex::Locker lck(m_lock);
+  assert(idx > m_lower_bound);
+  assert(idx < m_upper_bound);
+  prof_log_data_any_d& data(m_data[idx - m_lower_bound - 1]);
+  if (data.type != PROF_LOG_DATA_ANY_DOUBLE)
+    return 0.0;
+  return data.u.dbl;
 }
 
-ProfLogger::~ProfLogger()
+void ProfLogger::
+write_json_to_fp(FILE *fp)
 {
-  out.close();
+  Mutex::Locker lck(m_lock);
+
+  prof_log_data_vec_t::const_iterator d = m_data.begin();
+  prof_log_data_vec_t::const_iterator d_end = m_data.end();
+  for (; d != d_end; ++d) {
+    const prof_log_data_any_d &data(*d);
+    if (d->count != COUNT_DISABLED) {
+      switch (d->type) {
+       case PROF_LOG_DATA_ANY_U64:
+         fprintf(fp, "\"%s\" : { \"count\" : %" PRId64 ", "
+                 "\"sum\" : %" PRId64 " },\n", 
+                 data.name, data.count, data.u.u64);
+         break;
+       case PROF_LOG_DATA_ANY_DOUBLE:
+         fprintf(fp, "\"%s\" : { \"count\" : %" PRId64 ", "
+                 "\"sum\" : %g },\n",
+                 data.name, data.count, data.u.dbl);
+         break;
+       default:
+         assert(0);
+         break;
+      }
+    }
+    else {
+      switch (d->type) {
+       case PROF_LOG_DATA_ANY_U64:
+         fprintf(fp, "\"%s\" : %" PRId64 ",\n", data.name, data.u.u64);
+         break;
+       case PROF_LOG_DATA_ANY_DOUBLE:
+         fprintf(fp, "\"%s\" : %g,\n", data.name, data.u.dbl);
+         break;
+       default:
+         assert(0);
+         break;
+      }
+    }
+  }
 }
 
-void ProfLogger::reopen()
+ProfLogger::
+ProfLogger(CephContext *cct, const std::string &name,
+          int lower_bound, int upper_bound)
+  : m_cct(cct),
+    m_lower_bound(lower_bound),
+    m_upper_bound(upper_bound),
+    m_name(std::string("ProfLogger::") + name.c_str()),
+    m_lock(m_name.c_str())
 {
-  Mutex::Locker l(*lock);
-  need_open = true;
+  m_data.resize(upper_bound - lower_bound - 1);
 }
 
-void ProfLogger::reset()
+ProfLogger::prof_log_data_any_d::
+prof_log_data_any_d()
+  : name(NULL),
+    type(PROF_LOG_DATA_ANY_NONE),
+    count(COUNT_DISABLED)
 {
-  Mutex::Locker l(*lock);
-  need_open = true;
-  need_reset = true;
+  memset(&u, 0, sizeof(u));
 }
 
-
-void ProfLogger::_flush(bool need_reopen, bool need_reset, int last_flush)
+ProfLoggerBuilder::
+ProfLoggerBuilder(CephContext *cct, const std::string &name,
+                  int first, int last)
+  : m_prof_logger(new ProfLogger(cct, name, first, last))
 {
-  if (need_reopen)
-    _open_log();
-  if (need_reset) {
-    // reset the counters
-    for (int i=0; i<type->num_keys; i++) {
-      this->vals[i] = 0;
-      this->fvals[i] = 0;
-    }
-    need_reset = false;
-  }
-
-  ldout(cct, 20) << "ProfLogger::_flush on " << this << dendl;
-
-  // header?
-  wrote_header_last++;
-  if (wrote_header_last > 10) {
-    out << "#" << type->num_keys;
-    for (int i=0; i<type->num_keys; i++) {
-      out << "\t" << (type->key_name[i] ? type->key_name[i] : "???");
-      if (type->avg_keys[i])
-       out << "\t(n)\t(var)";
-    }
-    out << std::endl;  //out << "\t (" << type->keymap.size() << ")" << endl;
-    wrote_header_last = 0;
-  }
-
-  // write line to log
-  out << last_flush;
-  for (int i=0; i<type->num_keys; i++) {
-    if (type->avg_keys[i]) {
-      if (vals[i] > 0) {
-       double avg = (fvals[i] / (double)vals[i]);
-       double var = 0.0;
-       if (cct->_conf->profiling_logger_calc_variance &&
-           (unsigned)vals[i] == vals_to_avg[i].size()) {
-         for (vector<double>::iterator p = vals_to_avg[i].begin(); p != vals_to_avg[i].end(); ++p)
-           var += (avg - *p) * (avg - *p);
-       }
-       char s[256];
-       snprintf(s, sizeof(s), "\t%.5lf\t%lld\t%.5lf", avg, (long long int)vals[i], var);
-       out << s;
-      } else
-       out << "\t0\t0\t0";
-    } else {
-      if (fvals[i] > 0 && vals[i] == 0)
-       out << "\t" << fvals[i];
-      else {
-       //cout << this << " p " << i << " and size is " << vals.size() << std::endl;
-       out << "\t" << vals[i];
-      }
-    }
-  }
-
-  // reset the counters
-  for (int i=0; i<type->num_keys; i++) {
-    if (type->inc_keys[i]) {
-      this->vals[i] = 0;
-      this->fvals[i] = 0;
-    }
-  }
-
-  out << std::endl;
 }
 
-
-
-void ProfLogger::inc(int key, int64_t v)
+ProfLoggerBuilder::
+~ProfLoggerBuilder()
 {
-  if (!cct->_conf->profiling_logger)
-    return;
-  lock->Lock();
-  int i = type->lookup_key(key);
-  vals[i] += v;
-  lock->Unlock();
+  if (m_prof_logger)
+    delete m_prof_logger;
+  m_prof_logger = NULL;
 }
 
-void ProfLogger::finc(int key, double v)
+void ProfLoggerBuilder::
+add_u64(int idx, const char *name)
 {
-  if (!cct->_conf->profiling_logger)
-    return;
-  lock->Lock();
-  int i = type->lookup_key(key);
-  fvals[i] += v;
-  lock->Unlock();
+  add_impl(idx, name, PROF_LOG_DATA_ANY_U64, COUNT_DISABLED);
 }
 
-void ProfLogger::set(int key, int64_t v)
+void ProfLoggerBuilder::
+add_fl(int idx, const char *name)
 {
-  if (!cct->_conf->profiling_logger)
-    return;
-  lock->Lock();
-  int i = type->lookup_key(key);
-  vals[i] = v;
-  lock->Unlock();
+  add_impl(idx, name, PROF_LOG_DATA_ANY_DOUBLE, COUNT_DISABLED);
 }
 
-
-void ProfLogger::fset(int key, double v)
+void ProfLoggerBuilder::
+add_fl_avg(int idx, const char *name)
 {
-  if (!cct->_conf->profiling_logger)
-    return;
-  lock->Lock();
-  int i = type->lookup_key(key);
-  fvals[i] = v;
-  lock->Unlock();
+  add_impl(idx, name, PROF_LOG_DATA_ANY_DOUBLE, 0);
 }
 
-void ProfLogger::favg(int key, double v)
+void ProfLoggerBuilder::
+add_impl(int idx, const char *name, int ty, uint64_t count)
 {
-  if (!cct->_conf->profiling_logger)
-    return;
-  lock->Lock();
-  int i = type->lookup_key(key);
-  vals[i]++;
-  fvals[i] += v;
-  if (cct->_conf->profiling_logger_calc_variance)
-    vals_to_avg[i].push_back(v);
-  lock->Unlock();
+  assert(idx > m_prof_logger->m_lower_bound);
+  assert(idx < m_prof_logger->m_upper_bound);
+  ProfLogger::prof_log_data_vec_t &vec(m_prof_logger->m_data);
+  ProfLogger::prof_log_data_any_d
+    &data(vec[idx - m_prof_logger->m_lower_bound - 1]);
+  data.name = name;
+  data.type = ty;
+  data.count = count;
 }
 
-int64_t ProfLogger::get(int key)
+ProfLogger *ProfLoggerBuilder::
+create_proflogger()
 {
-  if (!cct->_conf->profiling_logger)
-    return 0;
-  lock->Lock();
-  int i = type->lookup_key(key);
-  int64_t r = 0;
-  if (i >= 0 && i < (int)vals.size())
-    r = vals[i];
-  lock->Unlock();
-  return r;
+  ProfLogger::prof_log_data_vec_t::const_iterator d = m_prof_logger->m_data.begin();
+  ProfLogger::prof_log_data_vec_t::const_iterator d_end = m_prof_logger->m_data.end();
+  for (; d != d_end; ++d) {
+    assert(d->type != PROF_LOG_DATA_ANY_NONE);
+  }
+  ProfLogger *ret = m_prof_logger;
+  m_prof_logger = NULL;
+  return ret;
 }
-
index f64e8c42bd1d7eeb0ce61d5ad8959c093a3b5e15..6d1f89d655c98b9380544ad57b395672ad634394 100644 (file)
@@ -3,7 +3,7 @@
 /*
  * Ceph - scalable distributed file system
  *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ * Copyright (C) 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
  */
 
 
-#ifndef CEPH_LOGGER_H
-#define CEPH_LOGGER_H
+#ifndef CEPH_PROF_LOG_H
+#define CEPH_PROF_LOG_H
 
-#include "common/config.h"
-#include "common/Clock.h"
-#include "common/ProfLogType.h"
-#include "common/Timer.h"
-#include "include/types.h"
+#include "common/config_obs.h"
+#include "common/Mutex.h"
 
+#include <stdint.h>
 #include <string>
-#include <fstream>
 #include <vector>
 
-class ProfLogger;
+class ProfLoggerBuilder;
+class CephContext;
+class Thread;
 
-class ProfLoggerCollection
+/*
+ * ProfLog manages the profiler logging for a Ceph process.
+ */
+class ProfLoggerCollection : public md_config_obs_t
 {
 public:
-  ProfLoggerCollection(CephContext *cct_);
+  ProfLoggerCollection(CephContext *cct);
   ~ProfLoggerCollection();
-  void logger_reopen_all();
-  void logger_reset_all();
-  void logger_add(class ProfLogger *l);
-  void logger_remove(class ProfLogger *l);
-  void flush_all_loggers();
-  void logger_tare(utime_t when);
-  void logger_start();
-private:
-  Mutex lock; // big lock.  lame, but this way I protect ProfLogType too!
-  SafeTimer logger_timer;
-  Context *logger_event;
-  list<ProfLogger*> logger_list;
-  utime_t start;
-  int last_flush; // in seconds since start
-  bool need_reopen;
-  bool need_reset;
-  CephContext *cct;
-};
-
-class ProfLoggerConfObs : public md_config_obs_t {
-public:
-  ProfLoggerConfObs(ProfLoggerCollection *coll_);
-  ~ProfLoggerConfObs();
   virtual const char** get_tracked_conf_keys() const;
   virtual void handle_conf_change(const md_config_t *conf,
                          const std::set <std::string> &changed);
+  void logger_add(class ProfLogger *l);
+  void logger_remove(class ProfLogger *l);
 private:
-  ProfLoggerCollection *coll;
-};
+  bool init(const std::string &uri);
+  void shutdown();
 
-class ProfLogger {
- protected:
-   CephContext *cct;
-  // my type
-  std::string name, filename;
-  ProfLogType *type;
+  CephContext *m_cct;
+  Thread* m_thread;
 
-  bool need_open;
-  bool need_reset;
-  bool need_close;
+  /** Protects m_loggers */
+  Mutex m_lock;
 
-  // values for this instance
-  std::vector<int64_t> vals;
-  std::vector<double> fvals;
-  std::vector< std::vector<double> > vals_to_avg;  // for calculating variance
+  int m_shutdown_fd;
+  std::set <ProfLogger*> m_loggers;
 
-  std::ofstream out;
+  friend class ProfLogThread;
+};
 
-  // what i've written
-  //int last_logged;
-  int wrote_header_last;
+class ProfLogger
+{
+public:
+  ~ProfLogger();
 
-  void _open_log();
+  void inc(int idx, uint64_t v = 1);
+  void set(int idx, uint64_t v);
+  uint64_t get(int idx);
 
- private:
-  Mutex *lock;
+  void fset(int idx, double v);
+  void finc(int idx, double v);
+  double fget(int idx);
 
- public:
-  ProfLogger(CephContext *cct_, const std::string &n, ProfLogType *t) :
-    cct(cct_), name(n), type(t),
-    need_open(true), need_reset(false), need_close(false),
-    vals(t->num_keys), fvals(t->num_keys), vals_to_avg(t->num_keys),
-    wrote_header_last(10000), lock(NULL) { }
-  ~ProfLogger();
+  void write_json_to_fp(FILE *fp);
 
-  void inc(int f, int64_t v = 1);
-  void set(int f, int64_t v);
-  int64_t get(int f);
+private:
+  ProfLogger(CephContext *cct, const std::string &name,
+            int lower_bound, int upper_bound);
+  ProfLogger(const ProfLogger &rhs);
+  ProfLogger& operator=(const ProfLogger &rhs);
+
+  /** Represents a ProfLogger data element. */
+  struct prof_log_data_any_d {
+    prof_log_data_any_d();
+    const char *name;
+    int type;
+    union {
+      uint64_t u64;
+      double dbl;
+    } u;
+    uint64_t count;
+  };
+  typedef std::vector<prof_log_data_any_d> prof_log_data_vec_t;
 
-  void fset(int f, double v);
-  void finc(int f, double v);
-  void favg(int f, double v);
+  CephContext *m_cct;
+  int m_lower_bound;
+  int m_upper_bound;
+  const std::string m_name;
 
-  void _flush(bool need_reopen, bool need_reset, int last_flush);
+  /** Protects m_data */
+  Mutex m_lock;
 
-  void reopen();
-  void reset();
-  void close();
+  prof_log_data_vec_t m_data;
 
-  friend class ProfLoggerCollection;
+  friend class ProfLoggerBuilder;
 };
 
+/* Class for constructing ProfLoggers.
+ *
+ * This class peforms some validation that the parameters we have supplied are
+ * correct in create_proflogger().
+ *
+ * In the future, we will probably get rid of the first/last arguments, since
+ * ProfLoggerBuilder can deduce them itself.
+ */
 class ProfLoggerBuilder
 {
 public:
   ProfLoggerBuilder(CephContext *cct, const std::string &name,
-                   int first, int last)
-      : m_cct(cct),
-        m_name(name)
-  {
-    m_plt = new ProfLogType(first, last);
-  }
-
-  void add_u64(int key, const char *name) {
-    m_plt->add_inc(key, name);
-  }
-  void add_fl(int key, const char *name) {
-    m_plt->add_inc(key, name);
-  }
-  void add_fl_avg(int key, const char *name) {
-    m_plt->add_avg(key, name);
-  }
-  ProfLogger* create_proflogger() {
-    // TODO: remove m_plt
-    m_plt->validate();
-    return new ProfLogger(m_cct, m_name, m_plt);
-  }
-
+                   int first, int last);
+  ~ProfLoggerBuilder();
+  void add_u64(int key, const char *name);
+  void add_fl(int key, const char *name);
+  void add_fl_avg(int key, const char *name);
+  ProfLogger* create_proflogger();
 private:
   ProfLoggerBuilder(const ProfLoggerBuilder &rhs);
   ProfLoggerBuilder& operator=(const ProfLoggerBuilder &rhs);
+  void add_impl(int idx, const char *name, int ty, uint64_t count);
 
-  CephContext *m_cct;
-  std::string m_name;
-  ProfLogType *m_plt;
+  ProfLogger *m_prof_logger;
 };
 
 #endif
index 4c894a3f2bdce5277bea4a8cebd2ae089db72a14..a4c333c5662cbf344187902da0c2a7fe76e8227f 100644 (file)
@@ -46,7 +46,6 @@ public:
       }
       if (_reopen_logs) {
        _cct->_doss->reopen_logs(_cct->_conf);
-       _cct->_prof_logger_collection->logger_reopen_all();
        _reopen_logs = false;
       }
     }
@@ -79,14 +78,12 @@ CephContext(uint32_t module_type_)
     _dout(_doss),
     _module_type(module_type_),
     _service_thread(NULL),
-    _prof_logger_collection(NULL),
-    _prof_logger_conf_obs(NULL)
+    _prof_logger_collection(NULL)
 {
   pthread_spin_init(&_service_thread_lock, PTHREAD_PROCESS_SHARED);
   _prof_logger_collection = new ProfLoggerCollection(this);
-  _prof_logger_conf_obs = new ProfLoggerConfObs(_prof_logger_collection);
   _conf->add_observer(_doss);
-  _conf->add_observer(_prof_logger_conf_obs);
+  _conf->add_observer(_prof_logger_collection);
 }
 
 CephContext::
@@ -94,7 +91,7 @@ CephContext::
 {
   join_service_thread();
 
-  _conf->remove_observer(_prof_logger_conf_obs);
+  _conf->remove_observer(_prof_logger_collection);
   _conf->remove_observer(_doss);
 
   delete _prof_logger_collection;
index 0558acb801049829ca3def954d03cb68e6eebb46..15e44acf649e539afcd220e95f5b5750115a2f13 100644 (file)
@@ -132,11 +132,7 @@ struct config_option config_optionsp[] = {
   OPTION(monmap, OPT_STR, 0),
   OPTION(mon_host, OPT_STR, 0),
   OPTION(daemonize, OPT_BOOL, false),
-  OPTION(profiling_logger, OPT_BOOL, false),
-  OPTION(profiling_logger_interval, OPT_INT, 1),
-  OPTION(profiling_logger_calc_variance, OPT_BOOL, false),
-  OPTION(profiling_logger_subdir, OPT_STR, 0),
-  OPTION(profiling_logger_dir, OPT_STR, "/var/log/ceph/stat"),
+  OPTION(profiling_logger_uri, OPT_STR, ""),
   OPTION(log_file, OPT_STR, 0),
   OPTION(log_dir, OPT_STR, 0),
   OPTION(log_sym_dir, OPT_STR, 0),
index c99ac912a58f6984fe5065b1d71a3b5d6c377261..3b32b484f09182eeba3e76c922eac083d8a44cb8 100644 (file)
@@ -139,11 +139,7 @@ public:
   bool daemonize;
 
   // profiling logger
-  bool profiling_logger;
-  int profiling_logger_interval;
-  bool profiling_logger_calc_variance;
-  std::string profiling_logger_subdir;
-  std::string profiling_logger_dir;
+  std::string profiling_logger_uri;
 
   std::string log_file;
   std::string log_dir;
index a94147fabe55adae3108bc3fb304a854731d755a..720b39b89f643887911fa0541aff0efaf267787f 100644 (file)
@@ -28,6 +28,8 @@
 #include "messages/MHeartbeat.h"
 #include "messages/MMDSLoadTargets.h"
 
+#include <fstream>
+#include <iostream>
 #include <vector>
 #include <map>
 using std::map;
index c42c6515cdc8c5badfed8902454c660abe64f7c7..d9920a8982b0625b0f298d9741255e8f1e115750 100644 (file)
@@ -85,6 +85,7 @@
 #include "common/Timer.h"
 
 #include <errno.h>
+#include <fstream>
 #include <iostream>
 #include <sstream>
 #include <string>
@@ -5423,7 +5424,7 @@ void MDCache::trim_inode(CDentry *dn, CInode *in, CDir *con, map<int, MCacheExpi
       mds->logger->inc("outt");
     else {
       mds->logger->inc("outut");
-      mds->logger->favg("oututl", ceph_clock_now(g_ceph_context) - in->hack_load_stamp);
+      mds->logger->fset("oututl", ceph_clock_now(g_ceph_context) - in->hack_load_stamp);
     }
   }
   */
index bf2d67c42341e5a1b563271acff29ce15da43963..378f7edbc835f474e90873fdbaf27c31e20c2df8 100644 (file)
@@ -44,7 +44,7 @@ MDLog::~MDLog()
 }
 
 
-void MDLog::open_logger()
+void MDLog::create_logger()
 {
   char name[80];
   snprintf(name, sizeof(name), "mds.%s.log", g_conf->name.get_id().c_str());
index 104c39f51464d0b1ebc9c98dee8cfa29043e4ca2..ce5b4152bc4f1182de955844f1724adcb8a2728e 100644 (file)
@@ -143,7 +143,7 @@ private:
   void init_journaler();
   
 public:
-  void open_logger();
+  void create_logger();
   
   // replay state
   map<inodeno_t, set<inodeno_t> >   pending_exports;
index ff285d39950b93822a7a975c869e09bdbb466b4b..cb948ab204053b1f6360ceb51da7f8cf1be451a2 100644 (file)
@@ -186,9 +186,9 @@ MDS::~MDS() {
     messenger->destroy();
 }
 
-void MDS::open_logger()
+void MDS::create_logger()
 {
-  dout(10) << "open_logger" << dendl;
+  dout(10) << "create_logger" << dendl;
   {
     char name[80];
     snprintf(name, sizeof(name), "mds.%s.%llu.log",
@@ -283,14 +283,8 @@ void MDS::open_logger()
     g_ceph_context->GetProfLoggerCollection()->logger_add(mlogger);
   }
 
-  mdlog->open_logger();
-  server->open_logger();
-
-  {
-    ProfLoggerCollection *coll = g_ceph_context->GetProfLoggerCollection();
-    coll->logger_tare(mdsmap->get_created());
-    coll->logger_start();
-  }
+  mdlog->create_logger();
+  server->create_logger();
 }
 
 
@@ -519,7 +513,7 @@ int MDS::init(int wanted_state)
   // schedule tick
   reset_tick();
 
-  open_logger();
+  create_logger();
 
   mds_lock.Unlock();
 
index f95d25a5d0195e06cfe4d06b3e276703bbc647df..253681455416be6c50cf77cf417062446cf2f2ff 100644 (file)
@@ -346,7 +346,7 @@ class MDS : public Dispatcher {
   // start up, shutdown
   int init(int wanted_state=MDSMap::STATE_BOOT);
 
-  void open_logger();
+  void create_logger();
 
   void bcast_mds_map();  // to mounted clients
 
index 8a6ce3e314d117d63584b79e905fe6d2a8397fac..9fd56d6c44ebbcf416734b1291e19f396fc401da 100644 (file)
@@ -48,7 +48,6 @@
 #include "include/filepath.h"
 #include "common/Timer.h"
 #include "common/ProfLogger.h"
-#include "common/ProfLogType.h"
 
 #include <errno.h>
 #include <fcntl.h>
@@ -64,7 +63,7 @@ using namespace std;
 #undef dout_prefix
 #define dout_prefix *_dout << "mds" << mds->get_nodeid() << ".server "
 
-void Server::open_logger()
+void Server::create_logger()
 {
   char name[80];
   snprintf(name, sizeof(name), "mds.%s.server.log", g_conf->name.get_id().c_str());
@@ -824,7 +823,7 @@ void Server::early_reply(MDRequest *mdr, CInode *tracei, CDentry *tracedn)
 
   mds->logger->inc(l_mds_reply);
   double lat = ceph_clock_now(g_ceph_context) - mdr->client_request->get_recv_stamp();
-  mds->logger->favg(l_mds_replyl, lat);
+  mds->logger->fset(l_mds_replyl, lat);
   dout(20) << "lat " << lat << dendl;
 }
 
@@ -866,7 +865,7 @@ void Server::reply_request(MDRequest *mdr, MClientReply *reply, CInode *tracei,
 
     mds->logger->inc(l_mds_reply);
     double lat = ceph_clock_now(g_ceph_context) - mdr->client_request->get_recv_stamp();
-    mds->logger->favg(l_mds_replyl, lat);
+    mds->logger->fset(l_mds_replyl, lat);
     dout(20) << "lat " << lat << dendl;
     
     if (tracei)
index ec1ee66793fd9b40d4a58715932408176548fd8c..793f9029546dac1c928d5c32be604fcaf25191b3 100644 (file)
@@ -61,7 +61,7 @@ public:
     delete logger;
   }
 
-  void open_logger();
+  void create_logger();
 
   // message handler
   void dispatch(Message *m);
index 78b85dd33d1e5bbd3d487dba827d23b42534871c..91ce192cb4359384f42cd2303ba468140d6b7b33 100644 (file)
@@ -1909,12 +1909,7 @@ void FileStore::start_logger(int whoami, utime_t tare)
   logger = plb.create_proflogger();
   if (journal)
     journal->logger = logger;
-  {
-    ProfLoggerCollection *coll = g_ceph_context->GetProfLoggerCollection();
-    coll->logger_add(logger);
-    coll->logger_tare(tare);
-    coll->logger_start();
-  }
+  g_ceph_context->GetProfLoggerCollection()->logger_add(logger);
 }
 
 void FileStore::stop_logger()
index 945864d0597173ae47b970bd050126b6b755dbe7..976b65b76c4c29b2c81cdd18989cd49e1fceabcf 100644 (file)
@@ -84,6 +84,7 @@
 
 #include "common/errno.h"
 
+#include <fstream>
 #include <iostream>
 #include <errno.h>
 #include <sys/stat.h>
@@ -406,7 +407,7 @@ OSD::OSD(int id, Messenger *internal_messenger, Messenger *external_messenger,
   cluster_messenger(internal_messenger),
   client_messenger(external_messenger),
   monc(mc),
-  logger(NULL), logger_started(false),
+  logger(NULL),
   store(NULL),
   map_in_progress(false),
   clog(hbm->cct, client_messenger, &mc->monmap, mc, LogClient::NO_FLAGS),
@@ -574,7 +575,7 @@ int OSD::init()
     return -EINVAL;
   }
 
-  open_logger();
+  create_logger();
     
   // i'm ready!
   client_messenger->add_dispatcher_head(this);
@@ -621,9 +622,9 @@ int OSD::init()
   return 0;
 }
 
-void OSD::open_logger()
+void OSD::create_logger()
 {
-  dout(10) << "open_logger" << dendl;
+  dout(10) << "create_logger" << dendl;
 
   char name[80];
   snprintf(name, sizeof(name), "osd.%d.log", whoami);
@@ -684,20 +685,6 @@ void OSD::open_logger()
 
   logger = osd_plb.create_proflogger();
   g_ceph_context->GetProfLoggerCollection()->logger_add(logger);
-
-  if (osdmap->get_epoch() > 0)
-    start_logger();
-}
-
-void OSD::start_logger()
-{
-  ProfLoggerCollection *coll = g_ceph_context->GetProfLoggerCollection();
-  coll->logger_tare(osdmap->get_created());
-  coll->logger_start();
-  logger_started = true;
-
-  // start the objectstore logger too
-  store->start_logger(whoami, osdmap->get_created());
 }
 
 int OSD::shutdown()
@@ -2216,10 +2203,6 @@ void OSD::handle_command(MMonCommand *m)
       clog.error() << "cannot parse pgid from command '" << m->cmd << "'\n";
 
     }
-  } else if (m->cmd.size() == 2 && m->cmd[0] == "logger" && m->cmd[1] == "reset") {
-    g_ceph_context->GetProfLoggerCollection()->logger_reset_all();
-  } else if (m->cmd.size() == 2 && m->cmd[0] == "logger" && m->cmd[1] == "reopen") {
-    g_ceph_context->reopen_logs();
   } else if (m->cmd[0] == "heap") {
     if (ceph_using_tcmalloc())
       ceph_heap_profiler_handle_command(m->cmd, clog);
@@ -3076,9 +3059,6 @@ void OSD::handle_osd_map(MOSDMap *m)
          (!newmap->exists(*p) || !newmap->is_up(*p)))    // but not the new one
        note_down_osd(*p);
     
-    if (!logger_started)
-      g_ceph_context->GetProfLoggerCollection()->logger_start();
-
     osdmap = newmap;
 
     superblock.current_epoch = cur;
index a3aa6cc7f3f2705df335e80205f59d47d2f130af..b785d7f2c9726500ebc1a86dc716f70cf70213e8 100644 (file)
@@ -127,7 +127,6 @@ protected:
   Messenger   *client_messenger;
   MonClient   *monc;
   ProfLogger      *logger;
-  bool         logger_started;
   ObjectStore *store;
 
   // cover OSDMap update data when using multiple msgrs
@@ -151,8 +150,7 @@ protected:
   Cond dispatch_cond;
   int dispatch_running;
 
-  void open_logger();
-  void start_logger();
+  void create_logger();
   void tick();
   void _dispatch(Message *m);
 
index 8751c335492b6dd1a1e6a150525628de3d450104..4d20f69441eef2d84b386e1bb535417eb3206639 100644 (file)
@@ -673,23 +673,23 @@ void ReplicatedPG::log_op_stats(OpContext *ctx)
 
   osd->logger->inc(l_osd_op_outb, outb);
   osd->logger->inc(l_osd_op_inb, inb);
-  osd->logger->favg(l_osd_op_lat, latency);
+  osd->logger->fset(l_osd_op_lat, latency);
 
   if (op->may_read() && op->may_write()) {
     osd->logger->inc(l_osd_op_rw);
     osd->logger->inc(l_osd_op_rw_inb, inb);
     osd->logger->inc(l_osd_op_rw_outb, outb);
-    osd->logger->favg(l_osd_op_rw_rlat, rlatency);
-    osd->logger->favg(l_osd_op_rw_lat, latency);
+    osd->logger->fset(l_osd_op_rw_rlat, rlatency);
+    osd->logger->fset(l_osd_op_rw_lat, latency);
   } else if (op->may_read()) {
     osd->logger->inc(l_osd_op_r);
     osd->logger->inc(l_osd_op_r_outb, outb);
-    osd->logger->favg(l_osd_op_r_lat, latency);
+    osd->logger->fset(l_osd_op_r_lat, latency);
   } else if (op->may_write()) {
     osd->logger->inc(l_osd_op_w);
     osd->logger->inc(l_osd_op_w_inb, inb);
-    osd->logger->favg(l_osd_op_w_rlat, rlatency);
-    osd->logger->favg(l_osd_op_w_lat, latency);
+    osd->logger->fset(l_osd_op_w_rlat, rlatency);
+    osd->logger->fset(l_osd_op_w_lat, latency);
   } else
     assert(0);
 
@@ -711,11 +711,11 @@ void ReplicatedPG::log_subop_stats(MOSDSubOp *op, int tag_inb, int tag_lat)
   osd->logger->inc(l_osd_sop);
 
   osd->logger->inc(l_osd_sop_inb, inb);
-  osd->logger->favg(l_osd_sop_lat, latency);
+  osd->logger->fset(l_osd_sop_lat, latency);
 
   if (tag_inb)
     osd->logger->inc(tag_inb, inb);
-  osd->logger->favg(tag_lat, latency);
+  osd->logger->fset(tag_lat, latency);
 
   dout(15) << "log_subop_stats " << *op << " inb " << inb << " latency " << latency << dendl;
 }
index 43eca74aa0dde2082840be30e8ec707e42028428..2696965312d2c0574d9b152fd2ce2b45236ea5bd 100644 (file)
@@ -394,7 +394,7 @@ void Journaler::_finish_flush(int r, uint64_t start, utime_t stamp)
   if (logger) {
     utime_t lat = ceph_clock_now(cct);
     lat -= stamp;
-    logger->favg(logger_key_lat, lat);
+    logger->fset(logger_key_lat, lat);
   }
 
   // adjust safe_pos
index 906b3f4e293e19f1873debd62190d555ddb1e980..3de4e85809ad0d6c8fd823aa03c74575b1be59b8 100755 (executable)
@@ -249,8 +249,7 @@ DAEMONOPTS="
        log file = out/\$host
        log per instance = true
        log sym history = 100
-        profiling logger = true
-       profiling logger dir = log
+        profiling logger uri = out/proflog.$name
        chdir = \"\"
        pid file = out/\$name.pid
 "