summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorSage Weil <sage@newdream.net>2010-12-11 08:09:17 -0800
committerSage Weil <sage@newdream.net>2010-12-11 08:09:17 -0800
commit239b7677e7a9df86d35cbfb25226c3f1a06771c5 (patch)
tree661a7e07bb20b5e5e2cfab53386bd08123485bac
parent46242586eddcc948f71260f8c1ea2e8b1845a9f8 (diff)
parent027d5bfdca196954937c1b038354ca15b5c699b7 (diff)
downloadceph-239b7677e7a9df86d35cbfb25226c3f1a06771c5.tar.gz
Merge remote branch 'origin/syslog' into unstable
Conflicts: src/mon/Paxos.cc src/osd/PG.cc
-rw-r--r--src/Makefile.am10
-rw-r--r--src/auth/Crypto.cc2
-rw-r--r--src/auth/ExportControl.cc2
-rw-r--r--src/auth/KeyRing.cc2
-rw-r--r--src/auth/RotatingKeyRing.cc2
-rw-r--r--src/auth/cephx/CephxClientHandler.cc4
-rw-r--r--src/auth/cephx/CephxKeyServer.cc4
-rw-r--r--src/auth/cephx/CephxProtocol.cc2
-rw-r--r--src/auth/cephx/CephxServiceHandler.cc2
-rw-r--r--src/client/Client.cc2
-rw-r--r--src/client/SyntheticClient.cc5
-rw-r--r--src/common/ClassHandler.cc2
-rw-r--r--src/common/DoutStreambuf.cc628
-rw-r--r--src/common/DoutStreambuf.h106
-rw-r--r--src/common/Finisher.cc2
-rw-r--r--src/common/LogClient.cc87
-rw-r--r--src/common/LogClient.h110
-rw-r--r--src/common/LogEntry.h (renamed from src/include/LogEntry.h)28
-rw-r--r--src/common/Thread.h12
-rw-r--r--src/common/Timer.cc2
-rw-r--r--src/common/WorkQueue.cc2
-rw-r--r--src/common/common_init.cc7
-rw-r--r--src/common/debug.cc224
-rw-r--r--src/common/debug.h116
-rw-r--r--src/common/likely.h24
-rw-r--r--src/config.cc4
-rw-r--r--src/config.h5
-rw-r--r--src/cosd.cc2
-rw-r--r--src/ebofs/Allocator.cc2
-rw-r--r--src/ebofs/BlockDevice.cc6
-rw-r--r--src/ebofs/BufferCache.cc6
-rw-r--r--src/ebofs/Ebofs.cc2
-rw-r--r--src/ebofs/Table.h4
-rw-r--r--src/ebofs/nodes.h3
-rw-r--r--src/librados.cc2
-rw-r--r--src/mds/AnchorClient.cc2
-rw-r--r--src/mds/AnchorServer.cc2
-rw-r--r--src/mds/CDentry.cc2
-rw-r--r--src/mds/CDir.cc23
-rw-r--r--src/mds/CInode.cc38
-rw-r--r--src/mds/InoTable.cc12
-rw-r--r--src/mds/Locker.cc7
-rw-r--r--src/mds/MDBalancer.cc2
-rw-r--r--src/mds/MDCache.cc41
-rw-r--r--src/mds/MDLog.cc2
-rw-r--r--src/mds/MDS.cc30
-rw-r--r--src/mds/MDS.h2
-rw-r--r--src/mds/MDSTable.cc2
-rw-r--r--src/mds/MDSTableClient.cc2
-rw-r--r--src/mds/MDSTableServer.cc2
-rw-r--r--src/mds/Migrator.cc2
-rw-r--r--src/mds/Server.cc29
-rw-r--r--src/mds/SessionMap.cc2
-rw-r--r--src/mds/SnapServer.cc2
-rw-r--r--src/mds/journal.cc16
-rw-r--r--src/mds/snap.cc3
-rw-r--r--src/messages/MLog.h9
-rw-r--r--src/messages/MLogAck.h2
-rw-r--r--src/mon/AuthMonitor.cc3
-rw-r--r--src/mon/ClassMonitor.cc3
-rw-r--r--src/mon/Elector.cc3
-rw-r--r--src/mon/LogMonitor.cc15
-rw-r--r--src/mon/LogMonitor.h2
-rw-r--r--src/mon/MDSMonitor.cc8
-rw-r--r--src/mon/MonClient.cc2
-rw-r--r--src/mon/Monitor.cc32
-rw-r--r--src/mon/Monitor.h4
-rw-r--r--src/mon/MonitorStore.cc27
-rw-r--r--src/mon/MonmapMonitor.cc3
-rw-r--r--src/mon/OSDMonitor.cc21
-rw-r--r--src/mon/PGMonitor.cc8
-rw-r--r--src/mon/Paxos.cc9
-rw-r--r--src/mon/PaxosService.cc3
-rw-r--r--src/msg/FakeMessenger.cc4
-rw-r--r--src/msg/Message.cc6
-rw-r--r--src/msg/SimpleMessenger.cc26
-rw-r--r--src/os/FileJournal.cc8
-rw-r--r--src/os/FileStore.cc14
-rw-r--r--src/os/JournalingObjectStore.cc2
-rw-r--r--src/os/ObjectStore.cc2
-rw-r--r--src/osbdb/OSBDB.cc4
-rw-r--r--src/osd/OSD.cc77
-rw-r--r--src/osd/OSD.h5
-rw-r--r--src/osd/OSDMap.cc2
-rw-r--r--src/osd/PG.cc77
-rw-r--r--src/osd/RAID4PG.cc3
-rw-r--r--src/osd/ReplicatedPG.cc24
-rw-r--r--src/osdc/Filer.cc2
-rw-r--r--src/osdc/Journaler.cc4
-rw-r--r--src/osdc/ObjectCacher.cc4
-rw-r--r--src/osdc/Objecter.cc2
-rw-r--r--src/streamtest.cc2
-rw-r--r--src/test/TestDoutStreambuf.cc88
-rw-r--r--src/test_trans.cc2
-rw-r--r--src/testcrypto.cc4
-rw-r--r--src/tools/ceph.cc2
-rw-r--r--src/tools/cmd.cc1
97 files changed, 1445 insertions, 720 deletions
diff --git a/src/Makefile.am b/src/Makefile.am
index 8c16203f368..8d6ee4b4b0e 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -202,6 +202,10 @@ testencoding_SOURCES = test/TestEncoding.cc
testencoding_LDADD = libceph.la libcrush.la -lpthread -lm -lcrypto
bin_PROGRAMS += testencoding
+testdout_streambuf_SOURCES = test/TestDoutStreambuf.cc
+testdout_streambuf_LDADD = libceph.la libcrush.la -lpthread
+bin_PROGRAMS += testdout_streambuf
+
endif
# librados
@@ -447,7 +451,8 @@ libcommon_files = \
include/ceph_frag.cc \
config.cc \
common/page.cc \
- common/lockdep.cc
+ common/lockdep.cc \
+ common/DoutStreambuf.cc
libcrush_a_SOURCES = \
crush/builder.c \
@@ -579,11 +584,13 @@ noinst_HEADERS = \
common/Mutex.h\
common/RWLock.h\
common/Semaphore.h\
+ common/DoutStreambuf.h\
common/Thread.h\
common/Throttle.h\
common/Timer.h\
common/tls.h\
common/WorkQueue.h\
+ common/LogEntry.h\
common/LogClient.h\
config.h\
crush/CrushWrapper.h\
@@ -601,7 +608,6 @@ noinst_HEADERS = \
include/Context.h\
include/CompatSet.h\
include/Distribution.h\
- include/LogEntry.h\
include/Spinlock.h\
include/addr_parsing.h\
include/assert.h\
diff --git a/src/auth/Crypto.cc b/src/auth/Crypto.cc
index 467e086b3c9..a561e7dbc33 100644
--- a/src/auth/Crypto.cc
+++ b/src/auth/Crypto.cc
@@ -137,7 +137,7 @@ int CryptoAES::encrypt(bufferptr& secret, const bufferlist& in, bufferlist& out)
const unsigned char *in_buf;
if (secret.length() < AES_KEY_LEN) {
- derr(0) << "key is too short" << dendl;
+ dout(0) << "key is too short" << dendl;
return false;
}
string ciphertext;
diff --git a/src/auth/ExportControl.cc b/src/auth/ExportControl.cc
index 7339060c0b4..0c3790b46d3 100644
--- a/src/auth/ExportControl.cc
+++ b/src/auth/ExportControl.cc
@@ -388,7 +388,7 @@ void GroupEntry::init(ConfFile *cf, const char *section, const char *options)
} else if (strcmp(op, "ro") == 0) {
set_readonly(true);
} else {
- derr(0) << "Error: unknown option '" << op << "'" << dendl;
+ dout(0) << "Error: unknown option '" << op << "'" << dendl;
}
op = strsep(&tmp, ", ");
}
diff --git a/src/auth/KeyRing.cc b/src/auth/KeyRing.cc
index 025cf4cba22..5d00faaf49a 100644
--- a/src/auth/KeyRing.cc
+++ b/src/auth/KeyRing.cc
@@ -23,7 +23,7 @@
#define DOUT_SUBSYS auth
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "auth: "
+#define dout_prefix *_dout << "auth: "
using namespace std;
diff --git a/src/auth/RotatingKeyRing.cc b/src/auth/RotatingKeyRing.cc
index 725b558b7cf..1756f34591d 100644
--- a/src/auth/RotatingKeyRing.cc
+++ b/src/auth/RotatingKeyRing.cc
@@ -10,7 +10,7 @@
#define DOUT_SUBSYS auth
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "auth: "
+#define dout_prefix *_dout << "auth: "
bool RotatingKeyRing::need_new_secrets()
diff --git a/src/auth/cephx/CephxClientHandler.cc b/src/auth/cephx/CephxClientHandler.cc
index 098a9ddb1a6..09d3fca6dcf 100644
--- a/src/auth/cephx/CephxClientHandler.cc
+++ b/src/auth/cephx/CephxClientHandler.cc
@@ -24,7 +24,7 @@
#define DOUT_SUBSYS auth
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "cephx client: "
+#define dout_prefix *_dout << "cephx client: "
int CephxClientHandler::build_request(bufferlist& bl)
@@ -151,7 +151,7 @@ int CephxClientHandler::handle_response(int ret, bufferlist::iterator& indata)
if (decode_decrypt(secrets, secret_key, indata) == 0) {
rotating_secrets->set_secrets(secrets);
} else {
- derr(0) << "could not set rotating key: decode_decrypt failed" << dendl;
+ dout(0) << "could not set rotating key: decode_decrypt failed" << dendl;
}
}
}
diff --git a/src/auth/cephx/CephxKeyServer.cc b/src/auth/cephx/CephxKeyServer.cc
index fc17ddfea17..1188f3bc00f 100644
--- a/src/auth/cephx/CephxKeyServer.cc
+++ b/src/auth/cephx/CephxKeyServer.cc
@@ -21,7 +21,7 @@
#define DOUT_SUBSYS auth
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "cephx keyserverdata: "
+#define dout_prefix *_dout << "cephx keyserverdata: "
bool KeyServerData::get_service_secret(uint32_t service_id, ExpiringCryptoKey& secret, uint64_t& secret_id)
{
@@ -118,7 +118,7 @@ bool KeyServerData::get_caps(EntityName& name, string& type, AuthCapsInfo& caps_
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "cephx keyserver: "
+#define dout_prefix *_dout << "cephx keyserver: "
KeyServer::KeyServer() : lock("KeyServer::lock")
diff --git a/src/auth/cephx/CephxProtocol.cc b/src/auth/cephx/CephxProtocol.cc
index a733d79d9bf..6986e7924c0 100644
--- a/src/auth/cephx/CephxProtocol.cc
+++ b/src/auth/cephx/CephxProtocol.cc
@@ -6,7 +6,7 @@
#define DOUT_SUBSYS auth
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "cephx: "
+#define dout_prefix *_dout << "cephx: "
diff --git a/src/auth/cephx/CephxServiceHandler.cc b/src/auth/cephx/CephxServiceHandler.cc
index ee039d5534e..0dd523339db 100644
--- a/src/auth/cephx/CephxServiceHandler.cc
+++ b/src/auth/cephx/CephxServiceHandler.cc
@@ -27,7 +27,7 @@
#define DOUT_SUBSYS auth
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "cephx server " << entity_name << ": "
+#define dout_prefix *_dout << "cephx server " << entity_name << ": "
int CephxServiceHandler::start_session(EntityName& name, bufferlist::iterator& indata, bufferlist& result_bl, AuthCapsInfo& caps)
{
diff --git a/src/client/Client.cc b/src/client/Client.cc
index c9876fc5b1b..057aa8ee32e 100644
--- a/src/client/Client.cc
+++ b/src/client/Client.cc
@@ -63,7 +63,7 @@ using namespace std;
#define DOUT_SUBSYS client
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "client" << whoami << " "
+#define dout_prefix *_dout << "client" << whoami << " "
#define tout if (g_conf.client_trace) traceout
diff --git a/src/client/SyntheticClient.cc b/src/client/SyntheticClient.cc
index 97c228c479d..e19faaf5ff1 100644
--- a/src/client/SyntheticClient.cc
+++ b/src/client/SyntheticClient.cc
@@ -38,7 +38,7 @@ using namespace std;
#define DOUT_SUBSYS client
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "client" << (whoami >= 0 ? whoami:client->get_nodeid()) << " "
+#define dout_prefix *_dout << "client" << (whoami >= 0 ? whoami:client->get_nodeid()) << " "
// traces
//void trace_include(SyntheticClient *syn, Client *cl, string& prefix);
@@ -1614,7 +1614,7 @@ int SyntheticClient::dump_placement(string& fn) {
struct stat stbuf;
int lstat_result = client->lstat(fn.c_str(), &stbuf);
if (lstat_result < 0) {
- derr(0) << "lstat error for file " << fn << dendl;
+ dout(0) << "lstat error for file " << fn << dendl;
return lstat_result;
}
@@ -2201,7 +2201,6 @@ int SyntheticClient::create_objects(int nobj, int osize, int inflight)
lock.Unlock();
dout(5) << "create_objects done" << dendl;
- derr(0) << "create_objects done" << dendl;
return 0;
}
diff --git a/src/common/ClassHandler.cc b/src/common/ClassHandler.cc
index a2a13075bf3..405ab7388e4 100644
--- a/src/common/ClassHandler.cc
+++ b/src/common/ClassHandler.cc
@@ -14,7 +14,7 @@
#define DOUT_SUBSYS osd
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl
+#define dout_prefix *_dout
static ClassHandler::ClassData null_cls_data;
diff --git a/src/common/DoutStreambuf.cc b/src/common/DoutStreambuf.cc
new file mode 100644
index 00000000000..4279e4fbebd
--- /dev/null
+++ b/src/common/DoutStreambuf.cc
@@ -0,0 +1,628 @@
+// -*- 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) 2010 Dreamhost
+ *
+ * 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 "config.h"
+#include "common/DoutStreambuf.h"
+#include "common/errno.h"
+#include "common/Mutex.h"
+
+#include <values.h>
+#include <assert.h>
+#include <errno.h>
+#include <fstream>
+#include <iostream>
+#include <memory>
+#include <sstream>
+#include <streambuf>
+#include <string.h>
+#include <syslog.h>
+
+///////////////////////////// Globals /////////////////////////////
+// TODO: get rid of this lock using thread-local storage
+extern Mutex _dout_lock;
+
+//////////////////////// Helper functions //////////////////////////
+// Try a 0-byte write to a file descriptor to see if it open.
+static bool fd_is_open(int fd)
+{
+ char buf;
+ ssize_t res = TEMP_FAILURE_RETRY(write(fd, &buf, 0));
+ return (res == 0);
+}
+
+static bool empty(const char *str)
+{
+ if (!str)
+ return true;
+ if (!str[0])
+ return true;
+ return false;
+}
+
+static string cpp_str(const char *str)
+{
+ if (!str)
+ return "(NULL)";
+ if (str[0] == '\0')
+ return "(empty)";
+ return str;
+}
+
+static std::string normalize_relative(const char *from)
+{
+ if (from[0] == '/')
+ return string(from);
+
+ char c[512];
+ char *cwd = getcwd(c, sizeof(c));
+ ostringstream oss;
+ oss << cwd << "/" << from;
+ return oss.str();
+}
+
+/* Complain about errors even without a logfile */
+static void primitive_log(const std::string &str)
+{
+ std::cerr << str;
+ std::cerr.flush();
+ syslog(LOG_USER | LOG_NOTICE, "%s", str.c_str());
+}
+
+static inline bool prio_is_visible_on_stderr(int prio)
+{
+ return prio <= 5;
+}
+
+static inline int dout_prio_to_syslog_prio(int prio)
+{
+ if (prio <= 3)
+ return LOG_CRIT;
+ if (prio <= 5)
+ return LOG_ERR;
+ if (prio <= 15)
+ return LOG_WARNING;
+ if (prio <= 30)
+ return LOG_NOTICE;
+ if (prio <= 40)
+ return LOG_INFO;
+ return LOG_DEBUG;
+}
+
+static int safe_write(int fd, const char *buf, signed int len)
+{
+ int res;
+
+ assert(len != 0);
+ while (1) {
+ res = write(fd, buf, len);
+ if (res < 0) {
+ int err = errno;
+ if (err != EINTR) {
+ ostringstream oss;
+ oss << __func__ << ": failed to write to fd " << fd << ": "
+ << cpp_strerror(err) << "\n";
+ primitive_log(oss.str());
+ return err;
+ }
+ }
+ len -= res;
+ buf += res;
+ if (len <= 0)
+ return 0;
+ }
+}
+
+static std::string get_basename(const std::string &filename)
+{
+ size_t last_slash = filename.find_last_of("/");
+ if (last_slash == std::string::npos)
+ return filename;
+ return filename.substr(last_slash + 1);
+}
+
+static std::string get_dirname(const std::string &filename)
+{
+ size_t last_slash = filename.find_last_of("/");
+ if (last_slash == std::string::npos)
+ return ".";
+ if (last_slash == 0)
+ return filename;
+ return filename.substr(0, last_slash);
+}
+
+static int create_symlink(string oldpath, const string &newpath)
+{
+ // Create relative symlink if the files are in the same directory
+ if (get_dirname(oldpath) == get_dirname(newpath)) {
+ oldpath = string("./") + get_basename(oldpath);
+ }
+
+ while (1) {
+ if (::symlink(oldpath.c_str(), newpath.c_str()) == 0)
+ return 0;
+ int err = errno;
+ if (err == EEXIST) {
+ // Handle EEXIST
+ if (::unlink(newpath.c_str())) {
+ err = errno;
+ ostringstream oss;
+ oss << __func__ << ": failed to remove '" << newpath << "': "
+ << cpp_strerror(err) << "\n";
+ primitive_log(oss.str());
+ return err;
+ }
+ }
+ else {
+ // Other errors
+ ostringstream oss;
+ oss << __func__ << ": failed to symlink(oldpath='" << oldpath
+ << "', newpath='" << newpath << "'): " << cpp_strerror(err) << "\n";
+ primitive_log(oss.str());
+ return err;
+ }
+ }
+}
+
+///////////////////////////// DoutStreambuf /////////////////////////////
+template <typename charT, typename traits>
+DoutStreambuf<charT, traits>::DoutStreambuf()
+ : flags(0), ofd(-1)
+{
+ // Initialize get pointer to zero so that underflow is called on the first read.
+ this->setg(0, 0, 0);
+
+ // Initialize output_buffer
+ _clear_output_buffer();
+}
+
+// This function is called when the output buffer is filled.
+// In this function, the buffer should be written to wherever it should
+// be written to (in this case, the streambuf object that this is controlling).
+template <typename charT, typename traits>
+typename DoutStreambuf<charT, traits>::int_type
+DoutStreambuf<charT, traits>::overflow(DoutStreambuf<charT, traits>::int_type c)
+{
+ {
+ // zero-terminate the buffer
+ charT* end_ptr = this->pptr();
+ *end_ptr++ = '\0';
+ *end_ptr++ = '\0';
+// char buf[1000];
+// hex2str(obuf, end_ptr - obuf, buf, sizeof(buf));
+// printf("overflow buffer: '%s'\n", buf);
+ }
+
+ // Loop over all lines in the buffer.
+ int prio = 100;
+ charT* start = obuf;
+ while (true) {
+ char* end = strchrnul(start, '\n');
+ if (start == end) {
+ if (*start == '\0')
+ break;
+ // skip zero-length lines
+ ++start;
+ continue;
+ }
+ if (*start == '\1') {
+ // Decode some control characters
+ ++start;
+ unsigned char tmp = *((unsigned char*)start);
+ prio = tmp - 11;
+ ++start;
+ }
+ *end = '\n';
+ char next = *(end+1);
+ *(end+1) = '\0';
+
+ // Now 'start' points to a NULL-terminated string, which we want to
+ // output with priority 'prio'
+ int len = strlen(start);
+ if (flags & DOUTSB_FLAG_SYSLOG) {
+ //printf("syslogging: '%s' len=%d\n", start, len);
+ syslog(LOG_USER | dout_prio_to_syslog_prio(prio), "%s", start);
+ }
+ if (flags & DOUTSB_FLAG_STDOUT) {
+ // Just write directly out to the stdout fileno. There's no point in
+ // using something like fputs to write to a temporary buffer,
+ // because we would just have to flush that temporary buffer
+ // immediately.
+ if (safe_write(STDOUT_FILENO, start, len))
+ flags &= ~DOUTSB_FLAG_STDOUT;
+ }
+ if (flags & DOUTSB_FLAG_STDERR) {
+ // Only write to stderr if the message is important enough.
+ if (prio_is_visible_on_stderr(prio)) {
+ if (safe_write(STDERR_FILENO, start, len))
+ flags &= ~DOUTSB_FLAG_STDERR;
+ }
+ }
+ if (flags & DOUTSB_FLAG_OFILE) {
+ if (safe_write(ofd, start, len))
+ flags &= ~DOUTSB_FLAG_OFILE;
+ }
+
+ *(end+1) = next;
+ start = end + 1;
+ }
+
+ _clear_output_buffer();
+
+ // A value different than EOF (or traits::eof() for other traits) signals success.
+ // If the function fails, either EOF (or traits::eof() for other traits) is returned or an
+ // exception is thrown.
+ return traits_ty::not_eof(c);
+}
+
+template <typename charT, typename traits>
+void DoutStreambuf<charT, traits>::handle_stderr_closed()
+{
+ assert(_dout_lock.is_locked());
+ flags &= ~DOUTSB_FLAG_STDERR;
+}
+
+template <typename charT, typename traits>
+void DoutStreambuf<charT, traits>::handle_stdout_closed()
+{
+ assert(_dout_lock.is_locked());
+ flags &= ~DOUTSB_FLAG_STDOUT;
+}
+
+template <typename charT, typename traits>
+void DoutStreambuf<charT, traits>::read_global_config()
+{
+ assert(_dout_lock.is_locked());
+ flags = 0;
+
+ if (g_conf.log_to_syslog) {
+ flags |= DOUTSB_FLAG_SYSLOG;
+ }
+ if (g_conf.log_to_stdout) {
+ if (fd_is_open(STDOUT_FILENO)) {
+ flags |= DOUTSB_FLAG_STDOUT;
+ }
+ }
+ if (fd_is_open(STDERR_FILENO)) {
+ flags |= DOUTSB_FLAG_STDERR;
+ }
+ if (g_conf.log_to_file) {
+ if (_read_ofile_config()) {
+ flags |= DOUTSB_FLAG_OFILE;
+ }
+ }
+}
+
+template <typename charT, typename traits>
+void DoutStreambuf<charT, traits>::
+set_flags(int flags_)
+{
+ assert(_dout_lock.is_locked());
+ flags = flags_;
+}
+
+template <typename charT, typename traits>
+void DoutStreambuf<charT, traits>::
+set_prio(int prio)
+{
+ charT* p = this->pptr();
+ *p++ = '\1';
+ unsigned char val = (prio + 11);
+ *p++ = val;
+ this->pbump(2);
+}
+
+template <typename charT, typename traits>
+int DoutStreambuf<charT, traits>::handle_pid_change()
+{
+ assert(_dout_lock.is_locked());
+ if (!(flags & DOUTSB_FLAG_OFILE))
+ return 0;
+
+ string new_opath(_calculate_opath());
+ if (opath == new_opath)
+ return 0;
+
+ if (!isym_path.empty()) {
+ // Re-create the instance symlink
+ int ret = create_symlink(new_opath, isym_path);
+ if (ret) {
+ ostringstream oss;
+ oss << __func__ << ": failed to (re)create instance symlink";
+ primitive_log(oss.str());
+ return ret;
+ }
+ }
+
+ if (!rsym_path.empty()) {
+ // Re-create the rank symlink
+ int ret = create_symlink(new_opath, rsym_path);
+ if (ret) {
+ ostringstream oss;
+ oss << __func__ << ": failed to (re)create rank symlink";
+ primitive_log(oss.str());
+ return ret;
+ }
+ }
+
+ int ret = ::rename(opath.c_str(), new_opath.c_str());
+ if (ret) {
+ int err = errno;
+ ostringstream oss;
+ oss << __func__ << ": failed to rename '" << opath << "' to "
+ << "'" << new_opath << "': " << cpp_strerror(err) << "\n";
+ primitive_log(oss.str());
+ return err;
+ }
+
+ opath = new_opath;
+
+ return 0;
+}
+
+template <typename charT, typename traits>
+int DoutStreambuf<charT, traits>::create_rank_symlink(int n)
+{
+ Mutex::Locker l(_dout_lock);
+
+ if (!(flags & DOUTSB_FLAG_OFILE))
+ return 0;
+
+ ostringstream rss;
+ std::string symlink_dir(_get_symlink_dir());
+ rss << symlink_dir << "/" << g_conf.type << "." << n;
+ string rsym_path_(rss.str());
+ int ret = create_symlink(opath, rsym_path_);
+ if (ret) {
+ ostringstream oss;
+ oss << __func__ << ": failed to create rank symlink with n = "
+ << n << "\n";
+ primitive_log(oss.str());
+ return ret;
+ }
+
+ rsym_path = rsym_path_;
+ return 0;
+}
+
+template <typename charT, typename traits>
+std::string DoutStreambuf<charT, traits>::config_to_str() const
+{
+ assert(_dout_lock.is_locked());
+ ostringstream oss;
+ oss << "g_conf.log_to_syslog = " << g_conf.log_to_syslog << "\n";
+ oss << "g_conf.log_to_stdout = " << g_conf.log_to_stdout << "\n";
+ oss << "g_conf.log_to_file = " << g_conf.log_to_file << "\n";
+ oss << "g_conf.log_file = '" << cpp_str(g_conf.log_file) << "'\n";
+ oss << "g_conf.log_dir = '" << cpp_str(g_conf.log_dir) << "'\n";
+ oss << "g_conf.g_conf.log_per_instance = '"
+ << g_conf.log_per_instance << "'\n";
+ oss << "flags = 0x" << std::hex << flags << std::dec << "\n";
+ oss << "ofd = " << ofd << "\n";
+ oss << "opath = '" << opath << "'\n";
+ oss << "isym_path = '" << isym_path << "'\n";
+ oss << "rsym_path = '" << rsym_path << "'\n";
+ oss << "log_sym_history = " << g_conf.log_sym_history << "\n";
+ return oss.str();
+}
+
+// This is called to flush the buffer.
+// This is called when we're done with the file stream (or when .flush() is called).
+template <typename charT, typename traits>
+typename DoutStreambuf<charT, traits>::int_type
+DoutStreambuf<charT, traits>::sync()
+{
+ //std::cout << "flush!" << std::endl;
+ typename DoutStreambuf<charT, traits>::int_type
+ ret(this->overflow(traits_ty::eof()));
+ if (ret == traits_ty::eof())
+ return -1;
+
+ return 0;
+}
+
+template <typename charT, typename traits>
+typename DoutStreambuf<charT, traits>::int_type
+DoutStreambuf<charT, traits>::underflow()
+{
+ // We can't read from this
+ // TODO: some more elegant way of preventing callers from trying to get input from this stream
+ assert(0);
+}
+
+template <typename charT, typename traits>
+void DoutStreambuf<charT, traits>::_clear_output_buffer()
+{
+ // Set up the put pointer.
+ // Overflow is called when this buffer is filled
+ this->setp(obuf, obuf + OBUF_SZ - 5);
+}
+
+template <typename charT, typename traits>
+std::string DoutStreambuf<charT, traits>::_calculate_opath() const
+{
+ assert(_dout_lock.is_locked());
+
+ // If g_conf.log_file was specified, that takes the highest priority
+ if (!empty(g_conf.log_file)) {
+ return normalize_relative(g_conf.log_file);
+ }
+
+ string log_dir;
+ if (empty(g_conf.log_dir))
+ log_dir = normalize_relative(".");
+ else
+ log_dir = normalize_relative(g_conf.log_dir);
+
+ if (g_conf.log_per_instance) {
+ char hostname[255];
+ memset(hostname, 0, sizeof(hostname));
+ int ret = gethostname(hostname, sizeof(hostname));
+ if (ret) {
+ int err = errno;
+ ostringstream oss;
+ oss << __func__ << ": error calling gethostname: " << cpp_strerror(err) << "\n";
+ primitive_log(oss.str());
+ return "";
+ }
+ ostringstream oss;
+ oss << log_dir << "/" << hostname << "." << getpid();
+ return oss.str();
+ }
+ else {
+ ostringstream oss;
+ oss << log_dir << "/" << g_conf.type << "." << g_conf.id << ".log";
+ return oss.str();
+ }
+}
+
+template <typename charT, typename traits>
+std::string DoutStreambuf<charT, traits>::_get_symlink_dir() const
+{
+ if (!empty(g_conf.log_sym_dir))
+ return normalize_relative(g_conf.log_sym_dir);
+ else
+ return get_dirname(opath);
+}
+
+template <typename charT, typename traits>
+bool DoutStreambuf<charT, traits>::_read_ofile_config()
+{
+ int ret;
+
+ isym_path = "";
+ rsym_path = "";
+ opath = _calculate_opath();
+ if (opath.empty()) {
+ ostringstream oss;
+ oss << __func__ << ": _calculate_opath failed.\n";
+ primitive_log(oss.str());
+ return false;
+ }
+
+ if (empty(g_conf.log_file) && g_conf.log_per_instance) {
+ // Calculate instance symlink path (isym_path)
+ ostringstream iss;
+ std::string symlink_dir(_get_symlink_dir());
+ iss << symlink_dir << "/" << g_conf.type << "." << g_conf.id;
+ isym_path = iss.str();
+
+ // Rotate isym_path
+ ret = _rotate_files(isym_path);
+ if (ret) {
+ ostringstream oss;
+ oss << __func__ << ": failed to rotate instance symlinks";
+ primitive_log(oss.str());
+ return ret;
+ }
+
+ // Create isym_path
+ ret = create_symlink(opath, isym_path);
+ if (ret) {
+ ostringstream oss;
+ oss << __func__ << ": failed to create instance symlink";
+ primitive_log(oss.str());
+ return ret;
+ }
+ }
+
+ assert(ofd == -1);
+ ofd = open(opath.c_str(),
+ O_CREAT | O_WRONLY | O_APPEND, S_IWUSR | S_IRUSR);
+ if (ofd < 0) {
+ int err = errno;
+ ostringstream oss;
+ oss << "failed to open log file '" << opath << "': "
+ << cpp_strerror(err) << "\n";
+ primitive_log(oss.str());
+ return false;
+ }
+
+ return true;
+}
+
+template <typename charT, typename traits>
+int DoutStreambuf<charT, traits>::_rotate_files(const std::string &base)
+{
+ // Given a file name base, and a directory like this:
+ // base
+ // base.1
+ // base.2
+ // base.3
+ // base.4
+ // unrelated_blah
+ // unrelated_blah.1
+ //
+ // We'll take the following actions:
+ // base rename to base.1
+ // base.1 rename to base.2
+ // base.2 rename to base.3
+ // base.3 (unlink)
+ // base.4 (unlink)
+ // unrelated_blah (do nothing)
+ // unrelated_blah.1 (do nothing)
+
+ signed int i;
+ for (i = -1; i < INT_MAX; ++i) {
+ ostringstream oss;
+ oss << base;
+ if (i != -1)
+ oss << "." << i;
+ string path(oss.str());
+
+ if (::access(path.c_str(), R_OK | W_OK))
+ break;
+ }
+ signed int max_symlink = i - 1;
+
+ for (signed int j = max_symlink; j >= -1; --j) {
+ ostringstream oss;
+ oss << base;
+ if (j != -1)
+ oss << "." << j;
+ string path(oss.str());
+
+ signed int k = j + 1;
+ if (k >= g_conf.log_sym_history) {
+ if (::unlink(path.c_str())) {
+ int err = errno;
+ ostringstream ess;
+ ess << __func__ << ": failed to unlink '" << path << "': "
+ << cpp_strerror(err) << "\n";
+ primitive_log(ess.str());
+ return err;
+ }
+ //*_dout << "---- " << getpid() << " removed " << path << " ----"
+ // << std::endl;
+ }
+ else {
+ ostringstream pss;
+ pss << base << "." << k;
+ string new_path(pss.str());
+ if (::rename(path.c_str(), new_path.c_str())) {
+ int err = errno;
+ ostringstream ess;
+ ess << __func__ << ": failed to rename '" << path << "' to "
+ << "'" << new_path << "': " << cpp_strerror(err) << "\n";
+ primitive_log(ess.str());
+ return err;
+ }
+// *_dout << "---- " << getpid() << " renamed " << path << " -> "
+// << newpath << " ----" << std::endl;
+ }
+ }
+ return 0;
+}
+
+// Explicit template instantiation
+template class DoutStreambuf <char>;
diff --git a/src/common/DoutStreambuf.h b/src/common/DoutStreambuf.h
new file mode 100644
index 00000000000..76968198b24
--- /dev/null
+++ b/src/common/DoutStreambuf.h
@@ -0,0 +1,106 @@
+// -*- 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) 2010 Dreamhost
+ *
+ * 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.
+ *
+ */
+
+/*
+ * DoutStreambuf
+ *
+ * The stream buffer used by dout
+ */
+#ifndef CEPH_DOUT_STREAMBUF_H
+#define CEPH_DOUT_STREAMBUF_H
+
+#include <iosfwd>
+#include <string>
+
+template <typename charT, typename traits = std::char_traits<charT> >
+class DoutStreambuf : public std::basic_streambuf<charT, traits>
+{
+public:
+ enum dout_streambuf_flags_t {
+ DOUTSB_FLAG_SYSLOG = 0x01,
+ DOUTSB_FLAG_STDOUT = 0x02,
+ DOUTSB_FLAG_STDERR = 0x04,
+ DOUTSB_FLAG_OFILE = 0x08,
+ };
+
+ typedef traits traits_ty;
+ typedef typename traits_ty::int_type int_type;
+ typedef typename traits_ty::pos_type pos_type;
+ typedef typename traits_ty::off_type off_type;
+
+ // The size of the output buffer.
+ static const size_t OBUF_SZ = 32000;
+
+ DoutStreambuf();
+
+ // Call when you close stderr. Not strictly necessary, since we would get an
+ // error the next time we tried to write to stdedrr. But nicer than waiting
+ // for the error to happen.
+ void handle_stderr_closed();
+
+ // Call when you close stdout.
+ void handle_stdout_closed();
+
+ // Set the flags based on the global configuration
+ void read_global_config();
+
+ // Set the flags directly (for debug use only)
+ void set_flags(int flags_);
+
+ // Set the priority of the messages being put into the stream
+ void set_prio(int prio);
+
+ // Call after calling daemon()
+ // A change in the process ID sometimes requires us to change our output
+ // path name.
+ int handle_pid_change();
+
+ // Create a rank symlink to the log file
+ int create_rank_symlink(int n);
+
+ std::string config_to_str() const;
+
+protected:
+ // Called when the buffer fills up
+ virtual int_type overflow(int_type c);
+
+ // Called when the buffer is flushed
+ virtual int_type sync();
+
+ // Called when we try to read, but there are no more chars in the buffer
+ virtual int_type underflow();
+
+private:
+ void _clear_output_buffer();
+ std::string _calculate_opath() const;
+ std::string _get_symlink_dir() const;
+ bool _read_ofile_config();
+ int _rotate_files(const std::string &base);
+
+ // Output buffer
+ charT obuf[OBUF_SZ];
+
+ // Output flags
+ int flags;
+
+ // ofile stuff
+ int ofd;
+ std::string opath;
+
+ // symlinks
+ std::string isym_path;
+ std::string rsym_path;
+};
+
+#endif
diff --git a/src/common/Finisher.cc b/src/common/Finisher.cc
index 05fba0c0b3e..7315b9c4f48 100644
--- a/src/common/Finisher.cc
+++ b/src/common/Finisher.cc
@@ -5,7 +5,7 @@
#include "common/debug.h"
#define DOUT_SUBSYS finisher
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "finisher(" << this << ") "
+#define dout_prefix *_dout << "finisher(" << this << ") "
void Finisher::start()
{
diff --git a/src/common/LogClient.cc b/src/common/LogClient.cc
index 921adcc5485..26f054411d0 100644
--- a/src/common/LogClient.cc
+++ b/src/common/LogClient.cc
@@ -27,6 +27,7 @@
#include <iostream>
#include <errno.h>
#include <sys/stat.h>
+#include <syslog.h>
#ifdef DARWIN
#include <sys/param.h>
@@ -37,25 +38,59 @@
#include "config.h"
-void LogClient::log(log_type type, const char *s)
+/*
+ * Given a clog log_type, return the equivalent syslog priority
+ */
+static inline int clog_type_to_syslog_prio(clog_type t)
+{
+ switch (t) {
+ case CLOG_DEBUG:
+ return LOG_DEBUG;
+ case CLOG_INFO:
+ return LOG_INFO;
+ case CLOG_WARN:
+ return LOG_WARNING;
+ case CLOG_ERROR:
+ return LOG_ERR;
+ case CLOG_SEC:
+ return LOG_CRIT;
+ default:
+ assert(0);
+ return 0;
+ }
+}
+
+LogClientTemp::LogClientTemp(clog_type type_, LogClient &parent_)
+ : type(type_), parent(parent_)
{
- string str(s);
- log(type, str);
}
-void LogClient::log(log_type type, stringstream& ss)
+LogClientTemp::LogClientTemp(const LogClientTemp &rhs)
+ : type(rhs.type), parent(rhs.parent)
+{
+ // don't want to-- nor can we-- copy the ostringstream
+}
+
+LogClientTemp::~LogClientTemp()
+{
+ if (ss.peek() != EOF)
+ parent.do_log(type, ss);
+}
+
+void LogClient::do_log(clog_type type, std::stringstream& ss)
{
while (!ss.eof()) {
string s;
getline(ss, s);
- log(type, s);
+ if (!s.empty())
+ do_log(type, s);
}
}
-void LogClient::log(log_type type, string& s)
+void LogClient::do_log(clog_type type, const std::string& s)
{
Mutex::Locker l(log_lock);
- dout(0) << "log " << (log_type)type << " : " << s << dendl;
+ dout(0) << "log " << type << " : " << s << dendl;
LogEntry e;
e.who = messenger->get_myinst();
e.stamp = g_clock.now();
@@ -76,6 +111,33 @@ void LogClient::send_log()
void LogClient::_send_log()
{
+ if (g_conf.clog_to_syslog)
+ _send_log_to_syslog();
+ if (g_conf.clog_to_monitors)
+ _send_log_to_monitors();
+}
+
+void LogClient::_send_log_to_syslog()
+{
+ std::deque<LogEntry>::const_reverse_iterator rbegin = log_queue.rbegin();
+ std::deque<LogEntry>::const_reverse_iterator rend = log_queue.rend();
+ for (std::deque<LogEntry>::const_reverse_iterator a = rbegin; a != rend; ++a) {
+ const LogEntry entry(*a);
+ if (entry.seq < last_syslog)
+ break;
+ ostringstream oss;
+ oss << entry;
+ string str(oss.str());
+ syslog(clog_type_to_syslog_prio(entry.type) | LOG_USER, "%s", str.c_str());
+ }
+ if (rbegin != rend) {
+ const LogEntry entry(*rbegin);
+ last_syslog = entry.seq;
+ }
+}
+
+void LogClient::_send_log_to_monitors()
+{
if (log_queue.empty())
return;
MLog *log = new MLog(monmap->get_fsid(), log_queue);
@@ -96,9 +158,14 @@ void LogClient::handle_log_ack(MLogAck *m)
dout(10) << "handle_log_ack " << *m << dendl;
version_t last = m->last;
- while (log_queue.size() && log_queue.begin()->seq <= last) {
- dout(10) << " logged " << log_queue.front() << dendl;
- log_queue.pop_front();
+
+ deque<LogEntry>::iterator q = log_queue.begin();
+ while (q != log_queue.end()) {
+ const LogEntry &entry(*q);
+ if (entry.seq > last)
+ break;
+ dout(10) << " logged " << entry << dendl;
+ q = log_queue.erase(q);
}
m->put();
}
diff --git a/src/common/LogClient.h b/src/common/LogClient.h
index a6584739f64..c63bddd7302 100644
--- a/src/common/LogClient.h
+++ b/src/common/LogClient.h
@@ -15,51 +15,107 @@
#ifndef CEPH_LOGCLIENT_H
#define CEPH_LOGCLIENT_H
-#include "msg/Dispatcher.h"
-
+#include "common/LogEntry.h"
#include "common/Mutex.h"
-#include "include/LogEntry.h"
+#include "msg/Dispatcher.h"
+#include <iosfwd>
#include <sstream>
-class Messenger;
+class LogClient;
class MLog;
class MLogAck;
-class MonMap;
+class Messenger;
class MonClient;
+class MonMap;
-class LogClient : public Dispatcher {
- Messenger *messenger;
- MonMap *monmap;
- MonClient *monc;
+class LogClientTemp
+{
+public:
+ LogClientTemp(clog_type type_, LogClient &parent_);
+ LogClientTemp(const LogClientTemp &rhs);
+ ~LogClientTemp();
+
+ template<typename T>
+ std::ostream& operator<<(const T& rhs)
+ {
+ return ss << rhs;
+ }
+
+private:
+ clog_type type;
+ LogClient &parent;
+ stringstream ss;
+};
+class LogClient : public Dispatcher
+{
+public:
+ enum logclient_flag_t {
+ NO_FLAGS = 0,
+ FLAG_SYNC = 0x1,
+ };
+
+ LogClient(Messenger *m, MonMap *mm, MonClient *mc, enum logclient_flag_t flags) :
+ messenger(m), monmap(mm), monc(mc), is_synchronous(flags & FLAG_SYNC),
+ log_lock("LogClient::log_lock"), last_log(0), last_syslog(0) { }
+
+ void send_log();
+ void handle_log_ack(MLogAck *m);
+ void set_synchronous(bool sync) { is_synchronous = sync; }
+
+ LogClientTemp debug() {
+ return LogClientTemp(CLOG_DEBUG, *this);
+ }
+ void debug(std::stringstream &s) {
+ do_log(CLOG_DEBUG, s);
+ }
+ LogClientTemp info() {
+ return LogClientTemp(CLOG_INFO, *this);
+ }
+ void info(std::stringstream &s) {
+ do_log(CLOG_INFO, s);
+ }
+ LogClientTemp warn() {
+ return LogClientTemp(CLOG_WARN, *this);
+ }
+ void warn(std::stringstream &s) {
+ do_log(CLOG_WARN, s);
+ }
+ LogClientTemp error() {
+ return LogClientTemp(CLOG_ERROR, *this);
+ }
+ void error(std::stringstream &s) {
+ do_log(CLOG_ERROR, s);
+ }
+ LogClientTemp sec() {
+ return LogClientTemp(CLOG_SEC, *this);
+ }
+ void sec(std::stringstream &s) {
+ do_log(CLOG_SEC, s);
+ }
+
+private:
+ void do_log(clog_type type, std::stringstream& ss);
+ void do_log(clog_type type, const std::string& s);
bool ms_dispatch(Message *m);
- bool is_synchronous;
void _send_log();
-
+ void _send_log_to_syslog();
+ void _send_log_to_monitors();
void ms_handle_connect(Connection *con);
-
bool ms_handle_reset(Connection *con) { return false; }
void ms_handle_remote_reset(Connection *con) {}
-
- public:
-
- // -- log --
+ Messenger *messenger;
+ MonMap *monmap;
+ MonClient *monc;
+ bool is_synchronous;
Mutex log_lock;
- deque<LogEntry> log_queue;
version_t last_log;
+ uint64_t last_syslog;
+ std::deque<LogEntry> log_queue;
- void log(log_type type, const char *s);
- void log(log_type type, string& s);
- void log(log_type type, stringstream& s);
- void send_log();
- void handle_log_ack(MLogAck *m);
- void set_synchronous(bool sync) { is_synchronous = sync; }
-
- LogClient(Messenger *m, MonMap *mm, MonClient *mc=0) :
- messenger(m), monmap(mm), monc(mc), is_synchronous(false),
- log_lock("LogClient::log_lock"), last_log(0) { }
+ friend class LogClientTemp;
};
#endif
diff --git a/src/include/LogEntry.h b/src/common/LogEntry.h
index 1cc02bcb274..04c80b607a4 100644
--- a/src/include/LogEntry.h
+++ b/src/common/LogEntry.h
@@ -19,12 +19,12 @@
#include "include/encoding.h"
typedef enum {
- LOG_DEBUG = 0,
- LOG_INFO = 1,
- LOG_SEC = 2,
- LOG_WARN = 3,
- LOG_ERROR = 4,
-} log_type;
+ CLOG_DEBUG = 0,
+ CLOG_INFO = 1,
+ CLOG_SEC = 2,
+ CLOG_WARN = 3,
+ CLOG_ERROR = 4,
+} clog_type;
struct LogEntryKey {
entity_inst_t who;
@@ -55,7 +55,7 @@ struct LogEntry {
entity_inst_t who;
utime_t stamp;
uint64_t seq;
- log_type type;
+ clog_type type;
string msg;
LogEntryKey key() const { return LogEntryKey(who, stamp, seq); }
@@ -78,7 +78,7 @@ struct LogEntry {
::decode(stamp, bl);
::decode(seq, bl);
::decode(t, bl);
- type = (log_type)t;
+ type = (clog_type)t;
::decode(msg, bl);
}
};
@@ -118,18 +118,18 @@ struct LogSummary {
};
WRITE_CLASS_ENCODER(LogSummary)
-inline ostream& operator<<(ostream& out, const log_type& t)
+inline ostream& operator<<(ostream& out, clog_type t)
{
switch (t) {
- case LOG_DEBUG:
+ case CLOG_DEBUG:
return out << "[DBG]";
- case LOG_INFO:
+ case CLOG_INFO:
return out << "[INF]";
- case LOG_WARN:
+ case CLOG_WARN:
return out << "[WRN]";
- case LOG_ERROR:
+ case CLOG_ERROR:
return out << "[ERR]";
- case LOG_SEC:
+ case CLOG_SEC:
return out << "[SEC]";
default:
return out << "[???]";
diff --git a/src/common/Thread.h b/src/common/Thread.h
index a567d2dbac4..e7da795dbdd 100644
--- a/src/common/Thread.h
+++ b/src/common/Thread.h
@@ -75,7 +75,7 @@ class Thread {
if (r) {
char buf[80];
- generic_derr(0) << "pthread_create failed with message: " << strerror_r(r, buf, sizeof(buf)) << dendl;
+ generic_dout(0) << "pthread_create failed with message: " << strerror_r(r, buf, sizeof(buf)) << dendl;
} else {
_num_threads.inc();
generic_dout(10) << "thread " << thread_id << " start" << dendl;
@@ -84,7 +84,7 @@ class Thread {
}
int join(void **prval = 0) {
if (thread_id == 0) {
- generic_derr(0) << "WARNING: join on thread that was never started" << dendl;
+ generic_dout(0) << "WARNING: join on thread that was never started" << dendl;
assert(0);
return -EINVAL; // never started.
}
@@ -93,17 +93,17 @@ class Thread {
if (status != 0) {
switch (status) {
case -EINVAL:
- generic_derr(0) << "thread " << thread_id << " join status = EINVAL" << dendl;
+ generic_dout(0) << "thread " << thread_id << " join status = EINVAL" << dendl;
break;
case -ESRCH:
- generic_derr(0) << "thread " << thread_id << " join status = ESRCH" << dendl;
+ generic_dout(0) << "thread " << thread_id << " join status = ESRCH" << dendl;
assert(0);
break;
case -EDEADLK:
- generic_derr(0) << "thread " << thread_id << " join status = EDEADLK" << dendl;
+ generic_dout(0) << "thread " << thread_id << " join status = EDEADLK" << dendl;
break;
default:
- generic_derr(0) << "thread " << thread_id << " join status = " << status << dendl;
+ generic_dout(0) << "thread " << thread_id << " join status = " << status << dendl;
}
assert(0); // none of these should happen.
}
diff --git a/src/common/Timer.cc b/src/common/Timer.cc
index 7a3439e37f1..9980790b36e 100644
--- a/src/common/Timer.cc
+++ b/src/common/Timer.cc
@@ -22,7 +22,7 @@
#define DOUT_SUBSYS timer
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "timer(" << this << ")."
+#define dout_prefix *_dout << "timer(" << this << ")."
#include <sstream>
#include <signal.h>
diff --git a/src/common/WorkQueue.cc b/src/common/WorkQueue.cc
index cae4cef6c8f..a3f67e0a0d2 100644
--- a/src/common/WorkQueue.cc
+++ b/src/common/WorkQueue.cc
@@ -19,7 +19,7 @@
#define DOUT_SUBSYS tp
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << name << " "
+#define dout_prefix *_dout << name << " "
void ThreadPool::worker()
diff --git a/src/common/common_init.cc b/src/common/common_init.cc
index f634247f8cb..a5c02b9fff1 100644
--- a/src/common/common_init.cc
+++ b/src/common/common_init.cc
@@ -44,13 +44,6 @@ void common_init(std::vector<const char*>& args, const char *module_type, bool i
}
#endif //HAVE_LIBTCMALLOC
- if (g_conf.log_file && g_conf.log_file[0])
- g_conf.log_to_stdout = false;
-
- // open log file?
- if (!g_conf.log_to_stdout)
- _dout_open_log();
-
if (init_keys && is_supported_auth(CEPH_AUTH_CEPHX)) {
g_keyring.load(g_conf.keyring);
diff --git a/src/common/debug.cc b/src/common/debug.cc
index 86def0548a1..10cd6ceed9e 100644
--- a/src/common/debug.cc
+++ b/src/common/debug.cc
@@ -1,212 +1,70 @@
-
-#include "include/types.h"
-#include "config.h"
-#include "debug.h"
#include "Mutex.h"
-#include "Clock.h"
-
#include "ceph_ver.h"
+#include "common/DoutStreambuf.h"
+#include "config.h"
+#include "debug.h"
#include <errno.h>
#include <fstream>
#include <iostream>
-using namespace std;
-
-#define _STR(x) #x
-#define STRINGIFY(x) _STR(x)
// debug output
-Mutex _dout_lock("_dout_lock", false, false /* no lockdep */);
-ostream *_dout = &std::cout;
-ostream *_derr = &std::cerr;
-char _dout_dir[PATH_MAX] = {0};
-char _dout_symlink_dir[PATH_MAX] = {0};
-char _dout_file[PATH_MAX] = {0};
-char _dout_path[PATH_MAX] = {0};
-char _dout_rank_symlink_path[PATH_MAX] = {0};
-char _dout_name_symlink_path[PATH_MAX] = {0};
-char *_dout_symlink_target = 0; // _dout_path or _dout_file
-bool _dout_is_open = false;
+std::ostream *_dout = NULL;
+DoutStreambuf <char> *_doss = NULL;
bool _dout_need_open = true;
-std::ofstream _dout_out;
-
-static void normalize_relative(const char *from, char *to, int tolen)
-{
- if (from[0] == '/')
- strncpy(to, from, tolen);
- else {
- char *c = getcwd(to, tolen);
- assert(c);
- strncat(to, "/", tolen);
- strncat(to, from, tolen);
- }
-}
-
-static void build_log_paths()
-{
- if (g_conf.log_file && g_conf.log_file[0]) {
- normalize_relative(g_conf.log_file, _dout_path, sizeof(_dout_path));
- } else {
- if (g_conf.log_per_instance) {
- char hostname[80];
- gethostname(hostname, 79);
- snprintf(_dout_file, sizeof(_dout_file), "%s.%d", hostname, getpid());
- } else {
- snprintf(_dout_file, sizeof(_dout_file), "%s.%s.log", g_conf.type, g_conf.id);
- }
- snprintf(_dout_path, sizeof(_dout_path), "%s/%s", _dout_dir, _dout_file);
- }
-}
-
-static bool log_to_file()
-{
- return (g_conf.log_dir || g_conf.log_file) && !g_conf.log_to_stdout;
-}
-
-static int create_symlink(const char *from)
-{
- ::unlink(from);
- int r = ::symlink(_dout_symlink_target, from);
- if (r) {
- char buf[80];
- *_dout << "---- " << getpid() << " failed to symlink " << _dout_symlink_target
- << " from " << from
- << ": " << strerror_r(errno, buf, sizeof(buf)) << std::endl;
- }
- return r;
-}
-
-static void rotate_file(const char *fn, int max)
-{
- char a[200], b[200];
- // rotate out old
- int n = 0;
- while (1) {
- struct stat st;
- snprintf(a, sizeof(a), "%s.%lld", fn, (long long)n);
- if (::lstat(a, &st) != 0)
- break;
- n++;
- }
- while (n >= 0) {
- if (n)
- snprintf(a, sizeof(a), "%s.%lld", fn, (long long)n-1);
- else
- snprintf(a, sizeof(a), "%s", fn);
- if (n >= max) {
- ::unlink(a);
- *_dout << "---- " << getpid() << " removed " << a << " ----" << std::endl;
- } else {
- snprintf(b, sizeof(b), "%s.%lld", fn, (long long)n);
- ::rename(a, b);
- *_dout << "---- " << getpid() << " renamed " << a << " -> " << b << " ----" << std::endl;
- }
- n--;
- }
-}
-
-static int create_name_symlink()
-{
- int r = 0;
- if (log_to_file() && g_conf.log_per_instance && !(g_conf.log_file && g_conf.log_file[0])) {
- snprintf(_dout_name_symlink_path, sizeof(_dout_name_symlink_path),
- "%s/%s.%s", _dout_symlink_dir, g_conf.type, g_conf.id);
-
- rotate_file(_dout_name_symlink_path, g_conf.log_sym_history);
- r = create_symlink(_dout_name_symlink_path);
- }
- return r;
-}
+Mutex _dout_lock("_dout_lock", false, false /* no lockdep */);
+#define _STR(x) #x
+#define STRINGIFY(x) _STR(x)
void _dout_open_log()
{
- bool need_symlink = false;
-
- // logging enabled?
- if (!log_to_file()) {
- _dout_need_open = false;
- return;
- }
-
- // calculate log dir, filename, etc.
- // do this _once_.
- if (!_dout_path[0]) {
-
- // normalize paths
- normalize_relative(g_conf.log_dir, _dout_dir, sizeof(_dout_dir));
- if (!g_conf.log_sym_dir)
- g_conf.log_sym_dir = strdup(g_conf.log_dir);
- normalize_relative(g_conf.log_sym_dir, _dout_symlink_dir, sizeof(_dout_symlink_dir));
+ assert(_dout_need_open);
+ assert(_dout_lock.is_locked());
- // make symlink targets absolute or relative?
- if ((g_conf.log_file && g_conf.log_file[0]) ||
- strcmp(_dout_symlink_dir, _dout_dir) == 0)
- _dout_symlink_target = _dout_file;
- else
- _dout_symlink_target = _dout_path;
-
- build_log_paths();
-
- need_symlink = true;
+ if (!_doss) {
+ _doss = new DoutStreambuf <char>();
}
-
- _dout_out.close();
- _dout_out.open(_dout_path, ios::out | ios::app);
- if (!_dout_out.is_open()) {
- std::cerr << "error opening output file " << _dout_path << std::endl;
- _dout = &std::cout;
- } else {
- _dout_need_open = false;
- _dout_is_open = true;
- _dout = &_dout_out;
- *_dout << g_clock.now() << " --- " << getpid() << " opened log "
- << _dout_path << " ---" << std::endl;
+ _doss->read_global_config();
+ if (!_dout) {
+ _dout = new std::ostream(_doss);
}
- *_dout << "ceph version " << VERSION << " (commit:" << STRINGIFY(CEPH_GIT_VER) << ")" << std::endl;
- if (need_symlink)
- create_name_symlink();
+ *_dout << "ceph version " << VERSION << " (commit:"
+ << STRINGIFY(CEPH_GIT_VER) << ")" << std::endl;
+ _dout_need_open = false;
}
-int dout_rename_output_file() // after calling daemon()
+int dout_handle_daemonize()
{
Mutex::Locker l(_dout_lock);
- if (log_to_file() && g_conf.log_per_instance) {
- char oldpath[PATH_MAX];
- char hostname[80];
- gethostname(hostname, 79);
-
- strcpy(oldpath, _dout_path);
-
- build_log_paths();
-
- *_dout << "---- " << getpid() << " renamed log " << oldpath << " -> " << _dout_path << " ----" << std::endl;
- ::rename(oldpath, _dout_path);
-
- // $type.$id symlink
- if (g_conf.log_per_instance && _dout_name_symlink_path[0])
- create_symlink(_dout_name_symlink_path);
- if (_dout_rank_symlink_path[0])
- create_symlink(_dout_rank_symlink_path);
- }
- return 0;
+ _doss->handle_stdout_closed();
+ _doss->handle_stderr_closed();
+ return _doss->handle_pid_change();
}
-int dout_create_rank_symlink(int64_t n)
+int dout_create_rank_symlink(int n)
{
Mutex::Locker l(_dout_lock);
- int r = 0;
- if (log_to_file() && !(g_conf.log_file && g_conf.log_file[0])) {
- if (_dout_need_open)
- _dout_open_log();
+ return _doss->create_rank_symlink(n);
+}
- snprintf(_dout_rank_symlink_path, sizeof(_dout_rank_symlink_path),
- "%s/%s%lld", _dout_symlink_dir, g_conf.type, (long long)n);
- r = create_symlink(_dout_rank_symlink_path);
+void hex2str(const char *s, int len, char *buf, int dest_len)
+{
+ int pos = 0;
+ for (int i=0; i<len && pos<dest_len; i++) {
+ if (i && !(i%8))
+ pos += snprintf(&buf[pos], dest_len-pos, " ");
+ if (i && !(i%16))
+ pos += snprintf(&buf[pos], dest_len-pos, "\n");
+ pos += snprintf(&buf[pos], dest_len-pos, "%.2x ", (int)(unsigned char)s[i]);
}
- return r;
}
-
-
+void hexdump(string msg, const char *s, int len)
+{
+ int buf_len = len*4;
+ char buf[buf_len];
+ hex2str(s, len, buf, buf_len);
+ generic_dout(0) << msg << ":\n" << buf << dendl;
+}
diff --git a/src/common/debug.h b/src/common/debug.h
index 7b60c38abe8..af6fe9957e1 100644
--- a/src/common/debug.h
+++ b/src/common/debug.h
@@ -1,117 +1,87 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// -*- 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-2010 Sage Weil <sage@newdream.net>
+ * Copyright (C) 2010 Dreamhost
+ *
+ * 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_DEBUG_H
#define CEPH_DEBUG_H
-#include "include/assert.h"
-#include "Mutex.h"
#include "Clock.h"
+#include "Mutex.h"
+#include "common/DoutStreambuf.h"
+#include "common/likely.h"
+#include "include/assert.h"
-#include <ostream>
-using std::ostream;
-
-// the streams
-extern ostream *_dout;
-extern ostream *_derr;
-
-extern Mutex _dout_lock;
+#include <iosfwd>
+extern std::ostream *_dout;
+extern DoutStreambuf <char> *_doss;
extern bool _dout_need_open;
-extern bool _dout_is_open;
+extern Mutex _dout_lock;
extern void _dout_open_log();
-static inline void dout_open_log() {
- _dout_lock.Lock();
- _dout_open_log();
- _dout_lock.Unlock();
-}
-extern int dout_rename_output_file(); // after calling daemon()
-extern int dout_create_rank_symlink(int64_t n);
+extern int dout_handle_daemonize();
-static inline void _dout_check_log() {
- _dout_lock.Lock();
- if (_dout_need_open)
- _dout_open_log();
- _dout_lock.Unlock();
-}
+extern int dout_create_rank_symlink(int n);
-static inline void _dout_begin_line() {
+static inline void _dout_begin_line(int prio) {
_dout_lock.Lock();
- if (_dout_need_open)
+ if (unlikely(_dout_need_open))
_dout_open_log();
- *_dout << g_clock.now() << " " << std::hex << pthread_self() << std::dec << " ";
-}
-static void _dout_begin_line_static() {
- _dout_begin_line();
-}
-static inline void _dout_end_line() {
- _dout_lock.Unlock();
-}
+ // Put priority information into dout
+ _doss->sputc(1);
+ _doss->sputc(prio + 11);
-struct _dbeginl_t { _dbeginl_t(int) {} };
-static const _dbeginl_t dbeginl = 0;
-inline ostream& operator<<(ostream& out, _dbeginl_t) {
- _dout_begin_line();
- return out;
+ // Some information that goes in every dout message
+ *_dout << g_clock.now() << " " << std::hex << pthread_self()
+ << std::dec << " ";
}
-struct _dbeginlstatic_t { _dbeginlstatic_t(int) {} };
-static const _dbeginlstatic_t dbeginlstatic = 0;
-inline ostream& operator<<(ostream& out, _dbeginlstatic_t) {
- _dout_begin_line_static();
- return out;
+static inline void _dout_end_line() {
+ _dout_lock.Unlock();
}
// intentionally conflict with endl
class _bad_endl_use_dendl_t { public: _bad_endl_use_dendl_t(int) {} };
static const _bad_endl_use_dendl_t endl = 0;
-inline ostream& operator<<(ostream& out, _bad_endl_use_dendl_t) {
+inline std::ostream& operator<<(std::ostream& out, _bad_endl_use_dendl_t) {
assert(0 && "you are using the wrong endl.. use std::endl or dendl");
return out;
}
-
// generic macros
-#define generic_dout(x) do { if ((x) <= g_conf.debug) { *_dout << dbeginl
-#define generic_derr(x) do { if ((x) <= g_conf.debug) { *_derr << dbeginl
+#define generic_dout(x) do { if ((x) <= g_conf.debug) {\
+ _dout_begin_line(x); *_dout
-#define pdout(x,p) do { if ((x) <= (p)) { *_dout << dbeginl
+#define pdout(x,p) do { if ((x) <= (p)) {\
+ _dout_begin_line(x); *_dout
#define debug_DOUT_SUBSYS debug
-#define dout_prefix *_dout << dbeginlstatic
+#define dout_prefix *_dout
#define DOUT_CONDVAR(x) g_conf.debug_ ## x
#define XDOUT_CONDVAR(x) DOUT_CONDVAR(x)
#define DOUT_COND(l) l <= XDOUT_CONDVAR(DOUT_SUBSYS)
-#define dout(l) do { if (DOUT_COND(l)) { dout_prefix
-#define derr(l) do { if (DOUT_COND(l)) { dout_prefix
+#define dout(l) do { if (DOUT_COND(l)) {\
+ _dout_begin_line(l); dout_prefix
#define dendl std::endl; _dout_end_line(); } } while (0)
-inline static void hex2str(const char *s, int len, char *buf, int dest_len)
-{
- int pos = 0;
- for (int i=0; i<len && pos<dest_len; i++) {
- if (i && !(i%8))
- pos += snprintf(&buf[pos], dest_len-pos, " ");
- if (i && !(i%16))
- pos += snprintf(&buf[pos], dest_len-pos, "\n");
- pos += snprintf(&buf[pos], dest_len-pos, "%.2x ", (int)(unsigned char)s[i]);
- }
-}
-
-inline static void hexdump(string msg, const char *s, int len)
-{
- int buf_len = len*4;
- char buf[buf_len];
- hex2str(s, len, buf, buf_len);
- generic_dout(0) << msg << ":\n" << buf << dendl;
-}
-
+extern void hex2str(const char *s, int len, char *buf, int dest_len);
+extern void hexdump(string msg, const char *s, int len);
#endif
diff --git a/src/common/likely.h b/src/common/likely.h
new file mode 100644
index 00000000000..e8146a3f69e
--- /dev/null
+++ b/src/common/likely.h
@@ -0,0 +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) 2010 Dreamhost
+ *
+ * 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_LIKELY_DOT_H
+#define CEPH_LIKELY_DOT_H
+
+/*
+ * Likely / Unlikely macros
+ */
+#define likely(x) __builtin_expect((x),1)
+#define unlikely(x) __builtin_expect((x),0)
+
+#endif
diff --git a/src/config.cc b/src/config.cc
index 8b6cfd5ae23..e07cc0fcf9c 100644
--- a/src/config.cc
+++ b/src/config.cc
@@ -326,7 +326,11 @@ static struct config_option config_optionsp[] = {
OPTION(log_sym_dir, 0, OPT_STR, 0),
OPTION(log_sym_history, 0, OPT_INT, 10),
OPTION(log_to_stdout, 0, OPT_BOOL, true),
+ OPTION(log_to_syslog, 0, OPT_BOOL, false),
OPTION(log_per_instance, 0, OPT_BOOL, false),
+ OPTION(log_to_file, 0, OPT_BOOL, true),
+ OPTION(clog_to_monitors, 0, OPT_BOOL, true),
+ OPTION(clog_to_syslog, 0, OPT_BOOL, false),
OPTION(pid_file, 0, OPT_STR, "/var/run/ceph/$type.$id.pid"),
OPTION(conf, 'c', OPT_STR, "/etc/ceph/ceph.conf, ~/.ceph/config, ceph.conf"),
OPTION(chdir, 0, OPT_STR, "/"),
diff --git a/src/config.h b/src/config.h
index 1d1ddb72f9e..69d22c7fc95 100644
--- a/src/config.h
+++ b/src/config.h
@@ -82,7 +82,12 @@ struct md_config_t {
const char *log_sym_dir;
int log_sym_history;
bool log_to_stdout;
+ bool log_to_syslog;
bool log_per_instance;
+ bool log_to_file;
+
+ bool clog_to_monitors;
+ bool clog_to_syslog;
const char *pid_file;
diff --git a/src/cosd.cc b/src/cosd.cc
index 7e77799de00..e60c6e3eb83 100644
--- a/src/cosd.cc
+++ b/src/cosd.cc
@@ -184,7 +184,7 @@ int main(int argc, const char **argv)
cerr << TEXT_RED << " ** ERROR: unable to open OSD superblock on " << g_conf.osd_data << ": " << strerror_r(-r, buf, sizeof(buf)) << TEXT_NORMAL << std::endl;
if (r == -ENOTSUP)
cerr << TEXT_RED << " ** please verify that underlying storage supports xattrs" << TEXT_NORMAL << std::endl;
- derr(0) << "unable to open OSD superblock on " << g_conf.osd_data << ": " << strerror_r(-r, buf, sizeof(buf)) << dendl;
+ dout(0) << "unable to open OSD superblock on " << g_conf.osd_data << ": " << strerror_r(-r, buf, sizeof(buf)) << dendl;
exit(1);
}
if (w != whoami) {
diff --git a/src/ebofs/Allocator.cc b/src/ebofs/Allocator.cc
index da3b8d50614..6d4ef1d3fd8 100644
--- a/src/ebofs/Allocator.cc
+++ b/src/ebofs/Allocator.cc
@@ -21,7 +21,7 @@
#define DOUT_SUBSYS ebofs
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "ebofs(" << fs->dev.get_device_name() << ").allocator."
+#define dout_prefix *_dout << "ebofs(" << fs->dev.get_device_name() << ").allocator."
void Allocator::dump_freelist()
diff --git a/src/ebofs/BlockDevice.cc b/src/ebofs/BlockDevice.cc
index e3c958a668e..830d97be4aa 100644
--- a/src/ebofs/BlockDevice.cc
+++ b/src/ebofs/BlockDevice.cc
@@ -65,7 +65,7 @@ inline ostream& operator<<(ostream& out, BlockDevice::biovec &bio)
*/
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "bdev(" << dev << ").elevatorq."
+#define dout_prefix *_dout << "bdev(" << dev << ").elevatorq."
int BlockDevice::ElevatorQueue::dequeue_io(list<biovec*>& biols,
@@ -214,7 +214,7 @@ int BlockDevice::ElevatorQueue::dequeue_io(list<biovec*>& biols,
* BarrierQueue
*/
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "bdev(" << dev << ").barrierq."
+#define dout_prefix *_dout << "bdev(" << dev << ").barrierq."
void BlockDevice::BarrierQueue::barrier()
{
@@ -263,7 +263,7 @@ int BlockDevice::BarrierQueue::dequeue_io(list<biovec*>& biols,
*/
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "bdev(" << dev << ")."
+#define dout_prefix *_dout << "bdev(" << dev << ")."
block_t BlockDevice::get_num_blocks()
{
diff --git a/src/ebofs/BufferCache.cc b/src/ebofs/BufferCache.cc
index 5866ce10cd2..efdf03235d7 100644
--- a/src/ebofs/BufferCache.cc
+++ b/src/ebofs/BufferCache.cc
@@ -38,7 +38,7 @@ void do_apply_partial(bufferlist& bl, map<uint64_t, bufferlist>& pm)
/*********** BufferHead **************/
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "ebofs." << *this << "."
+#define dout_prefix *_dout << "ebofs." << *this << "."
void BufferHead::add_partial(uint64_t off, bufferlist& p)
@@ -124,7 +124,7 @@ void BufferHead::apply_partial()
/************ ObjectCache **************/
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "ebofs.oc."
+#define dout_prefix *_dout << "ebofs.oc."
void ObjectCache::rx_finish(ioh_t ioh, block_t start, block_t length, bufferlist& bl)
{
@@ -983,7 +983,7 @@ void ObjectCache::scrub_csums()
/************** BufferCache ***************/
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "ebofs.bc."
+#define dout_prefix *_dout << "ebofs.bc."
BufferHead *BufferCache::split(BufferHead *orig, block_t after)
diff --git a/src/ebofs/Ebofs.cc b/src/ebofs/Ebofs.cc
index dbb4b096358..40cf0c48fb7 100644
--- a/src/ebofs/Ebofs.cc
+++ b/src/ebofs/Ebofs.cc
@@ -31,7 +31,7 @@
#define DOUT_SUBSYS ebofs
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "ebofs(" << dev.get_device_name() << ")."
+#define dout_prefix *_dout << "ebofs(" << dev.get_device_name() << ")."
char *nice_blocks(block_t b)
diff --git a/src/ebofs/Table.h b/src/ebofs/Table.h
index c960fa89542..94ab3396587 100644
--- a/src/ebofs/Table.h
+++ b/src/ebofs/Table.h
@@ -21,7 +21,9 @@
/** table **/
-#define dbtout do { if (25 <= g_conf.debug_ebofs) { *_dout << dbeginl << "ebofs.table(" << this << ")."
+#define dbtout do { if (25 <= g_conf.debug_ebofs) {\
+ _dout_begin_line(25);\
+ *_dout << "ebofs.table(" << this << ")."
template<class K, class V>
diff --git a/src/ebofs/nodes.h b/src/ebofs/nodes.h
index 4f357df8c3f..4a42f777641 100644
--- a/src/ebofs/nodes.h
+++ b/src/ebofs/nodes.h
@@ -45,7 +45,8 @@
*/
#undef debofs
-#define debofs(x) do { if (x <= g_conf.debug_ebofs) { *_dout << dbeginl << "ebofs.nodepool."
+#define debofs(x) do { if (x <= g_conf.debug_ebofs) { \
+ _dout_begin_line(x); *_dout << "ebofs.nodepool."
class Node {
diff --git a/src/librados.cc b/src/librados.cc
index 5f0dca751ab..b5ddb156531 100644
--- a/src/librados.cc
+++ b/src/librados.cc
@@ -43,7 +43,7 @@ using namespace std;
#define RADOS_LIST_MAX_ENTRIES 1024
#define DOUT_SUBSYS rados
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "librados: "
+#define dout_prefix *_dout << "librados: "
class RadosClient : public Dispatcher
diff --git a/src/mds/AnchorClient.cc b/src/mds/AnchorClient.cc
index 80ebba34464..1d8dddb2196 100644
--- a/src/mds/AnchorClient.cc
+++ b/src/mds/AnchorClient.cc
@@ -24,7 +24,7 @@
#define DOUT_SUBSYS mds
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << mds->get_nodeid() << ".anchorclient "
+#define dout_prefix *_dout << "mds" << mds->get_nodeid() << ".anchorclient "
diff --git a/src/mds/AnchorServer.cc b/src/mds/AnchorServer.cc
index 64fadc48a27..7f94c3bc156 100644
--- a/src/mds/AnchorServer.cc
+++ b/src/mds/AnchorServer.cc
@@ -19,7 +19,7 @@
#define DOUT_SUBSYS mds
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << mds->get_nodeid() << ".anchorserver "
+#define dout_prefix *_dout << "mds" << mds->get_nodeid() << ".anchorserver "
// table
diff --git a/src/mds/CDentry.cc b/src/mds/CDentry.cc
index 30bddbb8f28..b7dbbc60500 100644
--- a/src/mds/CDentry.cc
+++ b/src/mds/CDentry.cc
@@ -28,7 +28,7 @@
#define DOUT_SUBSYS mds
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << dir->cache->mds->get_nodeid() << ".cache.den(" << dir->dirfrag() << " " << name << ") "
+#define dout_prefix *_dout << "mds" << dir->cache->mds->get_nodeid() << ".cache.den(" << dir->dirfrag() << " " << name << ") "
ostream& CDentry::print_db_line_prefix(ostream& out)
diff --git a/src/mds/CDir.cc b/src/mds/CDir.cc
index 661e6386988..a1511833a81 100644
--- a/src/mds/CDir.cc
+++ b/src/mds/CDir.cc
@@ -36,7 +36,7 @@
#define DOUT_SUBSYS mds
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << cache->mds->get_nodeid() << ".cache.dir(" << this->dirfrag() << ") "
+#define dout_prefix *_dout << "mds" << cache->mds->get_nodeid() << ".cache.dir(" << this->dirfrag() << ") "
@@ -1206,6 +1206,7 @@ void CDir::fetch(Context *c, const string& want_dn, bool ignore_authpinnability)
void CDir::_fetched(bufferlist &bl, const string& want_dn)
{
+ LogClient &clog = cache->mds->clog;
dout(10) << "_fetched " << bl.length()
<< " bytes for " << *this
<< " want_dn=" << want_dn
@@ -1217,9 +1218,8 @@ void CDir::_fetched(bufferlist &bl, const string& want_dn)
// empty?!?
if (bl.length() == 0) {
dout(0) << "_fetched missing object for " << *this << dendl;
- stringstream ss;
- ss << "dir " << ino() << "." << dirfrag() << " object missing on disk; some files may be lost";
- cache->mds->logclient.log(LOG_ERROR, ss);
+ clog.error() << "dir " << ino() << "." << dirfrag()
+ << " object missing on disk; some files may be lost\n";
log_mark_dirty();
@@ -1390,14 +1390,14 @@ void CDir::_fetched(bufferlist &bl, const string& want_dn)
dout(-12) << "_fetched badness: got (but i already had) " << *in
<< " mode " << in->inode.mode
<< " mtime " << in->inode.mtime << dendl;
- stringstream ss;
string dirpath, inopath;
this->inode->make_path_string(dirpath);
in->make_path_string(inopath);
- ss << "loaded dup inode " << inode.ino << " [" << first << "," << last << "] v" << inode.version
- << " at " << dirpath << "/" << dname
- << ", but inode " << in->vino() << " v" << in->inode.version << " already exists at " << inopath;
- cache->mds->logclient.log(LOG_ERROR, ss);
+ clog.error() << "loaded dup inode " << inode.ino
+ << " [" << first << "," << last << "] v" << inode.version
+ << " at " << dirpath << "/" << dname
+ << ", but inode " << in->vino() << " v" << in->inode.version
+ << " already exists at " << inopath << "\n";
continue;
} else {
// inode
@@ -1468,9 +1468,8 @@ void CDir::_fetched(bufferlist &bl, const string& want_dn)
}
}
if (!p.end()) {
- stringstream ss;
- ss << "dir " << dirfrag() << " has " << bl.length() - p.get_off() << " extra bytes";
- cache->mds->logclient.log(LOG_WARN, ss);
+ clog.warn() << "dir " << dirfrag() << " has "
+ << bl.length() - p.get_off() << " extra bytes\n";
}
//cache->mds->logger->inc("newin", num_new_inodes_loaded);
diff --git a/src/mds/CInode.cc b/src/mds/CInode.cc
index 44da8aaa007..660d28be571 100644
--- a/src/mds/CInode.cc
+++ b/src/mds/CInode.cc
@@ -45,7 +45,7 @@
#define DOUT_SUBSYS mds
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << mdcache->mds->get_nodeid() << ".cache.ino(" << inode.ino << ") "
+#define dout_prefix *_dout << "mds" << mdcache->mds->get_nodeid() << ".cache.ino(" << inode.ino << ") "
boost::pool<> CInode::pool(sizeof(CInode));
@@ -1598,6 +1598,8 @@ void CInode::_finish_frag_update(CDir *dir, Mutation *mut)
/* for more info on scatterlocks, see comments by Locker::scatter_writebehind */
void CInode::finish_scatter_gather_update(int type)
{
+ LogClient &clog = mdcache->mds->clog;
+
dout(10) << "finish_scatter_gather_update " << type << " on " << *this << dendl;
assert(is_auth());
@@ -1634,9 +1636,8 @@ void CInode::finish_scatter_gather_update(int type)
if (pf->fragstat.nfiles < 0 ||
pf->fragstat.nsubdirs < 0) {
- stringstream ss;
- ss << "bad/negative dir size on " << dir->dirfrag() << " " << pf->fragstat;
- mdcache->mds->logclient.log(LOG_ERROR, ss);
+ clog.error() << "bad/negative dir size on "
+ << dir->dirfrag() << " " << pf->fragstat << "\n";
if (pf->fragstat.nfiles < 0)
pf->fragstat.nfiles = 0;
@@ -1654,10 +1655,9 @@ void CInode::finish_scatter_gather_update(int type)
if (fg == frag_t()) { // i.e., we are the only frag
if (pi->dirstat.size() != pf->fragstat.size()) {
- stringstream ss;
- ss << "unmatched fragstat size on single dirfrag " << dir->dirfrag()
- << ", inode has " << pi->dirstat << ", dirfrag has " << pf->fragstat;
- mdcache->mds->logclient.log(LOG_ERROR, ss);
+ clog.error() << "unmatched fragstat size on single "
+ << "dirfrag " << dir->dirfrag() << ", inode has "
+ << pi->dirstat << ", dirfrag has " << pf->fragstat << "\n";
// trust the dirfrag for now
version_t v = pi->dirstat.version;
@@ -1674,9 +1674,8 @@ void CInode::finish_scatter_gather_update(int type)
if (pi->dirstat.nfiles < 0 ||
pi->dirstat.nsubdirs < 0) {
- stringstream ss;
- ss << "bad/negative dir size on " << ino() << ", inode has " << pi->dirstat;
- mdcache->mds->logclient.log(LOG_ERROR, ss);
+ clog.error() << "bad/negative dir size on " << ino()
+ << ", inode has " << pi->dirstat << "\n";
if (pi->dirstat.nfiles < 0)
pi->dirstat.nfiles = 0;
@@ -1737,10 +1736,9 @@ void CInode::finish_scatter_gather_update(int type)
if (fg == frag_t()) { // i.e., we are the only frag
if (pi->rstat.rbytes != pf->rstat.rbytes) {
- stringstream ss;
- ss << "unmatched rstat rbytes on single dirfrag " << dir->dirfrag()
- << ", inode has " << pi->rstat << ", dirfrag has " << pf->rstat;
- mdcache->mds->logclient.log(LOG_ERROR, ss);
+ clog.error() << "unmatched rstat rbytes on single dirfrag "
+ << dir->dirfrag() << ", inode has " << pi->rstat
+ << ", dirfrag has " << pf->rstat << "\n";
// trust the dirfrag for now
version_t v = pi->rstat.version;
@@ -1755,17 +1753,15 @@ void CInode::finish_scatter_gather_update(int type)
//assert(pi->rstat.rfiles >= 0);
if (pi->rstat.rfiles < 0) {
- stringstream ss;
- ss << "rfiles underflow " << pi->rstat.rfiles << " on " << *this;
- mdcache->mds->logclient.log(LOG_ERROR, ss);
+ clog.error() << "rfiles underflow " << pi->rstat.rfiles
+ << " on " << *this << "\n";
pi->rstat.rfiles = 0;
}
//assert(pi->rstat.rsubdirs >= 0);
if (pi->rstat.rsubdirs < 0) {
- stringstream ss;
- ss << "rsubdirs underflow " << pi->rstat.rfiles << " on " << *this;
- mdcache->mds->logclient.log(LOG_ERROR, ss);
+ clog.error() << "rsubdirs underflow " << pi->rstat.rfiles
+ << " on " << *this << "\n";
pi->rstat.rsubdirs = 0;
}
}
diff --git a/src/mds/InoTable.cc b/src/mds/InoTable.cc
index aacb1282e2f..80537304051 100644
--- a/src/mds/InoTable.cc
+++ b/src/mds/InoTable.cc
@@ -21,7 +21,7 @@
#define DOUT_SUBSYS mds
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << mds->get_nodeid() << "." << table_name << ": "
+#define dout_prefix *_dout << "mds" << mds->get_nodeid() << "." << table_name << ": "
void InoTable::reset_state()
{
@@ -104,9 +104,8 @@ void InoTable::replay_alloc_id(inodeno_t id)
free.erase(id);
projected_free.erase(id);
} else {
- stringstream ss;
- ss << "journal replay alloc " << id << " not in free " << free;
- mds->logclient.log(LOG_ERROR, ss);
+ mds->clog.error() << "journal replay alloc " << id
+ << " not in free " << free << "\n";
}
projected_version = ++version;
}
@@ -119,9 +118,8 @@ void InoTable::replay_alloc_ids(interval_set<inodeno_t>& ids)
free.subtract(ids);
projected_free.subtract(ids);
} else {
- stringstream ss;
- ss << "journal replay alloc " << ids << ", only " << is << " is in free " << free;
- mds->logclient.log(LOG_ERROR, ss);
+ mds->clog.error() << "journal replay alloc " << ids << ", only "
+ << is << " is in free " << free << "\n";
free.subtract(is);
projected_free.subtract(is);
}
diff --git a/src/mds/Locker.cc b/src/mds/Locker.cc
index e1e22c36db7..3754a166704 100644
--- a/src/mds/Locker.cc
+++ b/src/mds/Locker.cc
@@ -59,7 +59,7 @@
#undef dout_prefix
#define dout_prefix _prefix(mds)
static ostream& _prefix(MDS *mds) {
- return *_dout << dbeginl << "mds" << mds->get_nodeid() << ".locker ";
+ return *_dout << "mds" << mds->get_nodeid() << ".locker ";
}
/* This function DOES put the passed message before returning */
@@ -2081,9 +2081,8 @@ void Locker::process_request_cap_release(MDRequest *mdr, client_t client, const
dn->remove_client_lease(l, this);
}
} else {
- stringstream ss;
- ss << "client" << client << " released lease on dn " << dir->dirfrag() << "/" << dname << " which dne";
- mds->logclient.log(LOG_WARN, ss);
+ mds->clog.warn() << "client" << client << " released lease on dn "
+ << dir->dirfrag() << "/" << dname << " which dne\n";
}
}
}
diff --git a/src/mds/MDBalancer.cc b/src/mds/MDBalancer.cc
index 9a932101f92..95d6bdf68ba 100644
--- a/src/mds/MDBalancer.cc
+++ b/src/mds/MDBalancer.cc
@@ -39,7 +39,7 @@ using std::vector;
#undef DOUT_COND
#define DOUT_COND(l) l<=g_conf.debug_mds || l <= g_conf.debug_mds_balancer
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << mds->get_nodeid() << ".bal "
+#define dout_prefix *_dout << "mds" << mds->get_nodeid() << ".bal "
#define MIN_LOAD 50 // ??
#define MIN_REEXPORT 5 // will automatically reexport
diff --git a/src/mds/MDCache.cc b/src/mds/MDCache.cc
index 7977d87136d..6192744317a 100644
--- a/src/mds/MDCache.cc
+++ b/src/mds/MDCache.cc
@@ -93,7 +93,7 @@ using namespace std;
#undef dout_prefix
#define dout_prefix _prefix(mds)
static ostream& _prefix(MDS *mds) {
- return *_dout << dbeginl << "mds" << mds->get_nodeid() << ".cache ";
+ return *_dout << "mds" << mds->get_nodeid() << ".cache ";
}
long g_num_ino = 0;
@@ -1909,10 +1909,9 @@ void MDCache::predirty_journal_parents(Mutation *mut, EMetaBlob *blob,
assert(!"negative dirstat size" == g_conf.mds_verify_scatter);
if (parent->get_frag() == frag_t()) { // i.e., we are the only frag
if (pi->dirstat.size() != pf->fragstat.size()) {
- stringstream ss;
- ss << "unmatched fragstat size on single dirfrag " << parent->dirfrag()
- << ", inode has " << pi->dirstat << ", dirfrag has " << pf->fragstat;
- mds->logclient.log(LOG_ERROR, ss);
+ mds->clog.error() << "unmatched fragstat size on single dirfrag "
+ << parent->dirfrag() << ", inode has " << pi->dirstat
+ << ", dirfrag has " << pf->fragstat << "\n";
// trust the dirfrag for now
pi->dirstat = pf->fragstat;
@@ -1957,10 +1956,9 @@ void MDCache::predirty_journal_parents(Mutation *mut, EMetaBlob *blob,
if (parent->get_frag() == frag_t()) { // i.e., we are the only frag
if (pi->rstat.rbytes != pf->rstat.rbytes) {
- stringstream ss;
- ss << "unmatched rstat rbytes on single dirfrag " << parent->dirfrag()
- << ", inode has " << pi->rstat << ", dirfrag has " << pf->rstat;
- mds->logclient.log(LOG_ERROR, ss);
+ mds->clog.error() << "unmatched rstat rbytes on single dirfrag "
+ << parent->dirfrag() << ", inode has " << pi->rstat
+ << ", dirfrag has " << pf->rstat << "\n";
// trust the dirfrag for now
pi->rstat = pf->rstat;
@@ -4747,9 +4745,8 @@ void MDCache::do_file_recover()
// blech
if (pi->client_ranges.size() && !pi->get_max_size()) {
- stringstream ss;
- ss << "bad client_range " << pi->client_ranges << " on ino " << pi->ino;
- mds->logclient.log(LOG_WARN, ss);
+ mds->clog.warn() << "bad client_range " << pi->client_ranges
+ << " on ino " << pi->ino << "\n";
}
if (pi->client_ranges.size() && pi->get_max_size()) {
@@ -5199,7 +5196,7 @@ void MDCache::trim_inode(CDentry *dn, CInode *in, CDir *con, map<int, MCacheExpi
void MDCache::trim_non_auth()
{
dout(7) << "trim_non_auth" << dendl;
- stringstream warn_string_dirs;
+ stringstream warn_str_dirs;
// temporarily pin all subtree roots
for (map<CDir*, set<CDir*> >::iterator p = subtrees.begin();
@@ -5240,12 +5237,11 @@ void MDCache::trim_non_auth()
else if (dnl->is_primary()) {
CInode *in = dnl->get_inode();
list<CDir*> ls;
- warn_string_dirs << in->get_parent_dn()->get_name() << std::endl;
+ warn_str_dirs << in->get_parent_dn()->get_name() << "\n";
in->get_dirfrags(ls);
for (list<CDir*>::iterator p = ls.begin(); p != ls.end(); ++p) {
CDir *subdir = *p;
- warn_string_dirs << subdir->get_inode()->get_parent_dn()->get_name()
- << std::endl;
+ warn_str_dirs << subdir->get_inode()->get_parent_dn()->get_name() << "\n";
if (subdir->is_subtree_root())
remove_subtree(subdir);
in->close_dirfrag(subdir->dirfrag().frag);
@@ -5287,15 +5283,14 @@ void MDCache::trim_non_auth()
p != ls.end();
++p) {
dout(0) << " ... " << **p << dendl;
- warn_string_dirs << (*p)->get_inode()->get_parent_dn()->get_name()
- << std::endl;
+ warn_str_dirs << (*p)->get_inode()->get_parent_dn()->get_name() << "\n";
assert((*p)->get_num_ref() == 1); // SUBTREE
remove_subtree((*p));
in->close_dirfrag((*p)->dirfrag().frag);
}
dout(0) << " ... " << *in << dendl;
if (in->get_parent_dn())
- warn_string_dirs << in->get_parent_dn()->get_name() << std::endl;
+ warn_str_dirs << in->get_parent_dn()->get_name() << "\n";
assert(in->get_num_ref() == 0);
remove_inode(in);
}
@@ -5304,11 +5299,9 @@ void MDCache::trim_non_auth()
}
show_subtrees();
- if (warn_string_dirs.peek() != EOF) {
- stringstream warn_string;
- warn_string << "trim_non_auth has deleted paths: " << std::endl;
- warn_string << warn_string_dirs;
- mds->logclient.log(LOG_INFO, warn_string);
+ if (warn_str_dirs.peek() != EOF) {
+ mds->clog.info() << "trim_non_auth has deleted paths: " << "\n";
+ mds->clog.info(warn_str_dirs);
}
}
diff --git a/src/mds/MDLog.cc b/src/mds/MDLog.cc
index 75ddc1dac1c..109263d0f8f 100644
--- a/src/mds/MDLog.cc
+++ b/src/mds/MDLog.cc
@@ -30,7 +30,7 @@
#undef DOUT_COND
#define DOUT_COND(l) l<=g_conf.debug_mds || l <= g_conf.debug_mds_log
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << mds->get_nodeid() << ".log "
+#define dout_prefix *_dout << "mds" << mds->get_nodeid() << ".log "
// cons/des
diff --git a/src/mds/MDS.cc b/src/mds/MDS.cc
index 30dccd81a6b..8534e9c68da 100644
--- a/src/mds/MDS.cc
+++ b/src/mds/MDS.cc
@@ -69,7 +69,7 @@
#define DOUT_SUBSYS mds
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << whoami << '.' << incarnation << ' '
+#define dout_prefix *_dout << "mds" << whoami << '.' << incarnation << ' '
@@ -83,7 +83,7 @@ MDS::MDS(const char *n, Messenger *m, MonClient *mc) :
standby_for_rank(-1),
messenger(m),
monc(mc),
- logclient(messenger, &mc->monmap, mc),
+ clog(messenger, &mc->monmap, mc, LogClient::NO_FLAGS),
sessionmap(this) {
orig_argc = 0;
@@ -439,7 +439,7 @@ int MDS::init(int wanted_state)
dout(10) << sizeof(xlist<void*>::item) << "\t xlist<>::item *2=" << 2*sizeof(xlist<void*>::item) << dendl;
messenger->add_dispatcher_tail(this);
- messenger->add_dispatcher_head(&logclient);
+ messenger->add_dispatcher_head(&clog);
// get monmap
monc->set_messenger(messenger);
@@ -494,9 +494,7 @@ void MDS::tick()
// reschedule
reset_tick();
- _dout_check_log();
-
- logclient.send_log();
+ clog.send_log();
utime_t now = g_clock.now();
utime_t delay = now;
@@ -740,18 +738,16 @@ void MDS::handle_command(MMonCommand *m)
} else dout(0) << "bad migrate_dir target syntax" << dendl;
} else dout(0) << "bad migrate_dir syntax" << dendl;
} else if (m->cmd.size() == 1 && m->cmd[0] == "heapdump"){
- stringstream ss;
if (g_conf.tcmalloc_have) {
if (!g_conf.profiler_running()) {
- ss << g_conf.name << " can't dump heap: profiler not running";
+ clog.info() << g_conf.name << " can't dump heap: profiler not running\n";
} else {
- ss << g_conf.name << " dumping heap profile now";
+ clog.info() << g_conf.name << " dumping heap profile now\n";
g_conf.profiler_dump("admin request");
}
} else {
- ss << "tcmalloc not enabled, can't use profiler";
+ clog.info() << "tcmalloc not enabled, can't use profiler\n";
}
- logclient.log(LOG_INFO, ss);
} else if (m->cmd.size() == 1 && m->cmd[0] == "enable_profiler_options") {
char val[sizeof(int)*8+1];
snprintf(val, sizeof(val), "%i", g_conf.profiler_allocation_interval);
@@ -760,22 +756,16 @@ void MDS::handle_command(MMonCommand *m)
snprintf(val, sizeof(val), "%i", g_conf.profiler_highwater_interval);
setenv("HEAP_PROFILE_INUSE_INTERVAL",
val, g_conf.profiler_highwater_interval);
- stringstream ss;
- ss << g_conf.name << " set heap variables from current config";
- logclient.log(LOG_INFO, ss);
+ clog.info() << g_conf.name << " set heap variables from current config\n";
} else if (m->cmd.size() == 1 && m->cmd[0] == "start_profiler") {
char location[PATH_MAX];
snprintf(location, sizeof(location),
"%s/%s", g_conf.log_dir, g_conf.name);
g_conf.profiler_start(location);
- stringstream ss;
- ss << g_conf.name << " started profiler";
- logclient.log(LOG_INFO, ss);
+ clog.info() << g_conf.name << " started profiler\n";
} else if (m->cmd.size() == 1 && m->cmd[0] == "stop_profiler") {
g_conf.profiler_stop();
- stringstream ss;
- ss << g_conf.name << " stopped profiler";
- logclient.log(LOG_INFO, ss);
+ clog.info() << g_conf.name << " stopped profiler\n";
}
else dout(0) << "unrecognized command! " << m->cmd << dendl;
m->put();
diff --git a/src/mds/MDS.h b/src/mds/MDS.h
index 868bc78a008..8b982cf7202 100644
--- a/src/mds/MDS.h
+++ b/src/mds/MDS.h
@@ -155,7 +155,7 @@ class MDS : public Dispatcher {
OSDMap *osdmap;
Objecter *objecter;
Filer *filer; // for reading/writing to/from osds
- LogClient logclient;
+ LogClient clog;
// sub systems
Server *server;
diff --git a/src/mds/MDSTable.cc b/src/mds/MDSTable.cc
index 4ea7b336080..31dd74815d5 100644
--- a/src/mds/MDSTable.cc
+++ b/src/mds/MDSTable.cc
@@ -26,7 +26,7 @@
#define DOUT_SUBSYS mds
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << mds->get_nodeid() << "." << table_name << ": "
+#define dout_prefix *_dout << "mds" << mds->get_nodeid() << "." << table_name << ": "
class C_MT_Save : public Context {
diff --git a/src/mds/MDSTableClient.cc b/src/mds/MDSTableClient.cc
index a01390412bd..7bdd9734121 100644
--- a/src/mds/MDSTableClient.cc
+++ b/src/mds/MDSTableClient.cc
@@ -34,7 +34,7 @@ using std::cerr;
#define DOUT_SUBSYS mds
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << mds->get_nodeid() << ".tableclient(" << get_mdstable_name(table) << ") "
+#define dout_prefix *_dout << "mds" << mds->get_nodeid() << ".tableclient(" << get_mdstable_name(table) << ") "
void MDSTableClient::handle_request(class MMDSTableRequest *m)
diff --git a/src/mds/MDSTableServer.cc b/src/mds/MDSTableServer.cc
index d430098656c..84c000eff38 100644
--- a/src/mds/MDSTableServer.cc
+++ b/src/mds/MDSTableServer.cc
@@ -22,7 +22,7 @@
#define DOUT_SUBSYS mds
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << mds->get_nodeid() << ".tableserver(" << get_mdstable_name(table) << ") "
+#define dout_prefix *_dout << "mds" << mds->get_nodeid() << ".tableserver(" << get_mdstable_name(table) << ") "
/* This function DOES put the passed message before returning */
void MDSTableServer::handle_request(MMDSTableRequest *req)
diff --git a/src/mds/Migrator.cc b/src/mds/Migrator.cc
index 046eff57332..8b17df46ab5 100644
--- a/src/mds/Migrator.cc
+++ b/src/mds/Migrator.cc
@@ -63,7 +63,7 @@
#undef DOUT_COND
#define DOUT_COND(l) l <= g_conf.debug_mds || l <= g_conf.debug_mds_migrator
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << mds->get_nodeid() << ".migrator "
+#define dout_prefix *_dout << "mds" << mds->get_nodeid() << ".migrator "
/* This function DOES put the passed message before returning*/
void Migrator::dispatch(Message *m)
diff --git a/src/mds/Server.cc b/src/mds/Server.cc
index effca20a959..ec52ea28027 100644
--- a/src/mds/Server.cc
+++ b/src/mds/Server.cc
@@ -62,7 +62,7 @@ using namespace std;
#define DOUT_SUBSYS mds
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << mds->get_nodeid() << ".server "
+#define dout_prefix *_dout << "mds" << mds->get_nodeid() << ".server "
void Server::open_logger()
@@ -471,12 +471,10 @@ void Server::find_idle_sessions()
break;
}
- stringstream ss;
utime_t age = now;
age -= session->last_cap_renew;
- ss << "closing stale session " << session->inst << " after " << age;
- mds->logclient.log(LOG_INFO, ss);
-
+ mds->clog.info() << "closing stale session " << session->inst
+ << " after " << age << "\n";
dout(10) << "autoclosing stale session " << session->inst << " last " << session->last_cap_renew << dendl;
kill_session(session);
}
@@ -534,7 +532,6 @@ void Server::handle_client_reconnect(MClientReconnect *m)
return;
}
- stringstream ss;
utime_t delay = g_clock.now();
delay -= reconnect_start;
dout(10) << " reconnect_start " << reconnect_start << " delay " << delay << dendl;
@@ -542,10 +539,10 @@ void Server::handle_client_reconnect(MClientReconnect *m)
if (!mds->is_reconnect()) {
// XXX maybe in the future we can do better than this?
dout(1) << " no longer in reconnect state, ignoring reconnect, sending close" << dendl;
- ss << "denied reconnect attempt (mds is " << ceph_mds_state_name(mds->get_state())
+ mds->clog.info() << "denied reconnect attempt (mds is "
+ << ceph_mds_state_name(mds->get_state())
<< ") from " << m->get_source_inst()
- << " after " << delay << " (allowed interval " << g_conf.mds_reconnect_timeout << ")";
- mds->logclient.log(LOG_INFO, ss);
+ << " after " << delay << " (allowed interval " << g_conf.mds_reconnect_timeout << ")\n";
mds->messenger->send_message(new MClientSession(CEPH_SESSION_CLOSE), m->get_connection());
m->put();
return;
@@ -563,11 +560,12 @@ void Server::handle_client_reconnect(MClientReconnect *m)
mdlog->start_submit_entry(new ESession(session->inst, true, pv),
new C_MDS_session_finish(mds, session, sseq, true, pv));
mdlog->flush();
- ss << "reconnect by new " << session->inst << " after " << delay;
+ mds->clog.debug() << "reconnect by new " << session->inst
+ << " after " << delay << "\n";
} else {
- ss << "reconnect by " << session->inst << " after " << delay;
+ mds->clog.debug() << "reconnect by " << session->inst
+ << " after " << delay << "\n";
}
- mds->logclient.log(LOG_DEBUG, ss);
// snaprealms
for (vector<ceph_mds_snaprealm_reconnect>::iterator p = m->realms.begin();
@@ -1634,10 +1632,9 @@ CInode* Server::prepare_new_inode(MDRequest *mdr, CDir *dir, inodeno_t useino, u
if (useino && useino != in->inode.ino) {
dout(0) << "WARNING: client specified " << useino << " and i allocated " << in->inode.ino << dendl;
- stringstream ss;
- ss << mdr->client_request->get_source() << " specified ino " << useino
- << " but mds" << mds->whoami << " allocated " << in->inode.ino;
- mds->logclient.log(LOG_ERROR, ss);
+ mds->clog.error() << mdr->client_request->get_source()
+ << " specified ino " << useino
+ << " but mds" << mds->whoami << " allocated " << in->inode.ino << "\n";
//assert(0); // just for now.
}
diff --git a/src/mds/SessionMap.cc b/src/mds/SessionMap.cc
index 2aa40595b52..1c4dc11e38e 100644
--- a/src/mds/SessionMap.cc
+++ b/src/mds/SessionMap.cc
@@ -21,7 +21,7 @@
#define DOUT_SUBSYS mds
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << mds->get_nodeid() << ".sessionmap "
+#define dout_prefix *_dout << "mds" << mds->get_nodeid() << ".sessionmap "
void SessionMap::dump()
diff --git a/src/mds/SnapServer.cc b/src/mds/SnapServer.cc
index 91518141072..97778830ed6 100644
--- a/src/mds/SnapServer.cc
+++ b/src/mds/SnapServer.cc
@@ -27,7 +27,7 @@
#define DOUT_SUBSYS mds
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << mds->get_nodeid() << ".snap "
+#define dout_prefix *_dout << "mds" << mds->get_nodeid() << ".snap "
void SnapServer::reset_state()
diff --git a/src/mds/journal.cc b/src/mds/journal.cc
index 486356b7ccb..62afa550e1e 100644
--- a/src/mds/journal.cc
+++ b/src/mds/journal.cc
@@ -55,7 +55,7 @@
#undef DOUT_COND
#define DOUT_COND(l) l<=g_conf.debug_mds || l <= g_conf.debug_mds_log || l <= g_conf.debug_mds_log_expire
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "mds" << mds->get_nodeid() << ".journal "
+#define dout_prefix *_dout << "mds" << mds->get_nodeid() << ".journal "
// -----------------------
@@ -463,8 +463,8 @@ void EMetaBlob::replay(MDS *mds, LogSegment *logseg)
ss << "EMetaBlob.replay FIXME had dentry linked to wrong inode " << *dn
<< " " << *old_in
<< " should be " << p->inode.ino;
- dout(-10) << ss << dendl;
- mds->logclient.log(LOG_WARN, ss);
+ dout(-10) << ss.str() << dendl;
+ mds->clog.warn(ss);
dir->unlink_inode(dn);
mds->mdcache->remove_inode_recursive(old_in);
@@ -579,9 +579,8 @@ void EMetaBlob::replay(MDS *mds, LogSegment *logseg)
// [repair bad inotable updates]
if (inotablev > mds->inotable->get_version()) {
- stringstream ss;
- ss << "journal replay inotablev mismatch " << mds->inotable->get_version() << " -> " << inotablev;
- mds->logclient.log(LOG_ERROR, ss);
+ mds->clog.error() << "journal replay inotablev mismatch "
+ << mds->inotable->get_version() << " -> " << inotablev << "\n";
mds->inotable->force_replay_version(inotablev);
}
@@ -605,9 +604,8 @@ void EMetaBlob::replay(MDS *mds, LogSegment *logseg)
inodeno_t next = session->next_ino();
inodeno_t i = session->take_ino(used_preallocated_ino);
if (next != i) {
- stringstream ss;
- ss << " replayed op " << client_reqs << " used ino " << i << " but session next is " << next;
- mds->logclient.log(LOG_WARN, ss);
+ mds->clog.warn() << " replayed op " << client_reqs << " used ino " << i
+ << " but session next is " << next << "\n";
}
assert(i == used_preallocated_ino);
session->used_inos.clear();
diff --git a/src/mds/snap.cc b/src/mds/snap.cc
index 8d465f7c6f0..2061c58a85e 100644
--- a/src/mds/snap.cc
+++ b/src/mds/snap.cc
@@ -26,8 +26,7 @@
#undef dout_prefix
#define dout_prefix _prefix(mdcache->mds->get_nodeid(), inode, srnode.seq, this)
static ostream& _prefix(int whoami, CInode *inode, uint64_t seq, SnapRealm *realm) {
- return *_dout << dbeginl
- << " mds" << whoami
+ return *_dout << " mds" << whoami
<< ".cache.snaprealm(" << inode->ino()
<< " seq " << seq << " " << realm << ") ";
}
diff --git a/src/messages/MLog.h b/src/messages/MLog.h
index 2a6fab5c1cd..604df33d473 100644
--- a/src/messages/MLog.h
+++ b/src/messages/MLog.h
@@ -15,16 +15,19 @@
#ifndef CEPH_MLOG_H
#define CEPH_MLOG_H
-#include "include/LogEntry.h"
+#include "common/LogEntry.h"
#include "messages/PaxosServiceMessage.h"
+#include <deque>
+
class MLog : public PaxosServiceMessage {
public:
ceph_fsid_t fsid;
- deque<LogEntry> entries;
+ std::deque<LogEntry> entries;
MLog() : PaxosServiceMessage(MSG_LOG, 0) {}
- MLog(ceph_fsid_t& f, deque<LogEntry>& e) : PaxosServiceMessage(MSG_LOG, 0), fsid(f), entries(e) { }
+ MLog(ceph_fsid_t& f, const std::deque<LogEntry>& e)
+ : PaxosServiceMessage(MSG_LOG, 0), fsid(f), entries(e) { }
MLog(ceph_fsid_t& f) : PaxosServiceMessage(MSG_LOG, 0), fsid(f) {}
private:
~MLog() {}
diff --git a/src/messages/MLogAck.h b/src/messages/MLogAck.h
index 1617096230d..27b098276a1 100644
--- a/src/messages/MLogAck.h
+++ b/src/messages/MLogAck.h
@@ -15,8 +15,6 @@
#ifndef CEPH_MLOGACK_H
#define CEPH_MLOGACK_H
-#include "include/LogEntry.h"
-
class MLogAck : public Message {
public:
ceph_fsid_t fsid;
diff --git a/src/mon/AuthMonitor.cc b/src/mon/AuthMonitor.cc
index 5c1bfc04aaa..b22b5577fbb 100644
--- a/src/mon/AuthMonitor.cc
+++ b/src/mon/AuthMonitor.cc
@@ -38,8 +38,7 @@
#undef dout_prefix
#define dout_prefix _prefix(mon, paxos->get_version())
static ostream& _prefix(Monitor *mon, version_t v) {
- return *_dout << dbeginl
- << "mon." << mon->name << "@" << mon->rank
+ return *_dout << "mon." << mon->name << "@" << mon->rank
<< (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)")))
<< ".auth v" << v << " ";
}
diff --git a/src/mon/ClassMonitor.cc b/src/mon/ClassMonitor.cc
index 7899652bb3a..7413599d9a0 100644
--- a/src/mon/ClassMonitor.cc
+++ b/src/mon/ClassMonitor.cc
@@ -34,8 +34,7 @@
#undef dout_prefix
#define dout_prefix _prefix(mon, paxos->get_version())
static ostream& _prefix(Monitor *mon, version_t v) {
- return *_dout << dbeginl
- << "mon." << mon->name << "@" << mon->rank
+ return *_dout << "mon." << mon->name << "@" << mon->rank
<< (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)")))
<< ".class v" << v << " ";
}
diff --git a/src/mon/Elector.cc b/src/mon/Elector.cc
index c51cbbac035..16a5a365989 100644
--- a/src/mon/Elector.cc
+++ b/src/mon/Elector.cc
@@ -26,8 +26,7 @@
#undef dout_prefix
#define dout_prefix _prefix(mon, epoch)
static ostream& _prefix(Monitor *mon, epoch_t epoch) {
- return *_dout << dbeginl
- << "mon." << mon->name << "@" << mon->rank
+ return *_dout << "mon." << mon->name << "@" << mon->rank
<< (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)")))
<< ".elector(" << epoch << ") ";
}
diff --git a/src/mon/LogMonitor.cc b/src/mon/LogMonitor.cc
index 1f2cd1c294a..a29b08e9dee 100644
--- a/src/mon/LogMonitor.cc
+++ b/src/mon/LogMonitor.cc
@@ -33,8 +33,7 @@
#undef dout_prefix
#define dout_prefix _prefix(mon, paxos->get_version())
static ostream& _prefix(Monitor *mon, version_t v) {
- return *_dout << dbeginl
- << "mon." << mon->name << "@" << mon->rank
+ return *_dout << "mon." << mon->name << "@" << mon->rank
<< (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)")))
<< ".log v" << v << " ";
}
@@ -83,7 +82,7 @@ void LogMonitor::create_initial(bufferlist& bl)
LogEntry e;
memset(&e.who, 0, sizeof(e.who));
e.stamp = g_clock.now();
- e.type = LOG_ERROR;
+ e.type = CLOG_ERROR;
e.msg = "mkfs";
e.seq = 0;
pending_log.insert(pair<utime_t,LogEntry>(e.stamp, e));
@@ -134,15 +133,15 @@ bool LogMonitor::update_from_paxos()
s += "\n";
blog.append(s);
- if (le.type >= LOG_DEBUG)
+ if (le.type >= CLOG_DEBUG)
blogdebug.append(s);
- if (le.type >= LOG_INFO)
+ if (le.type >= CLOG_INFO)
bloginfo.append(s);
- if (le.type == LOG_SEC)
+ if (le.type == CLOG_SEC)
blogsec.append(s);
- if (le.type >= LOG_WARN)
+ if (le.type >= CLOG_WARN)
blogwarn.append(s);
- if (le.type >= LOG_ERROR)
+ if (le.type >= CLOG_ERROR)
blogerr.append(s);
summary.add(le);
diff --git a/src/mon/LogMonitor.h b/src/mon/LogMonitor.h
index 91e0185b132..330a50a4e31 100644
--- a/src/mon/LogMonitor.h
+++ b/src/mon/LogMonitor.h
@@ -23,7 +23,7 @@ using namespace std;
#include "msg/Messenger.h"
#include "PaxosService.h"
-#include "include/LogEntry.h"
+#include "common/LogEntry.h"
class MMonCommand;
class MLog;
diff --git a/src/mon/MDSMonitor.cc b/src/mon/MDSMonitor.cc
index adec992414c..c59104fa1f1 100644
--- a/src/mon/MDSMonitor.cc
+++ b/src/mon/MDSMonitor.cc
@@ -36,8 +36,7 @@
#undef dout_prefix
#define dout_prefix _prefix(mon, mdsmap)
static ostream& _prefix(Monitor *mon, MDSMap& mdsmap) {
- return *_dout << dbeginl
- << "mon." << mon->name << "@" << mon->rank
+ return *_dout << "mon." << mon->name << "@" << mon->rank
<< (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)")))
<< ".mds e" << mdsmap.get_epoch() << " ";
}
@@ -404,9 +403,8 @@ bool MDSMonitor::should_propose(double& delay)
void MDSMonitor::_updated(MMDSBeacon *m)
{
dout(10) << "_updated " << m->get_orig_source() << " " << *m << dendl;
- stringstream ss;
- ss << m->get_orig_source_inst() << " " << ceph_mds_state_name(m->get_state());
- mon->get_logclient()->log(LOG_INFO, ss);
+ mon->clog.info() << m->get_orig_source_inst() << " "
+ << ceph_mds_state_name(m->get_state()) << "\n";
if (m->get_state() == MDSMap::STATE_STOPPED) {
// send the map manually (they're out of the map, so they won't get it automatic)
diff --git a/src/mon/MonClient.cc b/src/mon/MonClient.cc
index 8d9ea06a32c..0c95ec265e7 100644
--- a/src/mon/MonClient.cc
+++ b/src/mon/MonClient.cc
@@ -36,7 +36,7 @@
#define DOUT_SUBSYS monc
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "monclient" << (hunting ? "(hunting)":"") << ": "
+#define dout_prefix *_dout << "monclient" << (hunting ? "(hunting)":"") << ": "
/*
diff --git a/src/mon/Monitor.cc b/src/mon/Monitor.cc
index dbaad259633..8f9168cd808 100644
--- a/src/mon/Monitor.cc
+++ b/src/mon/Monitor.cc
@@ -68,8 +68,7 @@
#undef dout_prefix
#define dout_prefix _prefix(this)
static ostream& _prefix(Monitor *mon) {
- return *_dout << dbeginl
- << "mon." << mon->name << "@" << mon->rank
+ return *_dout << "mon." << mon->name << "@" << mon->rank
<< (mon->is_starting() ?
(const char*)"(starting)" :
(mon->is_leader() ?
@@ -95,7 +94,7 @@ Monitor::Monitor(string nm, MonitorStore *s, Messenger *m, MonMap *map) :
lock("Monitor::lock"),
timer(lock),
monmap(map),
- logclient(messenger, monmap),
+ clog(messenger, monmap, NULL, LogClient::FLAG_SYNC),
store(s),
state(STATE_STARTING), stopping(false),
@@ -163,11 +162,9 @@ void Monitor::init()
for (vector<PaxosService*>::iterator ps = paxos_service.begin(); ps != paxos_service.end(); ps++)
(*ps)->init();
- logclient.set_synchronous(true);
-
// i'm ready!
messenger->add_dispatcher_tail(this);
- messenger->add_dispatcher_head(&logclient);
+ messenger->add_dispatcher_head(&clog);
// start ticker
timer.init();
@@ -208,9 +205,7 @@ void Monitor::call_election(bool is_new)
rank = monmap->get_rank(name);
if (is_new) {
- stringstream ss;
- ss << "mon." << name << " calling new monitor election";
- logclient.log(LOG_INFO, ss);
+ clog.info() << "mon." << name << " calling new monitor election\n";
}
dout(10) << "call_election" << dendl;
@@ -253,9 +248,8 @@ void Monitor::win_election(epoch_t epoch, set<int>& active)
quorum = active;
dout(10) << "win_election, epoch " << epoch << " quorum is " << quorum << dendl;
- stringstream ss;
- ss << "mon." << name << "@" << rank << " won leader election with quorum " << quorum;
- logclient.log(LOG_INFO, ss);
+ clog.info() << "mon." << name << "@" << rank
+ << " won leader election with quorum " << quorum << "\n";
for (vector<Paxos*>::iterator p = paxos.begin(); p != paxos.end(); p++)
(*p)->leader_init();
@@ -923,8 +917,6 @@ void Monitor::new_tick()
void Monitor::tick()
{
- _dout_check_log();
-
// ok go.
dout(11) << "tick" << dendl;
@@ -961,12 +953,12 @@ int Monitor::mkfs(bufferlist& osdmapbl)
{
// create it
int err = store->mkfs();
- if (err < 0) {
- char buf[80];
- cerr << "error " << err << " " << strerror_r(err, buf, sizeof(buf)) << std::endl;
+ if (err) {
+ dout(0) << TEXT_RED << "** ERROR: store->mkfs failed with error code "
+ << err << ". Aborting." << dendl;
exit(1);
}
-
+
bufferlist magicbl;
magicbl.append(CEPH_MON_ONDISK_MAGIC);
magicbl.append("\n");
@@ -974,9 +966,9 @@ int Monitor::mkfs(bufferlist& osdmapbl)
store->put_bl_ss(magicbl, "magic", 0);
}
catch (const MonitorStore::Error &e) {
- std::cerr << TEXT_RED << "** ERROR: initializing cmon failed: couldn't "
+ dout(0) << TEXT_RED << "** ERROR: initializing cmon failed: couldn't "
<< "initialize the monitor state machine: "
- << e.what() << TEXT_NORMAL << std::endl;
+ << e.what() << TEXT_NORMAL << dendl;
exit(1);
}
diff --git a/src/mon/Monitor.h b/src/mon/Monitor.h
index 0c1b1c5e11c..358bcfc0104 100644
--- a/src/mon/Monitor.h
+++ b/src/mon/Monitor.h
@@ -67,7 +67,7 @@ public:
MonMap *monmap;
- LogClient logclient;
+ LogClient clog;
KeyServer key_server;
private:
@@ -227,8 +227,6 @@ public:
void stop_cluster();
int mkfs(bufferlist& osdmapbl);
-
- LogClient *get_logclient() { return &logclient; }
};
int strict_strtol(const char *str, int base, std::string *err);
diff --git a/src/mon/MonitorStore.cc b/src/mon/MonitorStore.cc
index 17ca16f2d1d..7984432c629 100644
--- a/src/mon/MonitorStore.cc
+++ b/src/mon/MonitorStore.cc
@@ -21,7 +21,7 @@
#undef dout_prefix
#define dout_prefix _prefix(dir)
static ostream& _prefix(const string& dir) {
- return *_dout << dbeginl << "store(" << dir << ") ";
+ return *_dout << "store(" << dir << ") ";
}
@@ -64,7 +64,7 @@ int MonitorStore::mount()
// verify dir exists
DIR *d = ::opendir(dir.c_str());
if (!d) {
- derr(1) << "basedir " << dir << " dne" << dendl;
+ dout(1) << "basedir " << dir << " dne" << dendl;
return -ENOENT;
}
::closedir(d);
@@ -82,7 +82,7 @@ int MonitorStore::mount()
l.l_len = 0;
int r = ::fcntl(lock_fd, F_SETLK, &l);
if (r < 0) {
- derr(0) << "failed to lock " << t << ", is another cmon still running?" << dendl;
+ dout(0) << "failed to lock " << t << ", is another cmon still running?" << dendl;
return -errno;
}
@@ -106,13 +106,20 @@ int MonitorStore::umount()
int MonitorStore::mkfs()
{
- dout(1) << "mkfs" << dendl;
-
char cmd[1024];
- snprintf(cmd, sizeof(cmd), "test -d %s && /bin/rm -rf %s ; mkdir -p %s", dir.c_str(), dir.c_str(), dir.c_str());
- dout(1) << cmd << dendl;
- int r = system(cmd);
- return r;
+ snprintf(cmd, sizeof(cmd), "test -d %s && /bin/rm -rf %s ; mkdir -p %s",
+ dir.c_str(), dir.c_str(), dir.c_str());
+ dout(6) << "MonitorStore::mkfs: running command '" << cmd << "'" << dendl;
+ int res = system(cmd);
+ int r = WEXITSTATUS(res);
+ if (r) {
+ dout(0) << "FAILED to create monfs at " << dir.c_str() << " for "
+ << g_conf.id << ": cmd '" << cmd << "'" << dendl;
+ return r;
+ }
+
+ dout(0) << "created monfs at " << dir.c_str() << " for " << g_conf.id << dendl;
+ return 0;
}
void MonitorStore::sync()
@@ -245,7 +252,7 @@ int MonitorStore::get_bl_ss(bufferlist& bl, const char *a, const char *b)
int r = ::read(fd, bp.c_str()+off, len-off);
if (r < 0) {
char buf[80];
- derr(0) << "errno on read " << strerror_r(errno, buf, sizeof(buf)) << dendl;
+ dout(0) << "errno on read " << strerror_r(errno, buf, sizeof(buf)) << dendl;
}
assert(r>0);
off += r;
diff --git a/src/mon/MonmapMonitor.cc b/src/mon/MonmapMonitor.cc
index 914d534cab5..c575163d19b 100644
--- a/src/mon/MonmapMonitor.cc
+++ b/src/mon/MonmapMonitor.cc
@@ -28,8 +28,7 @@
#undef dout_prefix
#define dout_prefix _prefix(mon)
static ostream& _prefix(Monitor *mon) {
- return *_dout << dbeginl
- << "mon." << mon->name << "@" << mon->rank
+ return *_dout << "mon." << mon->name << "@" << mon->rank
<< (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)")))
<< ".monmap v" << mon->monmap->epoch << " ";
}
diff --git a/src/mon/OSDMonitor.cc b/src/mon/OSDMonitor.cc
index d69f9e64e55..3180697bc50 100644
--- a/src/mon/OSDMonitor.cc
+++ b/src/mon/OSDMonitor.cc
@@ -42,8 +42,7 @@
#undef dout_prefix
#define dout_prefix _prefix(mon, osdmap)
static ostream& _prefix(Monitor *mon, OSDMap& osdmap) {
- return *_dout << dbeginl
- << "mon." << mon->name << "@" << mon->rank
+ return *_dout << "mon." << mon->name << "@" << mon->rank
<< (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)")))
<< ".osd e" << osdmap.get_epoch() << " ";
}
@@ -374,12 +373,10 @@ bool OSDMonitor::preprocess_failure(MOSDFailure *m)
bool OSDMonitor::prepare_failure(MOSDFailure *m)
{
- stringstream ss;
dout(1) << "prepare_failure " << m->get_target() << " from " << m->get_orig_source_inst()
<< " is reporting failure:" << m->if_osd_failed() << dendl;
-
- ss << m->get_target() << " failed (by " << m->get_orig_source_inst() << ")";
- mon->get_logclient()->log(LOG_INFO, ss);
+ mon->clog.info() << m->get_target() << " failed (by "
+ << m->get_orig_source_inst() << ")\n";
int target_osd = m->get_target().name.num();
int reporter = m->get_orig_source().num();
@@ -569,9 +566,7 @@ void OSDMonitor::_booted(MOSDBoot *m, bool logit)
<< " w " << m->sb.weight << " from " << m->sb.current_epoch << dendl;
if (logit) {
- stringstream ss;
- ss << m->get_orig_source_inst() << " boot";
- mon->get_logclient()->log(LOG_INFO, ss);
+ mon->clog.info() << m->get_orig_source_inst() << " boot\n";
}
send_latest(m, m->sb.current_epoch+1);
@@ -618,9 +613,7 @@ bool OSDMonitor::prepare_alive(MOSDAlive *m)
int from = m->get_orig_source().num();
if (0) { // we probably don't care much about these
- stringstream ss;
- ss << m->get_orig_source_inst() << " alive";
- mon->get_logclient()->log(LOG_DEBUG, ss);
+ mon->clog.debug() << m->get_orig_source_inst() << " alive\n";
}
dout(7) << "prepare_alive e" << m->map_epoch << " from " << m->get_orig_source_inst() << dendl;
@@ -932,9 +925,7 @@ void OSDMonitor::tick()
pending_inc.new_weight[o] = CEPH_OSD_OUT;
do_propose = true;
- stringstream ss;
- ss << "osd" << o << " out (down for " << down << ")";
- mon->get_logclient()->log(LOG_INFO, ss);
+ mon->clog.info() << "osd" << o << " out (down for " << down << ")\n";
} else
continue;
}
diff --git a/src/mon/PGMonitor.cc b/src/mon/PGMonitor.cc
index 0e6fc312eb8..1cf0b27bc0a 100644
--- a/src/mon/PGMonitor.cc
+++ b/src/mon/PGMonitor.cc
@@ -42,8 +42,7 @@
#undef dout_prefix
#define dout_prefix _prefix(mon, pg_map)
static ostream& _prefix(Monitor *mon, PGMap& pg_map) {
- return *_dout << dbeginl
- << "mon." << mon->name << "@" << mon->rank
+ return *_dout << "mon." << mon->name << "@" << mon->rank
<< (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)")))
<< ".pg v" << pg_map.version << " ";
}
@@ -387,9 +386,8 @@ bool PGMonitor::prepare_pg_stats(MPGStats *stats)
<< " state " << pg_state_string(p->second.state)
<< " but DNE in pg_map!!"
<< dendl;
- stringstream ss;
- ss << "got " << pgid << " pg_stat from osd" << from << " but dne in pg_map";
- mon->logclient.log(LOG_ERROR, ss);
+ mon->clog.error() << "got " << pgid << " pg_stat from osd" << from
+ << " but dne in pg_map\n";
continue;
}
diff --git a/src/mon/Paxos.cc b/src/mon/Paxos.cc
index 342f737153e..f0a253b5aed 100644
--- a/src/mon/Paxos.cc
+++ b/src/mon/Paxos.cc
@@ -25,8 +25,7 @@
#undef dout_prefix
#define dout_prefix _prefix(mon, mon->name, mon->rank, machine_name, state, last_committed)
static ostream& _prefix(Monitor *mon, const string& name, int rank, const char *machine_name, int state, version_t last_committed) {
- return *_dout << dbeginl
- << "mon." << name << "@" << rank
+ return *_dout << "mon." << name << "@" << rank
<< (mon->is_starting() ?
(const char*)"(starting)" :
(mon->is_leader() ?
@@ -569,10 +568,8 @@ void Paxos::warn_on_future_time(utime_t t, entity_name_t from)
utime_t warn_diff = now - last_clock_drift_warn;
if (warn_diff >
pow(g_conf.mon_clock_drift_warn_backoff, clock_drift_warned)) {
- stringstream ss;
- ss << "message from " << from << " was stamped " << diff
- << "s in the future, clocks not synchronized";
- mon->get_logclient()->log(LOG_WARN, ss);
+ mon->clog.warn() << "message from " << from << " was stamped " << diff
+ << "s in the future, clocks not synchronized";
last_clock_drift_warn = g_clock.now();
++clock_drift_warned;
}
diff --git a/src/mon/PaxosService.cc b/src/mon/PaxosService.cc
index 61b469a1303..544877c546f 100644
--- a/src/mon/PaxosService.cc
+++ b/src/mon/PaxosService.cc
@@ -24,8 +24,7 @@
#undef dout_prefix
#define dout_prefix _prefix(mon, paxos, paxos->machine_id)
static ostream& _prefix(Monitor *mon, Paxos *paxos, int machine_id) {
- return *_dout << dbeginl
- << "mon." << mon->name << "@" << mon->rank
+ return *_dout << "mon." << mon->name << "@" << mon->rank
<< (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)")))
<< ".paxosservice(" << get_paxos_name(machine_id) << ") ";
}
diff --git a/src/msg/FakeMessenger.cc b/src/msg/FakeMessenger.cc
index 2ee44e74c14..e63d98ca84f 100644
--- a/src/msg/FakeMessenger.cc
+++ b/src/msg/FakeMessenger.cc
@@ -25,9 +25,7 @@
#include "config.h"
-#define dout(x) if ((x) <= g_conf.debug_ms) *_dout << dbeginl << g_clock.now() << " "
-
-
+#define DOUT_SUBSYS ms
#include <stdio.h>
#include <stdlib.h>
diff --git a/src/msg/Message.cc b/src/msg/Message.cc
index a38adb8e0f7..7ee1b8d7ebe 100644
--- a/src/msg/Message.cc
+++ b/src/msg/Message.cc
@@ -153,14 +153,14 @@ Message *decode_message(ceph_msg_header& header, ceph_msg_footer& footer,
if (front_crc != footer.front_crc) {
dout(0) << "bad crc in front " << front_crc << " != exp " << footer.front_crc << dendl;
- dout(20);
+ dout(20) << " ";
front.hexdump(*_dout);
*_dout << dendl;
return 0;
}
if (middle_crc != footer.middle_crc) {
dout(0) << "bad crc in middle " << middle_crc << " != exp " << footer.middle_crc << dendl;
- dout(20);
+ dout(20) << " ";
middle.hexdump(*_dout);
*_dout << dendl;
return 0;
@@ -170,7 +170,7 @@ Message *decode_message(ceph_msg_header& header, ceph_msg_footer& footer,
__u32 data_crc = data.crc32c(0);
if (data_crc != footer.data_crc) {
dout(0) << "bad crc in data " << data_crc << " != exp " << footer.data_crc << dendl;
- dout(20);
+ dout(20) << " ";
data.hexdump(*_dout);
*_dout << dendl;
return 0;
diff --git a/src/msg/SimpleMessenger.cc b/src/msg/SimpleMessenger.cc
index bc40023daf7..846899058ae 100644
--- a/src/msg/SimpleMessenger.cc
+++ b/src/msg/SimpleMessenger.cc
@@ -39,7 +39,7 @@
#undef dout_prefix
#define dout_prefix _prefix(messenger)
static ostream& _prefix(SimpleMessenger *messenger) {
- return *_dout << dbeginl << "-- " << messenger->ms_addr << " ";
+ return *_dout << "-- " << messenger->ms_addr << " ";
}
@@ -66,7 +66,7 @@ int SimpleMessenger::Accepter::bind(int64_t force_nonce, entity_addr_t &bind_add
listen_sd = ::socket(family, SOCK_STREAM, 0);
if (listen_sd < 0) {
char buf[80];
- derr(0) << "accepter.bind unable to create socket: "
+ dout(0) << "accepter.bind unable to create socket: "
<< strerror_r(errno, buf, sizeof(buf)) << dendl;
cerr << "accepter.bind unable to create socket: "
<< strerror_r(errno, buf, sizeof(buf)) << std::endl;
@@ -88,7 +88,7 @@ int SimpleMessenger::Accepter::bind(int64_t force_nonce, entity_addr_t &bind_add
rc = ::bind(listen_sd, (struct sockaddr *) &listen_addr.ss_addr(), sizeof(listen_addr.ss_addr()));
if (rc < 0) {
char buf[80];
- derr(0) << "accepter.bind unable to bind to " << bind_addr.ss_addr()
+ dout(0) << "accepter.bind unable to bind to " << bind_addr.ss_addr()
<< ": " << strerror_r(errno, buf, sizeof(buf)) << dendl;
cerr << "accepter.bind unable to bind to " << bind_addr.ss_addr()
<< ": " << strerror_r(errno, buf, sizeof(buf)) << std::endl;
@@ -106,7 +106,7 @@ int SimpleMessenger::Accepter::bind(int64_t force_nonce, entity_addr_t &bind_add
}
if (rc < 0) {
char buf[80];
- derr(0) << "accepter.bind unable to bind to " << bind_addr.ss_addr()
+ dout(0) << "accepter.bind unable to bind to " << bind_addr.ss_addr()
<< " on any port in range " << CEPH_PORT_START << "-" << CEPH_PORT_LAST
<< ": " << strerror_r(errno, buf, sizeof(buf)) << dendl;
cerr << "accepter.bind unable to bind to " << bind_addr.ss_addr()
@@ -127,7 +127,7 @@ int SimpleMessenger::Accepter::bind(int64_t force_nonce, entity_addr_t &bind_add
rc = ::listen(listen_sd, 128);
if (rc < 0) {
char buf[80];
- derr(0) << "accepter.bind unable to listen on " << bind_addr.ss_addr()
+ dout(0) << "accepter.bind unable to listen on " << bind_addr.ss_addr()
<< ": " << strerror_r(errno, buf, sizeof(buf)) << dendl;
cerr << "accepter.bind unable to listen on " << bind_addr.ss_addr()
<< ": " << strerror_r(errno, buf, sizeof(buf)) << std::endl;
@@ -489,8 +489,7 @@ entity_addr_t SimpleMessenger::get_myaddr()
#undef dout_prefix
#define dout_prefix _pipe_prefix()
ostream& SimpleMessenger::Pipe::_pipe_prefix() {
- return *_dout << dbeginl
- << "-- " << messenger->ms_addr << " >> " << peer_addr << " pipe(" << this
+ return *_dout << "-- " << messenger->ms_addr << " >> " << peer_addr << " pipe(" << this
<< " sd=" << sd
<< " pgs=" << peer_global_seq
<< " cs=" << connect_seq
@@ -1457,7 +1456,7 @@ void SimpleMessenger::Pipe::fault(bool onconnect, bool onread)
void SimpleMessenger::Pipe::fail()
{
- derr(10) << "fail" << dendl;
+ dout(10) << "fail" << dendl;
assert(pipe_lock.is_locked());
stop();
@@ -1720,7 +1719,7 @@ void SimpleMessenger::Pipe::writer()
pipe_lock.Lock();
if (rc < 0) {
- derr(1) << "writer error sending " << m << ", "
+ dout(1) << "writer error sending " << m << ", "
<< errno << ": " << strerror_r(errno, buf, sizeof(buf)) << dendl;
fault();
}
@@ -2373,11 +2372,10 @@ int SimpleMessenger::start(bool nodaemon)
// daemonize?
if (g_conf.daemonize && !nodaemon) {
if (Thread::get_num_threads() > 0) {
- derr(0) << "messenger.start BUG: there are " << Thread::get_num_threads()
+ dout(0) << "messenger.start BUG: there are " << Thread::get_num_threads()
<< " already started that will now die! call messenger.start() sooner."
<< dendl;
}
- dout(1) << "messenger.start daemonizing" << dendl;
if (1) {
daemon(1, 0);
@@ -2398,8 +2396,8 @@ int SimpleMessenger::start(bool nodaemon)
::mkdir(g_conf.chdir, 0700);
::chdir(g_conf.chdir);
}
-
- dout_rename_output_file();
+ dout_handle_daemonize();
+ dout(1) << "messenger.start daemonized" << dendl;
}
// go!
@@ -2565,7 +2563,7 @@ void SimpleMessenger::submit_message(Message *m, const entity_addr_t& dest_addr,
dout(20) << "submit_message " << *m << " local" << dendl;
dispatch_queue.local_delivery(m, m->get_priority());
} else {
- derr(0) << "submit_message " << *m << " " << dest_addr << " local but no local endpoint, dropping." << dendl;
+ dout(0) << "submit_message " << *m << " " << dest_addr << " local but no local endpoint, dropping." << dendl;
assert(0); // hmpf, this is probably mds->mon beacon from newsyn.
m->put();
}
diff --git a/src/os/FileJournal.cc b/src/os/FileJournal.cc
index 7f33c84fa39..585e11c6d4b 100644
--- a/src/os/FileJournal.cc
+++ b/src/os/FileJournal.cc
@@ -26,7 +26,7 @@
#define DOUT_SUBSYS journal
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "journal "
+#define dout_prefix *_dout << "journal "
const static int64_t ONE_MEG(1 << 20);
@@ -321,12 +321,12 @@ int FileJournal::open(uint64_t next_seq)
err = -EINVAL;
}
if (header.alignment != block_size && directio) {
- derr(0) << "open journal alignment " << header.alignment << " does not match block size "
+ dout(0) << "open journal alignment " << header.alignment << " does not match block size "
<< block_size << " (required for direct_io journal mode)" << dendl;
err = -EINVAL;
}
if ((header.alignment % PAGE_SIZE) && directio) {
- derr(0) << "open journal alignment " << header.alignment << " is not multiple of page size " << PAGE_SIZE
+ dout(0) << "open journal alignment " << header.alignment << " is not multiple of page size " << PAGE_SIZE
<< " (required for direct_io journal mode)" << dendl;
err = -EINVAL;
}
@@ -635,7 +635,7 @@ void FileJournal::write_bl(off64_t& pos, bufferlist& bl)
int err = bl.write_fd(fd);
if (err) {
char buf[80];
- derr(0) << "write_bl failed with " << err << " " << strerror_r(-err, buf, sizeof(buf))
+ dout(0) << "write_bl failed with " << err << " " << strerror_r(-err, buf, sizeof(buf))
<< dendl;
}
pos += bl.length();
diff --git a/src/os/FileStore.cc b/src/os/FileStore.cc
index b078692e10a..2da241d7f9d 100644
--- a/src/os/FileStore.cc
+++ b/src/os/FileStore.cc
@@ -71,7 +71,7 @@
#define DOUT_SUBSYS filestore
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "filestore(" << basedir << ") "
+#define dout_prefix *_dout << "filestore(" << basedir << ") "
#include "include/buffer.h"
@@ -699,7 +699,7 @@ int FileStore::lock_fsid()
int r = ::fcntl(fsid_fd, F_SETLK, &l);
if (r < 0) {
char buf[80];
- derr(0) << "lock_fsid failed to lock " << basedir << "/fsid, is another cosd still running? " << strerror_r(errno, buf, sizeof(buf)) << dendl;
+ dout(0) << "lock_fsid failed to lock " << basedir << "/fsid, is another cosd still running? " << strerror_r(errno, buf, sizeof(buf)) << dendl;
return -errno;
}
return 0;
@@ -748,7 +748,7 @@ int FileStore::_detect_fs()
<< " " << strerror(errno)
<< dendl;*/
if (x != y) {
- derr(0) << "xattrs don't appear to work (" << strerror_r(errno, buf, sizeof(buf))
+ dout(0) << "xattrs don't appear to work (" << strerror_r(errno, buf, sizeof(buf))
<< ") on " << fn << ", be sure to mount underlying file system with 'user_xattr' option" << dendl;
return -errno;
}
@@ -1002,7 +1002,7 @@ int FileStore::mount()
struct stat st;
int r = ::stat(basedir.c_str(), &st);
if (r != 0) {
- derr(0) << "unable to stat basedir " << basedir << ", " << strerror_r(errno, buf, sizeof(buf)) << dendl;
+ dout(0) << "unable to stat basedir " << basedir << ", " << strerror_r(errno, buf, sizeof(buf)) << dendl;
return -errno;
}
@@ -1554,14 +1554,14 @@ int FileStore::_transaction_start(uint64_t bytes, uint64_t ops)
char buf[80];
int fd = ::open(basedir.c_str(), O_RDONLY);
if (fd < 0) {
- derr(0) << "transaction_start got " << strerror_r(errno, buf, sizeof(buf))
+ dout(0) << "transaction_start got " << strerror_r(errno, buf, sizeof(buf))
<< " from btrfs open" << dendl;
assert(0);
}
int r = ::ioctl(fd, BTRFS_IOC_TRANS_START);
if (r < 0) {
- derr(0) << "transaction_start got " << strerror_r(errno, buf, sizeof(buf))
+ dout(0) << "transaction_start got " << strerror_r(errno, buf, sizeof(buf))
<< " from btrfs ioctl" << dendl;
::close(fd);
return -errno;
@@ -2007,7 +2007,7 @@ int FileStore::_write(coll_t cid, const sobject_t& oid,
int flags = O_WRONLY|O_CREAT;
int fd = ::open(fn, flags, 0644);
if (fd < 0) {
- derr(0) << "write couldn't open " << fn << " flags " << flags << " errno " << errno << " " << strerror_r(errno, buf, sizeof(buf)) << dendl;
+ dout(0) << "write couldn't open " << fn << " flags " << flags << " errno " << errno << " " << strerror_r(errno, buf, sizeof(buf)) << dendl;
r = -errno;
goto out;
}
diff --git a/src/os/JournalingObjectStore.cc b/src/os/JournalingObjectStore.cc
index 60165a189ca..42d1901df42 100644
--- a/src/os/JournalingObjectStore.cc
+++ b/src/os/JournalingObjectStore.cc
@@ -5,7 +5,7 @@
#define DOUT_SUBSYS journal
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << "journal "
+#define dout_prefix *_dout << "journal "
diff --git a/src/os/ObjectStore.cc b/src/os/ObjectStore.cc
index 016fcceb9e5..67164ad2eca 100644
--- a/src/os/ObjectStore.cc
+++ b/src/os/ObjectStore.cc
@@ -6,7 +6,7 @@
#include "config.h"
#include "common/Clock.h"
-#define dout(x) if (x < g_conf.debug) *_dout << dbeginl << g_clock.now() << " ager: "
+#define dout_prefix *_dout << "ager: "
object_t ObjectStore::age_get_oid() {
if (!age_free_oids.empty()) {
diff --git a/src/osbdb/OSBDB.cc b/src/osbdb/OSBDB.cc
index a5ed2bf0ece..2adc2d5fb1c 100644
--- a/src/osbdb/OSBDB.cc
+++ b/src/osbdb/OSBDB.cc
@@ -19,8 +19,8 @@ Foundation. See file COPYING. */
using namespace std;
-#define dout(x) if (x <= g_conf.debug || x <= g_conf.debug_bdbstore) *_dout << dbeginl << "bdbstore(" << device << ")@" << __LINE__ << "."
-#define derr(x) if (x <= g_conf.debug || x <= g_conf.debug_bdbstore) *_derr << dbeginl << "bdbstore(" << device << ")@" << __LINE__ << "."
+#define dout_prefix *_dout << "bdbstore(" << device << ")@" << __LINE__ << "."
+#define derr(x) dout(x)
#define CLEANUP(onsafe) do { \
dout(6) << "DELETE " << hex << onsafe << dec << dendl; \
diff --git a/src/osd/OSD.cc b/src/osd/OSD.cc
index 5b260dd93a6..42437fc8ad5 100644
--- a/src/osd/OSD.cc
+++ b/src/osd/OSD.cc
@@ -98,7 +98,7 @@
#define dout_prefix _prefix(*_dout, whoami, osdmap)
static ostream& _prefix(ostream& out, int whoami, OSDMap *osdmap) {
- return out << dbeginl << "osd" << whoami << " " << (osdmap ? osdmap->get_epoch():0) << " ";
+ return out << "osd" << whoami << " " << (osdmap ? osdmap->get_epoch():0) << " ";
}
const coll_t coll_t::META_COLL("meta");
@@ -342,7 +342,7 @@ OSD::OSD(int id, Messenger *internal_messenger, Messenger *external_messenger, M
logger(NULL), logger_started(false),
store(NULL),
map_in_progress(false),
- logclient(client_messenger, &mc->monmap, mc),
+ clog(client_messenger, &mc->monmap, mc, LogClient::NO_FLAGS),
whoami(id),
dev_path(dev), journal_path(jdev),
dispatch_running(false),
@@ -518,7 +518,7 @@ int OSD::init()
// i'm ready!
client_messenger->add_dispatcher_head(this);
- client_messenger->add_dispatcher_head(&logclient);
+ client_messenger->add_dispatcher_head(&clog);
cluster_messenger->add_dispatcher_head(this);
heartbeat_messenger->add_dispatcher_head(&heartbeat_dispatcher);
@@ -787,7 +787,7 @@ int OSD::read_superblock()
}
if (whoami != superblock.whoami) {
- derr(0) << "read_superblock superblock says osd" << superblock.whoami
+ dout(0) << "read_superblock superblock says osd" << superblock.whoami
<< ", but i (think i) am osd" << whoami << dendl;
return -1;
}
@@ -1515,8 +1515,6 @@ void OSD::tick()
logger->set(l_osd_buf, buffer_total_alloc.read());
- _dout_check_log();
-
if (got_sigterm) {
dout(0) << "got SIGTERM, shutting down" << dendl;
cluster_messenger->send_message(new MGenericMessage(CEPH_MSG_SHUTDOWN),
@@ -1562,7 +1560,7 @@ void OSD::tick()
map_lock.put_read();
- logclient.send_log();
+ clog.send_log();
timer.add_event_after(1.0, new C_Tick(this));
@@ -1835,59 +1833,50 @@ void OSD::handle_command(MMonCommand *m)
stringstream ss;
uint64_t rate = (double)count / (end - start);
- ss << "bench: wrote " << prettybyte_t(count) << " in blocks of " << prettybyte_t(bsize)
- << " in " << (end-start)
- << " sec at " << prettybyte_t(rate) << "/sec";
- logclient.log(LOG_INFO, ss);
+ clog.info() << "bench: wrote " << prettybyte_t(count)
+ << " in blocks of " << prettybyte_t(bsize) << " in "
+ << (end-start) << " sec at " << prettybyte_t(rate) << "/sec\n";
} else if (m->cmd.size() == 2 && m->cmd[0] == "logger" && m->cmd[1] == "reset") {
logger_reset_all();
} else if (m->cmd.size() == 2 && m->cmd[0] == "logger" && m->cmd[1] == "reopen") {
logger_reopen_all();
} else if (m->cmd.size() == 1 && m->cmd[0] == "heapdump") {
- stringstream ss;
if (g_conf.tcmalloc_have) {
if (!g_conf.profiler_running()) {
- ss << "can't dump heap: profiler not running";
+ clog.info() << "can't dump heap: profiler not running\n";
} else {
- ss << g_conf.name << "dumping heap profile now";
+ clog.info() << g_conf.name << "dumping heap profile now\n";
g_conf.profiler_dump("admin request");
}
} else {
- ss << g_conf.name << " does not have tcmalloc, can't use profiler";
+ clog.info() << g_conf.name << " does not have tcmalloc, "
+ "can't use profiler\n";
}
- logclient.log(LOG_INFO, ss);
} else if (m->cmd.size() == 1 && m->cmd[0] == "enable_profiler_options") {
char val[sizeof(int)*8+1];
snprintf(val, sizeof(val), "%i", g_conf.profiler_allocation_interval);
setenv("HEAP_PROFILE_ALLOCATION_INTERVAL", val, g_conf.profiler_allocation_interval);
snprintf(val, sizeof(val), "%i", g_conf.profiler_highwater_interval);
setenv("HEAP_PROFILE_INUSE_INTERVAL", val, g_conf.profiler_highwater_interval);
- stringstream ss;
- ss << g_conf.name << " set heap variables from current config";
- logclient.log(LOG_INFO, ss);
+ clog.info() << g_conf.name << " set heap variables from current config";
} else if (m->cmd.size() == 1 && m->cmd[0] == "start_profiler") {
char location[PATH_MAX];
snprintf(location, sizeof(location),
"%s/%s", g_conf.log_dir, g_conf.name);
g_conf.profiler_start(location);
- stringstream ss;
- ss << g_conf.name << " started profiler with output " << location;
- logclient.log(LOG_INFO, ss);
+ clog.info() << g_conf.name << " started profiler with output "
+ << location << "\n";
} else if (m->cmd.size() == 1 && m->cmd[0] == "stop_profiler") {
g_conf.profiler_stop();
- stringstream ss;
- ss << g_conf.name << " stopped profiler";
- logclient.log(LOG_INFO, ss);
+ clog.info() << g_conf.name << " stopped profiler\n";
}
else if (m->cmd.size() > 1 && m->cmd[0] == "debug") {
if (m->cmd.size() == 3 && m->cmd[1] == "dump_missing") {
const string &file_name(m->cmd[2]);
std::ofstream fout(file_name.c_str());
if (!fout.is_open()) {
- stringstream ss;
- ss << "failed to open file '" << file_name << "'";
- logclient.log(LOG_INFO, ss);
+ clog.info() << "failed to open file '" << file_name << "'\n";
goto done;
}
@@ -1927,10 +1916,8 @@ void OSD::handle_command(MMonCommand *m)
}
else if (m->cmd.size() == 3 && m->cmd[1] == "kick_recovery_wq") {
g_conf.osd_recovery_delay_start = atoi(m->cmd[2].c_str());
- stringstream ss;
- ss << "kicking recovery queue. set osd_recovery_delay_start to "
- << g_conf.osd_recovery_delay_start;
- logclient.log(LOG_INFO, ss);
+ clog.info() << "kicking recovery queue. set osd_recovery_delay_start "
+ << "to " << g_conf.osd_recovery_delay_start << "\n";
defer_recovery_until = g_clock.now();
defer_recovery_until += g_conf.osd_recovery_delay_start;
@@ -2745,9 +2732,8 @@ void OSD::handle_osd_map(MOSDMap *m)
do_shutdown = true; // don't call shutdown() while we have everything paused
} else if (!osdmap->is_up(whoami) ||
osdmap->get_addr(whoami) != client_messenger->get_myaddr()) {
- stringstream ss;
- ss << "map e" << osdmap->get_epoch() << " wrongly marked me down";
- logclient.log(LOG_WARN, ss);
+ clog.warn() << "map e" << osdmap->get_epoch()
+ << " wrongly marked me down\n";
state = STATE_BOOTING;
up_epoch = 0;
@@ -3589,9 +3575,8 @@ void OSD::handle_pg_create(MOSDPGCreate *m)
}
if (up != acting) {
dout(10) << "mkpg " << pgid << " up " << up << " != acting " << acting << dendl;
- stringstream ss;
- ss << "mkpg " << pgid << " up " << up << " != acting " << acting;
- logclient.log(LOG_ERROR, ss);
+ clog.error() << "mkpg " << pgid << " up " << up << " != acting "
+ << acting << "\n";
continue;
}
@@ -3992,10 +3977,8 @@ void OSD::_process_pg_info(epoch_t epoch, int from,
// did a snap just get purged?
if (info.purged_snaps.size() < pg->info.purged_snaps.size()) {
- stringstream ss;
- ss << "pg " << pg->info.pgid << " replica got purged_snaps " << info.purged_snaps
- << " had " << pg->info.purged_snaps;
- logclient.log(LOG_WARN, ss);
+ clog.warn() << "pg " << pg->info.pgid << " replica got purged_snaps "
+ << info.purged_snaps << " had " << pg->info.purged_snaps << "\n";
pg->info.purged_snaps = info.purged_snaps;
} else {
interval_set<snapid_t> p = info.purged_snaps;
@@ -4751,11 +4734,9 @@ void OSD::handle_misdirected_op(PG *pg, MOSDOp *op)
op->put();
} else {
dout(7) << *pg << " misdirected op in " << op->get_map_epoch() << dendl;
- stringstream ss;
- ss << op->get_source_inst() << " misdirected " << op->get_reqid()
- << " " << pg->info.pgid << " to osd" << whoami
- << " not " << pg->acting;
- logclient.log(LOG_WARN, ss);
+ clog.warn() << op->get_source_inst() << " misdirected "
+ << op->get_reqid() << " " << pg->info.pgid << " to osd" << whoami
+ << " not " << pg->acting << "\n";
reply_op_error(op, -ENXIO);
}
}
@@ -4845,7 +4826,7 @@ void OSD::handle_op(MOSDOp *op)
if (!op->may_write()) {
stat_rd_ops++;
if (op->get_source().is_osd()) {
- //derr(-10) << "shed in " << stat_rd_ops_shed_in << " / " << stat_rd_ops << dendl;
+ //dout(-10) << "shed in " << stat_rd_ops_shed_in << " / " << stat_rd_ops << dendl;
stat_rd_ops_shed_in++;
}
}
diff --git a/src/osd/OSD.h b/src/osd/OSD.h
index 5961add8e9a..fa6db5c8138 100644
--- a/src/osd/OSD.h
+++ b/src/osd/OSD.h
@@ -30,7 +30,6 @@
#include "common/DecayCounter.h"
#include "common/ClassHandler.h"
-#include "include/LogEntry.h"
#include "include/CompatSet.h"
#include "auth/KeyRing.h"
@@ -118,7 +117,7 @@ protected:
Cond *map_in_progress_cond;
bool map_in_progress;
- LogClient logclient;
+ LogClient clog;
int whoami;
const char *dev_path, *journal_path;
@@ -969,8 +968,6 @@ public:
void force_remount();
- LogClient *get_logclient() { return &logclient; }
-
void init_op_flags(MOSDOp *op);
};
diff --git a/src/osd/OSDMap.cc b/src/osd/OSDMap.cc
index 33a6dac4424..c50135b7f2e 100644
--- a/src/osd/OSDMap.cc
+++ b/src/osd/OSDMap.cc
@@ -168,7 +168,7 @@ void OSDMap::build_simple_crush_map(CrushWrapper& crush, map<int, const char*>&
int rweights[ndom];
int nper = ((num_osd - 1) / ndom) + 1;
- derr(0) << ndom << " failure domains, " << nper << " osds each" << dendl;
+ dout(0) << ndom << " failure domains, " << nper << " osds each" << dendl;
int o = 0;
for (int i=0; i<ndom; i++) {
diff --git a/src/osd/PG.cc b/src/osd/PG.cc
index 22a09547659..67f80a75654 100644
--- a/src/osd/PG.cc
+++ b/src/osd/PG.cc
@@ -36,7 +36,7 @@
#undef dout_prefix
#define dout_prefix _prefix(this, osd->whoami, osd->osdmap)
static ostream& _prefix(const PG *pg, int whoami, OSDMap *osdmap) {
- return *_dout << dbeginl << "osd" << whoami << " " << (osdmap ? osdmap->get_epoch():0) << " " << *pg << " ";
+ return *_dout << "osd" << whoami << " " << (osdmap ? osdmap->get_epoch():0) << " " << *pg << " ";
}
/******* PGLog ********/
@@ -2402,9 +2402,8 @@ void PG::read_log(ObjectStore *store)
// [repair] in order?
if (e.version < last) {
dout(0) << "read_log " << pos << " out of order entry " << e << " follows " << last << dendl;
- stringstream ss;
- ss << info.pgid << " log has out of order entry " << e << " following " << last;
- osd->get_logclient()->log(LOG_ERROR, ss);
+ osd->clog.error() << info.pgid << " log has out of order entry "
+ << e << " following " << last << "\n";
reorder = true;
}
@@ -2415,9 +2414,8 @@ void PG::read_log(ObjectStore *store)
if (last.version == e.version.version) {
dout(0) << "read_log got dup " << e.version << " (last was " << last << ", dropping that one)" << dendl;
log.log.pop_back();
- stringstream ss;
- ss << info.pgid << " read_log got dup " << e.version << " after " << last;
- osd->get_logclient()->log(LOG_ERROR, ss);
+ osd->clog.error() << info.pgid << " read_log got dup "
+ << e.version << " after " << last << "\n";
}
uint64_t endpos = ondisklog.tail + p.get_off();
@@ -2428,11 +2426,12 @@ void PG::read_log(ObjectStore *store)
// [repair] at end of log?
if (!p.end() && e.version == info.last_update) {
- stringstream ss;
- ss << info.pgid << " log has extra data at " << endpos << "~" << (ondisklog.head-endpos)
- << " after " << info.last_update;
- osd->get_logclient()->log(LOG_ERROR, ss);
- dout(0) << "read_log " << endpos << " *** extra gunk at end of log, adjusting ondisklog.head" << dendl;
+ osd->clog.error() << info.pgid << " log has extra data at "
+ << endpos << "~" << (ondisklog.head-endpos) << " after "
+ << info.last_update << "\n";
+
+ dout(0) << "read_log " << endpos << " *** extra gunk at end of log, "
+ << "adjusting ondisklog.head" << dendl;
ondisklog.head = endpos;
break;
}
@@ -2547,7 +2546,7 @@ bool PG::check_log_for_corruption(ObjectStore *store)
getline(f, filename);
blb.write_file(filename.c_str(), 0644);
ss << ", saved to " << filename;
- osd->logclient.log(LOG_ERROR, ss);
+ osd->clog.error(ss);
}
return ok;
}
@@ -3060,7 +3059,6 @@ void PG::repair_object(const sobject_t& soid, ScrubMap::object *po, int bad_peer
void PG::scrub()
{
- stringstream ss;
ScrubMap scrubmap;
int errors = 0, fixed = 0;
bool repair = state_test(PG_STATE_REPAIR);
@@ -3195,8 +3193,7 @@ void PG::scrub()
if (anymissing) {
for (unsigned i=0; i<acting.size(); i++) {
if (p[i] == m[i]->objects.end() || *psoid != p[i]->first) {
- ss << info.pgid << " " << mode << " osd" << acting[i] << " missing " << *psoid;
- osd->get_logclient()->log(LOG_ERROR, ss);
+ osd->clog.error() << info.pgid << " " << mode << " osd" << acting[i] << " missing " << *psoid;
num_missing++;
if (repair)
@@ -3214,18 +3211,16 @@ void PG::scrub()
if (po->size != p[i]->second.size) {
dout(0) << "scrub osd" << acting[i] << " " << *psoid
<< " size " << p[i]->second.size << " != " << po->size << dendl;
- ss << info.pgid << " " << mode << " osd" << acting[i] << " " << *psoid
+ osd->clog.error() << info.pgid << " " << mode << " osd" << acting[i] << " " << *psoid
<< " size " << p[i]->second.size << " != " << po->size;
- osd->get_logclient()->log(LOG_ERROR, ss);
peerok = ok = false;
num_bad++;
}
if (po->attrs.size() != p[i]->second.attrs.size()) {
dout(0) << "scrub osd" << acting[i] << " " << *psoid
<< " attr count " << p[i]->second.attrs.size() << " != " << po->attrs.size() << dendl;
- ss << info.pgid << " " << mode << " osd" << acting[i] << " " << *psoid
- << " attr count " << p[i]->second.attrs.size() << " != " << po->attrs.size();
- osd->get_logclient()->log(LOG_ERROR, ss);
+ osd->clog.error() << info.pgid << " " << mode << " osd" << acting[i] << " " << *psoid
+ << " attr count " << p[i]->second.attrs.size() << " != " << po->attrs.size();
peerok = ok = false;
num_bad++;
}
@@ -3234,18 +3229,16 @@ void PG::scrub()
if (q->second.cmp(p[i]->second.attrs[q->first])) {
dout(0) << "scrub osd" << acting[i] << " " << *psoid
<< " attr " << q->first << " value mismatch" << dendl;
- ss << info.pgid << " " << mode << " osd" << acting[i] << " " << *psoid
- << " attr " << q->first << " value mismatch";
- osd->get_logclient()->log(LOG_ERROR, ss);
+ osd->clog.error() << info.pgid << " " << mode << " osd" << acting[i] << " " << *psoid
+ << " attr " << q->first << " value mismatch";
peerok = ok = false;
num_bad++;
}
} else {
dout(0) << "scrub osd" << acting[i] << " " << *psoid
<< " attr " << q->first << " missing" << dendl;
- ss << info.pgid << " " << mode << " osd" << acting[i] << " " << *psoid
+ osd->clog.error() << info.pgid << " " << mode << " osd" << acting[i] << " " << *psoid
<< " attr " << q->first << " missing";
- osd->get_logclient()->log(LOG_ERROR, ss);
peerok = ok = false;
num_bad++;
}
@@ -3264,9 +3257,12 @@ void PG::scrub()
}
if (num_missing || num_bad) {
- dout(0) << "scrub " << num_missing << " missing, " << num_bad << " bad objects" << dendl;
- ss << info.pgid << " " << mode << " " << num_missing << " missing, " << num_bad << " bad objects";
- osd->get_logclient()->log(LOG_ERROR, ss);
+ stringstream ss;
+ ss << info.pgid << " " << mode << " " << num_missing << " missing, "
+ << num_bad << " bad objects\n";
+ *_dout << ss.str();
+ _dout->flush();
+ osd->clog.error(ss);
state_set(PG_STATE_INCONSISTENT);
if (repair)
state_clear(PG_STATE_CLEAN);
@@ -3299,14 +3295,21 @@ void PG::scrub()
}
*/
- ss << info.pgid << " " << mode << " ";
- if (errors)
- ss << errors << " errors";
- else
- ss << "ok";
- if (repair)
- ss << ", " << fixed << " fixed";
- osd->get_logclient()->log(errors ? LOG_ERROR:LOG_INFO, ss);
+ {
+ stringstream oss;
+ oss << info.pgid << " " << mode << " ";
+ if (errors)
+ oss << errors << " errors";
+ else
+ oss << "ok";
+ if (repair)
+ oss << ", " << fixed << " fixed";
+ oss << "\n";
+ if (errors)
+ osd->clog.error(oss);
+ else
+ osd->clog.info(oss);
+ }
if (!(errors - fixed) && repair)
state_clear(PG_STATE_INCONSISTENT);
diff --git a/src/osd/RAID4PG.cc b/src/osd/RAID4PG.cc
index 9af7bc71215..5613b27a971 100644
--- a/src/osd/RAID4PG.cc
+++ b/src/osd/RAID4PG.cc
@@ -28,8 +28,7 @@
#undef dout_prefix
#define dout_prefix _prefix(this, osd->whoami, osd->osdmap)
static ostream& _prefix(PG *pg, int whoami, OSDMap *osdmap) {
- return *_dout << dbeginl
- << "osd" << whoami
+ return *_dout << "osd" << whoami
<< " " << (osdmap ? osdmap->get_epoch():0) << " "
<< *pg << " ";
}
diff --git a/src/osd/ReplicatedPG.cc b/src/osd/ReplicatedPG.cc
index 7e6a1e1b71c..74c25ba8118 100644
--- a/src/osd/ReplicatedPG.cc
+++ b/src/osd/ReplicatedPG.cc
@@ -16,6 +16,7 @@
#include "PGLS.h"
#include "common/arch.h"
+#include "common/errno.h"
#include "common/Logger.h"
#include "messages/MOSDOp.h"
@@ -37,7 +38,7 @@
#undef dout_prefix
#define dout_prefix _prefix(this, osd->whoami, osd->osdmap)
static ostream& _prefix(PG *pg, int whoami, OSDMap *osdmap) {
- return *_dout << dbeginl << "osd" << whoami
+ return *_dout << "osd" << whoami
<< " " << (osdmap ? osdmap->get_epoch():0) << " " << *pg << " ";
}
@@ -2610,7 +2611,7 @@ void ReplicatedPG::sub_op_modify(MOSDSubOp *op)
Context *onapply = new C_OSD_RepModifyApply(rm);
int r = osd->store->queue_transactions(&osr, rm->tls, onapply, oncommit);
if (r) {
- derr(0) << "error applying transaction: r = " << r << dendl;
+ dout(0) << "error applying transaction: r = " << r << dendl;
assert(0);
}
// op is cleaned up by oncommit/onapply when both are executed
@@ -3168,14 +3169,10 @@ void ReplicatedPG::sub_op_pull(MOSDSubOp *op)
struct stat st;
int r = osd->store->stat(coll_t(info.pgid), soid, &st);
if (r != 0) {
- stringstream ss;
- char buf[80];
- ss << op->get_source() << " tried to pull " << soid << " in " << info.pgid
- << " but got " << strerror_r(-r, buf, sizeof(buf));
- osd->logclient.log(LOG_ERROR, ss);
-
+ osd->clog.error() << op->get_source() << " tried to pull " << soid
+ << " in " << info.pgid << " but got "
+ << cpp_strerror(-r) << "\n";
// FIXME: do something more intelligent.. mark the pg as needing repair?
-
} else {
uint64_t size = st.st_size;
@@ -4012,7 +4009,7 @@ int ReplicatedPG::_scrub(ScrubMap& scrubmap, int& errors, int& fixed)
// did we finish the last oid?
if (head != sobject_t()) {
- derr(0) << " missing clone(s) for " << head << dendl;
+ dout(0) << " missing clone(s) for " << head << dendl;
assert(head == sobject_t()); // we had better be done
errors++;
}
@@ -4093,17 +4090,16 @@ int ReplicatedPG::_scrub(ScrubMap& scrubmap, int& errors, int& fixed)
<< stat.num_kb << "/" << info.stats.num_kb << " kb."
<< dendl;
- stringstream ss;
if (stat.num_objects != info.stats.num_objects ||
stat.num_object_clones != info.stats.num_object_clones ||
stat.num_bytes != info.stats.num_bytes ||
stat.num_kb != info.stats.num_kb) {
- ss << info.pgid << " " << mode << " stat mismatch, got "
+ osd->clog.error() << info.pgid << " " << mode
+ << " stat mismatch, got "
<< stat.num_objects << "/" << info.stats.num_objects << " objects, "
<< stat.num_object_clones << "/" << info.stats.num_object_clones << " clones, "
<< stat.num_bytes << "/" << info.stats.num_bytes << " bytes, "
- << stat.num_kb << "/" << info.stats.num_kb << " kb.";
- osd->get_logclient()->log(LOG_ERROR, ss);
+ << stat.num_kb << "/" << info.stats.num_kb << " kb.\n";
errors++;
if (repair) {
diff --git a/src/osdc/Filer.cc b/src/osdc/Filer.cc
index a9d55c2347e..004e0c69994 100644
--- a/src/osdc/Filer.cc
+++ b/src/osdc/Filer.cc
@@ -28,7 +28,7 @@
#define DOUT_SUBSYS filer
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << objecter->messenger->get_myname() << ".filer "
+#define dout_prefix *_dout << objecter->messenger->get_myname() << ".filer "
class Filer::C_Probe : public Context {
diff --git a/src/osdc/Journaler.cc b/src/osdc/Journaler.cc
index af12e382fb4..89b260c66f1 100644
--- a/src/osdc/Journaler.cc
+++ b/src/osdc/Journaler.cc
@@ -22,7 +22,7 @@
#define DOUT_SUBSYS journaler
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << objecter->messenger->get_myname() << ".journaler "
+#define dout_prefix *_dout << objecter->messenger->get_myname() << ".journaler "
@@ -741,7 +741,7 @@ bool Journaler::try_read_entry(bufferlist& bl)
<< read_pos << "~" << (sizeof(s)+s) << " (have " << read_buf.length() << ")" << dendl;
if (s == 0) {
- derr(0) << "try_read_entry got 0 len entry at offset " << read_pos << dendl;
+ dout(0) << "try_read_entry got 0 len entry at offset " << read_pos << dendl;
error = -EINVAL;
return false;
}
diff --git a/src/osdc/ObjectCacher.cc b/src/osdc/ObjectCacher.cc
index 3e34ab692f9..2f0360d78af 100644
--- a/src/osdc/ObjectCacher.cc
+++ b/src/osdc/ObjectCacher.cc
@@ -14,7 +14,7 @@
#define DOUT_SUBSYS objectcacher
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << oc->objecter->messenger->get_myname() << ".objectcacher.object(" << oid << ") "
+#define dout_prefix *_dout << oc->objecter->messenger->get_myname() << ".objectcacher.object(" << oid << ") "
@@ -416,7 +416,7 @@ void ObjectCacher::Object::truncate(loff_t s)
/*** ObjectCacher ***/
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << objecter->messenger->get_myname() << ".objectcacher "
+#define dout_prefix *_dout << objecter->messenger->get_myname() << ".objectcacher "
/* private */
diff --git a/src/osdc/Objecter.cc b/src/osdc/Objecter.cc
index 5ee5469de9d..8c2457ee77f 100644
--- a/src/osdc/Objecter.cc
+++ b/src/osdc/Objecter.cc
@@ -42,7 +42,7 @@
#define DOUT_SUBSYS objecter
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl << messenger->get_myname() << ".objecter "
+#define dout_prefix *_dout << messenger->get_myname() << ".objecter "
// messages ------------------------------
diff --git a/src/streamtest.cc b/src/streamtest.cc
index 6af529ca3f5..fa9a792546a 100644
--- a/src/streamtest.cc
+++ b/src/streamtest.cc
@@ -18,7 +18,7 @@
#include "common/common_init.h"
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl
+#define dout_prefix *_dout
struct io {
utime_t start, ack, commit;
diff --git a/src/test/TestDoutStreambuf.cc b/src/test/TestDoutStreambuf.cc
new file mode 100644
index 00000000000..18eec24051e
--- /dev/null
+++ b/src/test/TestDoutStreambuf.cc
@@ -0,0 +1,88 @@
+// -*- 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) 2010 Dreamhost
+ *
+ * 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.
+ *
+ */
+
+/*
+ * TestDoutStreambuf
+ *
+ * Puts some output into the DoutStreambuf class.
+ * Check your syslog to see what it did.
+ */
+#include "common/DoutStreambuf.h"
+#include "common/common_init.h"
+#include "config.h"
+
+#include <iostream>
+#include <sstream>
+#include <string>
+#include <syslog.h>
+
+using std::cout;
+using std::cerr;
+using std::string;
+
+int main(int argc, const char **argv)
+{
+ vector<const char*> args;
+ argv_to_vec(argc, argv, args);
+ env_to_vec(args);
+
+ ceph_set_default_id("admin");
+ common_set_defaults(false);
+ common_init(args, "ceph", true);
+
+ DoutStreambuf<char> *dos = new DoutStreambuf<char>();
+
+ _dout_lock.Lock();
+ dos->read_global_config();
+// dos->set_flags(DoutStreambuf<char>::DOUTSB_FLAG_SYSLOG |
+// DoutStreambuf<char>::DOUTSB_FLAG_STDOUT |
+// DoutStreambuf<char>::DOUTSB_FLAG_STDERR);
+ std::cout << "using configuration: " << dos->config_to_str() << std::endl;
+ _dout_lock.Unlock();
+
+ std::ostream oss(dos);
+ syslog(LOG_USER | LOG_NOTICE, "TestDoutStreambuf: starting test\n");
+
+ dos->set_prio(1);
+ oss << "1. I am logging to dout now!" << std::endl;
+
+ dos->set_prio(2);
+ oss << "2. And here is another line!" << std::endl;
+
+ oss.flush();
+
+ dos->set_prio(3);
+ oss << "3. And here is another line!" << std::endl;
+
+ dos->set_prio(16);
+ oss << "4. Stuff ";
+ oss << "that ";
+ oss << "will ";
+ oss << "all ";
+ oss << "be ";
+ oss << "on ";
+ oss << "one ";
+ oss << "line.\n";
+ oss.flush();
+
+ dos->set_prio(10);
+ oss << "5. There will be no blank lines here.\n" << std::endl;
+ oss.flush();
+ oss.flush();
+ oss.flush();
+
+ syslog(LOG_USER | LOG_NOTICE, "TestDoutStreambuf: ending test\n");
+
+ return 0;
+}
diff --git a/src/test_trans.cc b/src/test_trans.cc
index 076321622dd..3fb7a288cd6 100644
--- a/src/test_trans.cc
+++ b/src/test_trans.cc
@@ -18,7 +18,7 @@
#include "common/common_init.h"
#undef dout_prefix
-#define dout_prefix *_dout << dbeginl
+#define dout_prefix *_dout
struct Foo : public Thread {
void *entry() {
diff --git a/src/testcrypto.cc b/src/testcrypto.cc
index 020d3cd0343..1ec784aaa85 100644
--- a/src/testcrypto.cc
+++ b/src/testcrypto.cc
@@ -22,7 +22,7 @@ int main(int argc, char *argv[])
bufferlist enc_out;
if (key.encrypt(enc_in, enc_out) < 0) {
- derr(0) << "couldn't encode!" << dendl;
+ dout(0) << "couldn't encode!" << dendl;
exit(1);
}
@@ -38,7 +38,7 @@ int main(int argc, char *argv[])
dec_in = enc_out;
if (key.decrypt(dec_in, dec_out) < 0) {
- derr(0) << "couldn't decode!" << dendl;
+ dout(0) << "couldn't decode!" << dendl;
}
dout(0) << "decoded len: " << dec_out.length() << dendl;
diff --git a/src/tools/ceph.cc b/src/tools/ceph.cc
index 7a30e804f62..03e03caaa59 100644
--- a/src/tools/ceph.cc
+++ b/src/tools/ceph.cc
@@ -68,7 +68,7 @@ Context *resend_event = 0;
#include "mon/PGMap.h"
#include "osd/OSDMap.h"
#include "mds/MDSMap.h"
-#include "include/LogEntry.h"
+#include "common/LogEntry.h"
#include "include/ClassLibrary.h"
#include "mon/mon_types.h"
diff --git a/src/tools/cmd.cc b/src/tools/cmd.cc
index ef40e4456f0..a380e6e0964 100644
--- a/src/tools/cmd.cc
+++ b/src/tools/cmd.cc
@@ -18,6 +18,7 @@
#include "common/errno.h"
#include "config.h"
+#include <limits.h>
#include <errno.h>
#include <fcntl.h>
#include <iostream>