]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/log/Log.cc
import 15.2.5
[ceph.git] / ceph / src / log / Log.cc
index 6af49b4c8cd56fd2a37ecb7f00b37fe215681193..68a884e7111efe1706fc92ae97f54c581b58e96e 100644 (file)
@@ -3,27 +3,27 @@
 
 #include "Log.h"
 
-#include <errno.h>
-#include <syslog.h>
-
 #include "common/errno.h"
 #include "common/safe_io.h"
-#include "common/Clock.h"
 #include "common/Graylog.h"
 #include "common/valgrind.h"
 
-#include "include/assert.h"
+#include "include/ceph_assert.h"
 #include "include/compat.h"
 #include "include/on_exit.h"
 
 #include "Entry.h"
+#include "LogClock.h"
 #include "SubsystemMap.h"
 
-#define DEFAULT_MAX_NEW    100
-#define DEFAULT_MAX_RECENT 10000
+#include <errno.h>
+#include <fcntl.h>
+#include <syslog.h>
 
-#define PREALLOC 1000000
+#include <iostream>
+#include <set>
 
+#define MAX_LOG_BUF 65536
 
 namespace ceph {
 namespace logging {
@@ -38,65 +38,38 @@ static void log_on_exit(void *p)
   delete (Log **)p;// Delete allocated pointer (not Log object, the pointer only!)
 }
 
-Log::Log(SubsystemMap *s)
-  : m_indirect_this(NULL),
+Log::Log(const SubsystemMap *s)
+  : m_indirect_this(nullptr),
     m_subs(s),
-    m_queue_mutex_holder(0),
-    m_flush_mutex_holder(0),
-    m_new(), m_recent(),
-    m_fd(-1),
-    m_uid(0),
-    m_gid(0),
-    m_fd_last_error(0),
-    m_syslog_log(-2), m_syslog_crash(-2),
-    m_stderr_log(1), m_stderr_crash(-1),
-    m_graylog_log(-3), m_graylog_crash(-3),
-    m_stop(false),
-    m_max_new(DEFAULT_MAX_NEW),
-    m_max_recent(DEFAULT_MAX_RECENT),
-    m_inject_segv(false)
+    m_recent(DEFAULT_MAX_RECENT)
 {
-  int ret;
-
-  ret = pthread_mutex_init(&m_flush_mutex, NULL);
-  assert(ret == 0);
-
-  ret = pthread_mutex_init(&m_queue_mutex, NULL);
-  assert(ret == 0);
-
-  ret = pthread_cond_init(&m_cond_loggers, NULL);
-  assert(ret == 0);
-
-  ret = pthread_cond_init(&m_cond_flusher, NULL);
-  assert(ret == 0);
-
-  // kludge for prealloc testing
-  if (false)
-    for (int i=0; i < PREALLOC; i++)
-      m_recent.enqueue(new Entry);
+  m_log_buf.reserve(MAX_LOG_BUF);
 }
 
 Log::~Log()
 {
   if (m_indirect_this) {
-    *m_indirect_this = NULL;
+    *m_indirect_this = nullptr;
   }
 
-  assert(!is_started());
+  ceph_assert(!is_started());
   if (m_fd >= 0)
     VOID_TEMP_FAILURE_RETRY(::close(m_fd));
-
-  pthread_mutex_destroy(&m_queue_mutex);
-  pthread_mutex_destroy(&m_flush_mutex);
-  pthread_cond_destroy(&m_cond_loggers);
-  pthread_cond_destroy(&m_cond_flusher);
 }
 
 
 ///
+void Log::set_coarse_timestamps(bool coarse) {
+  std::scoped_lock lock(m_flush_mutex);
+  if (coarse)
+    Entry::clock().coarsen();
+  else
+    Entry::clock().refine();
+}
 
 void Log::set_flush_on_exit()
 {
+  std::scoped_lock lock(m_flush_mutex);
   // Make sure we flush on shutdown.  We do this by deliberately
   // leaking an indirect pointer to ourselves (on_exit() can't
   // unregister a callback).  This is not racy only becuase we
@@ -107,43 +80,45 @@ void Log::set_flush_on_exit()
   }
 }
 
-void Log::set_max_new(int n)
+void Log::set_max_new(std::size_t n)
 {
+  std::scoped_lock lock(m_queue_mutex);
   m_max_new = n;
 }
 
-void Log::set_max_recent(int n)
+void Log::set_max_recent(std::size_t n)
 {
-  pthread_mutex_lock(&m_flush_mutex);
-  m_flush_mutex_holder = pthread_self();
+  std::scoped_lock lock(m_flush_mutex);
   m_max_recent = n;
-  m_flush_mutex_holder = 0;
-  pthread_mutex_unlock(&m_flush_mutex);
 }
 
-void Log::set_log_file(string fn)
+void Log::set_log_file(std::string_view fn)
 {
+  std::scoped_lock lock(m_flush_mutex);
   m_log_file = fn;
 }
 
-void Log::set_log_stderr_prefix(const std::string& p)
+void Log::set_log_stderr_prefix(std::string_view p)
 {
+  std::scoped_lock lock(m_flush_mutex);
   m_log_stderr_prefix = p;
 }
 
 void Log::reopen_log_file()
 {
-  pthread_mutex_lock(&m_flush_mutex);
+  std::scoped_lock lock(m_flush_mutex);
+  if (!is_started()) {
+    return;
+  }
   m_flush_mutex_holder = pthread_self();
   if (m_fd >= 0)
     VOID_TEMP_FAILURE_RETRY(::close(m_fd));
   if (m_log_file.length()) {
-    m_fd = ::open(m_log_file.c_str(), O_CREAT|O_WRONLY|O_APPEND, 0644);
+    m_fd = ::open(m_log_file.c_str(), O_CREAT|O_WRONLY|O_APPEND|O_CLOEXEC, 0644);
     if (m_fd >= 0 && (m_uid || m_gid)) {
-      int r = ::fchown(m_fd, m_uid, m_gid);
-      if (r < 0) {
-       r = -errno;
-       cerr << "failed to chown " << m_log_file << ": " << cpp_strerror(r)
+      if (::fchown(m_fd, m_uid, m_gid) < 0) {
+       int e = errno;
+       std::cerr << "failed to chown " << m_log_file << ": " << cpp_strerror(e)
             << std::endl;
       }
     }
@@ -151,209 +126,189 @@ void Log::reopen_log_file()
     m_fd = -1;
   }
   m_flush_mutex_holder = 0;
-  pthread_mutex_unlock(&m_flush_mutex);
 }
 
 void Log::chown_log_file(uid_t uid, gid_t gid)
 {
-  pthread_mutex_lock(&m_flush_mutex);
+  std::scoped_lock lock(m_flush_mutex);
   if (m_fd >= 0) {
     int r = ::fchown(m_fd, uid, gid);
     if (r < 0) {
       r = -errno;
-      cerr << "failed to chown " << m_log_file << ": " << cpp_strerror(r)
+      std::cerr << "failed to chown " << m_log_file << ": " << cpp_strerror(r)
           << std::endl;
     }
   }
-  pthread_mutex_unlock(&m_flush_mutex);
 }
 
 void Log::set_syslog_level(int log, int crash)
 {
-  pthread_mutex_lock(&m_flush_mutex);
+  std::scoped_lock lock(m_flush_mutex);
   m_syslog_log = log;
   m_syslog_crash = crash;
-  pthread_mutex_unlock(&m_flush_mutex);
 }
 
 void Log::set_stderr_level(int log, int crash)
 {
-  pthread_mutex_lock(&m_flush_mutex);
+  std::scoped_lock lock(m_flush_mutex);
   m_stderr_log = log;
   m_stderr_crash = crash;
-  pthread_mutex_unlock(&m_flush_mutex);
 }
 
 void Log::set_graylog_level(int log, int crash)
 {
-  pthread_mutex_lock(&m_flush_mutex);
+  std::scoped_lock lock(m_flush_mutex);
   m_graylog_log = log;
   m_graylog_crash = crash;
-  pthread_mutex_unlock(&m_flush_mutex);
 }
 
 void Log::start_graylog()
 {
-  pthread_mutex_lock(&m_flush_mutex);
+  std::scoped_lock lock(m_flush_mutex);
   if (! m_graylog.get())
     m_graylog = std::make_shared<Graylog>(m_subs, "dlog");
-  pthread_mutex_unlock(&m_flush_mutex);
 }
 
 
 void Log::stop_graylog()
 {
-  pthread_mutex_lock(&m_flush_mutex);
+  std::scoped_lock lock(m_flush_mutex);
   m_graylog.reset();
-  pthread_mutex_unlock(&m_flush_mutex);
 }
 
-void Log::submit_entry(Entry *e)
+void Log::submit_entry(Entry&& e)
 {
-  pthread_mutex_lock(&m_queue_mutex);
+  std::unique_lock lock(m_queue_mutex);
   m_queue_mutex_holder = pthread_self();
 
-  if (m_inject_segv)
+  if (unlikely(m_inject_segv))
     *(volatile int *)(0) = 0xdead;
 
   // wait for flush to catch up
-  while (m_new.m_len > m_max_new)
-    pthread_cond_wait(&m_cond_loggers, &m_queue_mutex);
+  while (is_started() &&
+        m_new.size() > m_max_new) {
+    if (m_stop) break; // force addition
+    m_cond_loggers.wait(lock);
+  }
 
-  m_new.enqueue(e);
-  pthread_cond_signal(&m_cond_flusher);
+  m_new.emplace_back(std::move(e));
+  m_cond_flusher.notify_all();
   m_queue_mutex_holder = 0;
-  pthread_mutex_unlock(&m_queue_mutex);
 }
 
-
-Entry *Log::create_entry(int level, int subsys)
+void Log::flush()
 {
-  if (true) {
-    return new Entry(ceph_clock_now(),
-                    pthread_self(),
-                    level, subsys);
-  } else {
-    // kludge for perf testing
-    Entry *e = m_recent.dequeue();
-    e->m_stamp = ceph_clock_now();
-    e->m_thread = pthread_self();
-    e->m_prio = level;
-    e->m_subsys = subsys;
-    return e;
+  std::scoped_lock lock1(m_flush_mutex);
+  m_flush_mutex_holder = pthread_self();
+
+  {
+    std::scoped_lock lock2(m_queue_mutex);
+    m_queue_mutex_holder = pthread_self();
+    assert(m_flush.empty());
+    m_flush.swap(m_new);
+    m_cond_loggers.notify_all();
+    m_queue_mutex_holder = 0;
   }
+
+  _flush(m_flush, false);
+  m_flush_mutex_holder = 0;
 }
 
-Entry *Log::create_entry(int level, int subsys, size_t* expected_size)
+void Log::_log_safe_write(std::string_view sv)
 {
-  if (true) {
-    ANNOTATE_BENIGN_RACE_SIZED(expected_size, sizeof(*expected_size),
-                               "Log hint");
-    size_t size = __atomic_load_n(expected_size, __ATOMIC_RELAXED);
-    void *ptr = ::operator new(sizeof(Entry) + size);
-    return new(ptr) Entry(ceph_clock_now(),
-       pthread_self(), level, subsys,
-       reinterpret_cast<char*>(ptr) + sizeof(Entry), size, expected_size);
-  } else {
-    // kludge for perf testing
-    Entry *e = m_recent.dequeue();
-    e->m_stamp = ceph_clock_now();
-    e->m_thread = pthread_self();
-    e->m_prio = level;
-    e->m_subsys = subsys;
-    return e;
+  if (m_fd < 0)
+    return;
+  int r = safe_write(m_fd, sv.data(), sv.size());
+  if (r != m_fd_last_error) {
+    if (r < 0)
+      std::cerr << "problem writing to " << m_log_file
+           << ": " << cpp_strerror(r)
+           << std::endl;
+    m_fd_last_error = r;
   }
 }
 
-void Log::flush()
+void Log::_flush_logbuf()
 {
-  pthread_mutex_lock(&m_flush_mutex);
-  m_flush_mutex_holder = pthread_self();
-  pthread_mutex_lock(&m_queue_mutex);
-  m_queue_mutex_holder = pthread_self();
-  EntryQueue t;
-  t.swap(m_new);
-  pthread_cond_broadcast(&m_cond_loggers);
-  m_queue_mutex_holder = 0;
-  pthread_mutex_unlock(&m_queue_mutex);
-  _flush(&t, &m_recent, false);
-
-  // trim
-  while (m_recent.m_len > m_max_recent) {
-    m_recent.dequeue()->destroy();
+  if (m_log_buf.size()) {
+    _log_safe_write(std::string_view(m_log_buf.data(), m_log_buf.size()));
+    m_log_buf.resize(0);
   }
-
-  m_flush_mutex_holder = 0;
-  pthread_mutex_unlock(&m_flush_mutex);
 }
 
-void Log::_flush(EntryQueue *t, EntryQueue *requeue, bool crash)
+void Log::_flush(EntryVector& t, bool crash)
 {
-  Entry *e;
-  while ((e = t->dequeue()) != NULL) {
-    unsigned sub = e->m_subsys;
-
-    bool should_log = crash || m_subs->get_log_level(sub) >= e->m_prio;
+  long len = 0;
+  if (t.empty()) {
+    assert(m_log_buf.empty());
+    return;
+  }
+  if (crash) {
+    len = t.size();
+  }
+  for (auto& e : t) {
+    auto prio = e.m_prio;
+    auto stamp = e.m_stamp;
+    auto sub = e.m_subsys;
+    auto thread = e.m_thread;
+    auto str = e.strv();
+
+    bool should_log = crash || m_subs->get_log_level(sub) >= prio;
     bool do_fd = m_fd >= 0 && should_log;
-    bool do_syslog = m_syslog_crash >= e->m_prio && should_log;
-    bool do_stderr = m_stderr_crash >= e->m_prio && should_log;
-    bool do_graylog2 = m_graylog_crash >= e->m_prio && should_log;
+    bool do_syslog = m_syslog_crash >= prio && should_log;
+    bool do_stderr = m_stderr_crash >= prio && should_log;
+    bool do_graylog2 = m_graylog_crash >= prio && should_log;
 
-    e->hint_size();
     if (do_fd || do_syslog || do_stderr) {
-      size_t buflen = 0;
-
-      char *buf;
-      size_t buf_size = 80 + e->size();
-      bool need_dynamic = buf_size >= 0x10000; //avoids >64K buffers
-                                              //allocation at stack
-      char buf0[need_dynamic ? 1 : buf_size];
-      if (need_dynamic) {
-        buf = new char[buf_size];
-      } else {
-        buf = buf0;
-      }
+      const std::size_t cur = m_log_buf.size();
+      std::size_t used = 0;
+      const std::size_t allocated = e.size() + 80;
+      m_log_buf.resize(cur + allocated);
 
-      if (crash)
-       buflen += snprintf(buf, buf_size, "%6d> ", -t->m_len);
-      buflen += e->m_stamp.sprintf(buf + buflen, buf_size-buflen);
-      buflen += snprintf(buf + buflen, buf_size-buflen, " %lx %2d ",
-                       (unsigned long)e->m_thread, e->m_prio);
-
-      buflen += e->snprintf(buf + buflen, buf_size - buflen - 1);
-      if (buflen > buf_size - 1) { //paranoid check, buf was declared
-                                  //to hold everything
-        buflen = buf_size - 1;
-        buf[buflen] = 0;
+      char* const start = m_log_buf.data();
+      char* pos = start + cur;
+
+      if (crash) {
+        used += (std::size_t)snprintf(pos + used, allocated - used, "%6ld> ", -(--len));
       }
+      used += (std::size_t)append_time(stamp, pos + used, allocated - used);
+      used += (std::size_t)snprintf(pos + used, allocated - used, " %lx %2d ", (unsigned long)thread, prio);
+      memcpy(pos + used, str.data(), str.size());
+      used += str.size();
+      pos[used] = '\0';
+      ceph_assert((used + 1 /* '\n' */) < allocated);
 
       if (do_syslog) {
-        syslog(LOG_USER|LOG_INFO, "%s", buf);
+        syslog(LOG_USER|LOG_INFO, "%s", pos);
       }
 
       if (do_stderr) {
-        cerr << m_log_stderr_prefix << buf << std::endl;
+        std::cerr << m_log_stderr_prefix << std::string_view(pos, used) << std::endl;
       }
+
+      /* now add newline */
+      pos[used++] = '\n';
+
       if (do_fd) {
-        buf[buflen] = '\n';
-        int r = safe_write(m_fd, buf, buflen+1);
-       if (r != m_fd_last_error) {
-         if (r < 0)
-           cerr << "problem writing to " << m_log_file
-                << ": " << cpp_strerror(r)
-                << std::endl;
-         m_fd_last_error = r;
-       }
+        m_log_buf.resize(cur + used);
+      } else {
+        m_log_buf.resize(0);
+      }
+
+      if (m_log_buf.size() > MAX_LOG_BUF) {
+        _flush_logbuf();
       }
-      if (need_dynamic)
-        delete[] buf;
     }
+
     if (do_graylog2 && m_graylog) {
       m_graylog->log_entry(e);
     }
 
-    requeue->enqueue(e);
+    m_recent.push_back(std::move(e));
   }
+  t.clear();
+
+  _flush_logbuf();
 }
 
 void Log::_log_message(const char *s, bool crash)
@@ -366,100 +321,121 @@ void Log::_log_message(const char *s, bool crash)
     b += '\n';
     int r = safe_write(m_fd, b.c_str(), b.size());
     if (r < 0)
-      cerr << "problem writing to " << m_log_file << ": " << cpp_strerror(r) << std::endl;
+      std::cerr << "problem writing to " << m_log_file << ": " << cpp_strerror(r) << std::endl;
   }
   if ((crash ? m_syslog_crash : m_syslog_log) >= 0) {
     syslog(LOG_USER|LOG_INFO, "%s", s);
   }
 
   if ((crash ? m_stderr_crash : m_stderr_log) >= 0) {
-    cerr << s << std::endl;
+    std::cerr << s << std::endl;
   }
 }
 
 void Log::dump_recent()
 {
-  pthread_mutex_lock(&m_flush_mutex);
+  std::scoped_lock lock1(m_flush_mutex);
   m_flush_mutex_holder = pthread_self();
 
-  pthread_mutex_lock(&m_queue_mutex);
-  m_queue_mutex_holder = pthread_self();
-
-  EntryQueue t;
-  t.swap(m_new);
+  {
+    std::scoped_lock lock2(m_queue_mutex);
+    m_queue_mutex_holder = pthread_self();
+    assert(m_flush.empty());
+    m_flush.swap(m_new);
+    m_queue_mutex_holder = 0;
+  }
 
-  m_queue_mutex_holder = 0;
-  pthread_mutex_unlock(&m_queue_mutex);
-  _flush(&t, &m_recent, false);
+  _flush(m_flush, false);
 
-  EntryQueue old;
   _log_message("--- begin dump of recent events ---", true);
-  _flush(&m_recent, &old, true);
+  std::set<pthread_t> recent_pthread_ids;
+  {
+    EntryVector t;
+    t.insert(t.end(), std::make_move_iterator(m_recent.begin()), std::make_move_iterator(m_recent.end()));
+    m_recent.clear();
+    for (const auto& e : t) {
+      recent_pthread_ids.emplace(e.m_thread);
+    }
+    _flush(t, true);
+  }
 
   char buf[4096];
   _log_message("--- logging levels ---", true);
-  for (vector<Subsystem>::iterator p = m_subs->m_subsys.begin();
-       p != m_subs->m_subsys.end();
-       ++p) {
-    snprintf(buf, sizeof(buf), "  %2d/%2d %s", p->log_level, p->gather_level, p->name.c_str());
+  for (const auto& p : m_subs->m_subsys) {
+    snprintf(buf, sizeof(buf), "  %2d/%2d %s", p.log_level, p.gather_level, p.name);
     _log_message(buf, true);
   }
-
   sprintf(buf, "  %2d/%2d (syslog threshold)", m_syslog_log, m_syslog_crash);
   _log_message(buf, true);
   sprintf(buf, "  %2d/%2d (stderr threshold)", m_stderr_log, m_stderr_crash);
   _log_message(buf, true);
-  sprintf(buf, "  max_recent %9d", m_max_recent);
+
+  _log_message("--- pthread ID / name mapping for recent threads ---", true);
+  for (const auto pthread_id : recent_pthread_ids)
+  {
+    char pthread_name[16] = {0}; //limited by 16B include terminating null byte.
+    ceph_pthread_getname(pthread_id, pthread_name, sizeof(pthread_name));
+    snprintf(buf, sizeof(buf), "  %lx / %s", pthread_id, pthread_name);
+    _log_message(buf, true);
+  }
+
+  sprintf(buf, "  max_recent %9zu", m_max_recent);
   _log_message(buf, true);
-  sprintf(buf, "  max_new    %9d", m_max_new);
+  sprintf(buf, "  max_new    %9zu", m_max_new);
   _log_message(buf, true);
   sprintf(buf, "  log_file %s", m_log_file.c_str());
   _log_message(buf, true);
 
   _log_message("--- end dump of recent events ---", true);
 
+  assert(m_log_buf.empty());
+
   m_flush_mutex_holder = 0;
-  pthread_mutex_unlock(&m_flush_mutex);
 }
 
 void Log::start()
 {
-  assert(!is_started());
-  pthread_mutex_lock(&m_queue_mutex);
-  m_stop = false;
-  pthread_mutex_unlock(&m_queue_mutex);
+  ceph_assert(!is_started());
+  {
+    std::scoped_lock lock(m_queue_mutex);
+    m_stop = false;
+  }
   create("log");
 }
 
 void Log::stop()
 {
-  assert(is_started());
-  pthread_mutex_lock(&m_queue_mutex);
-  m_stop = true;
-  pthread_cond_signal(&m_cond_flusher);
-  pthread_cond_broadcast(&m_cond_loggers);
-  pthread_mutex_unlock(&m_queue_mutex);
-  join();
+  if (is_started()) {
+    {
+      std::scoped_lock lock(m_queue_mutex);
+      m_stop = true;
+      m_cond_flusher.notify_one();
+      m_cond_loggers.notify_all();
+    }
+    join();
+  }
 }
 
 void *Log::entry()
 {
-  pthread_mutex_lock(&m_queue_mutex);
-  m_queue_mutex_holder = pthread_self();
-  while (!m_stop) {
-    if (!m_new.empty()) {
-      m_queue_mutex_holder = 0;
-      pthread_mutex_unlock(&m_queue_mutex);
-      flush();
-      pthread_mutex_lock(&m_queue_mutex);
-      m_queue_mutex_holder = pthread_self();
-      continue;
-    }
+  reopen_log_file();
+  {
+    std::unique_lock lock(m_queue_mutex);
+    m_queue_mutex_holder = pthread_self();
+    while (!m_stop) {
+      if (!m_new.empty()) {
+        m_queue_mutex_holder = 0;
+        lock.unlock();
+        flush();
+        lock.lock();
+        m_queue_mutex_holder = pthread_self();
+        continue;
+      }
 
-    pthread_cond_wait(&m_cond_flusher, &m_queue_mutex);
+      m_cond_flusher.wait(lock);
+    }
+    m_queue_mutex_holder = 0;
   }
-  m_queue_mutex_holder = 0;
-  pthread_mutex_unlock(&m_queue_mutex);
   flush();
   return NULL;
 }