summaryrefslogtreecommitdiff
path: root/db/dur.cpp
diff options
context:
space:
mode:
Diffstat (limited to 'db/dur.cpp')
-rw-r--r--db/dur.cpp292
1 files changed, 229 insertions, 63 deletions
diff --git a/db/dur.cpp b/db/dur.cpp
index 15b4565..4861773 100644
--- a/db/dur.cpp
+++ b/db/dur.cpp
@@ -17,7 +17,7 @@
*/
/*
- phases
+ phases:
PREPLOGBUFFER
we will build an output buffer ourself and then use O_DIRECT
@@ -36,6 +36,22 @@
there could be a slow down immediately after remapping as fresh copy-on-writes for commonly written pages will
be required. so doing these remaps fractionally is helpful.
+ mutexes:
+
+ READLOCK dbMutex
+ LOCK groupCommitMutex
+ PREPLOGBUFFER()
+ READLOCK mmmutex
+ commitJob.reset()
+ UNLOCK dbMutex // now other threads can write
+ WRITETOJOURNAL()
+ WRITETODATAFILES()
+ UNLOCK mmmutex
+ UNLOCK groupCommitMutex
+
+ on the next write lock acquisition for dbMutex: // see MongoMutex::_acquiredWriteLock()
+ REMAPPRIVATEVIEW()
+
@see https://docs.google.com/drawings/edit?id=1TklsmZzm7ohIZkwgeK6rMvsdaR13KjtJYMsfLr175Zc
*/
@@ -46,11 +62,11 @@
#include "dur_journal.h"
#include "dur_commitjob.h"
#include "dur_recover.h"
+#include "dur_stats.h"
#include "../util/concurrency/race.h"
#include "../util/mongoutils/hash.h"
#include "../util/mongoutils/str.h"
#include "../util/timer.h"
-#include "dur_stats.h"
using namespace mongoutils;
@@ -58,8 +74,9 @@ namespace mongo {
namespace dur {
- void WRITETODATAFILES();
- void PREPLOGBUFFER();
+ void PREPLOGBUFFER(JSectHeader& outParm);
+ void WRITETOJOURNAL(JSectHeader h, AlignedBuilder& uncompressed);
+ void WRITETODATAFILES(const JSectHeader& h, AlignedBuilder& uncompressed);
/** declared later in this file
only used in this file -- use DurableInterface::commitNow() outside
@@ -84,12 +101,36 @@ namespace mongo {
Stats::S * Stats::other() {
return curr == &_a ? &_b : &_a;
}
+ string _CSVHeader();
+
+ string Stats::S::_CSVHeader() {
+ return "cmts jrnMB\twrDFMB\tcIWLk\tearly\tprpLgB wrToJ\twrToDF\trmpPrVw";
+ }
+
+ string Stats::S::_asCSV() {
+ stringstream ss;
+ ss <<
+ setprecision(2) <<
+ _commits << '\t' << fixed <<
+ _journaledBytes / 1000000.0 << '\t' <<
+ _writeToDataFilesBytes / 1000000.0 << '\t' <<
+ _commitsInWriteLock << '\t' <<
+ _earlyCommits << '\t' <<
+ (unsigned) (_prepLogBufferMicros/1000) << '\t' <<
+ (unsigned) (_writeToJournalMicros/1000) << '\t' <<
+ (unsigned) (_writeToDataFilesMicros/1000) << '\t' <<
+ (unsigned) (_remapPrivateViewMicros/1000);
+ return ss.str();
+ }
+ //int getAgeOutJournalFiles();
BSONObj Stats::S::_asObj() {
- return BSON(
+ BSONObjBuilder b;
+ b <<
"commits" << _commits <<
"journaledMB" << _journaledBytes / 1000000.0 <<
"writeToDataFilesMB" << _writeToDataFilesBytes / 1000000.0 <<
+ "compression" << _journaledBytes / (_uncompressedBytes+1.0) <<
"commitsInWriteLock" << _commitsInWriteLock <<
"earlyCommits" << _earlyCommits <<
"timeMs" <<
@@ -98,8 +139,15 @@ namespace mongo {
"writeToJournal" << (unsigned) (_writeToJournalMicros/1000) <<
"writeToDataFiles" << (unsigned) (_writeToDataFilesMicros/1000) <<
"remapPrivateView" << (unsigned) (_remapPrivateViewMicros/1000)
- )
- );
+ );
+ /*int r = getAgeOutJournalFiles();
+ if( r == -1 )
+ b << "ageOutJournalFiles" << "mutex timeout";
+ if( r == 0 )
+ b << "ageOutJournalFiles" << false;*/
+ if( cmdLine.journalCommitInterval != 0 )
+ b << "journalCommitIntervalMs" << cmdLine.journalCommitInterval;
+ return b.obj();
}
BSONObj Stats::asObj() {
@@ -123,14 +171,22 @@ namespace mongo {
}
void DurableImpl::setNoJournal(void *dst, void *src, unsigned len) {
+ // we are at least read locked, so we need not worry about REMAPPRIVATEVIEW herein.
+ DEV dbMutex.assertAtLeastReadLocked();
+
MemoryMappedFile::makeWritable(dst, len);
+ // we enter the RecoveryJob mutex here, so that if WRITETODATAFILES is happening we do not
+ // conflict with it
+ scoped_lock lk1( RecoveryJob::get()._mx );
+
// we stay in this mutex for everything to work with DurParanoid/validateSingleMapMatches
//
- // this also makes setNoJournal threadsafe, which is good as we call it from a read (not a write) lock
- // in class SlaveTracking
+ // either of these mutexes also makes setNoJournal threadsafe, which is good as we call it from a read
+ // (not a write) lock in class SlaveTracking
//
scoped_lock lk( privateViews._mutex() );
+
size_t ofs;
MongoMMF *f = privateViews.find_inlock(dst, ofs);
assert(f);
@@ -171,7 +227,7 @@ namespace mongo {
}
bool DurableImpl::awaitCommit() {
- commitJob.awaitNextCommit();
+ commitJob._notify.awaitBeyondNow();
return true;
}
@@ -211,7 +267,15 @@ namespace mongo {
return p;
}
+ bool DurableImpl::aCommitIsNeeded() const {
+ DEV commitJob._nSinceCommitIfNeededCall = 0;
+ return commitJob.bytes() > UncommittedBytesLimit;
+ }
+
bool DurableImpl::commitIfNeeded() {
+ if ( ! dbMutex.isWriteLocked() ) // we implicitly commit if needed when releasing write lock
+ return false;
+
DEV commitJob._nSinceCommitIfNeededCall = 0;
if (commitJob.bytes() > UncommittedBytesLimit) { // should this also fire if CmdLine::DurAlwaysCommit?
stats.curr->_earlyCommits++;
@@ -259,7 +323,7 @@ namespace mongo {
return;
}
}
- log() << "dur data after write area " << i.start() << " does not agree" << endl;
+ log() << "journal data after write area " << i.start() << " does not agree" << endl;
log() << " was: " << ((void*)b) << " " << hexdump((char*)b, 8) << endl;
log() << " now: " << ((void*)a) << " " << hexdump((char*)a, 8) << endl;
log() << " n: " << n << endl;
@@ -268,15 +332,6 @@ namespace mongo {
}
#endif
- /** write the buffer we have built to the journal and fsync it.
- outside of lock as that could be slow.
- */
- static void WRITETOJOURNAL(AlignedBuilder& ab) {
- Timer t;
- journal(ab);
- stats.curr->_writeToJournalMicros += t.micros();
- }
-
// Functor to be called over all MongoFiles
class validateSingleMapMatches {
@@ -285,8 +340,8 @@ namespace mongo {
void operator () (MongoFile *mf) {
if( mf->isMongoMMF() ) {
MongoMMF *mmf = (MongoMMF*) mf;
- const char *p = (const char *) mmf->getView();
- const char *w = (const char *) mmf->view_write();
+ const unsigned char *p = (const unsigned char *) mmf->getView();
+ const unsigned char *w = (const unsigned char *) mmf->view_write();
if (!p || !w) return; // File not fully opened yet
@@ -310,6 +365,8 @@ namespace mongo {
log() << endl; // separate blocks of mismatches
lastMismatch= i;
if( ++logged < 60 ) {
+ if( logged == 1 )
+ log() << "ofs % 628 = 0x" << hex << (i%628) << endl; // for .ns files to find offset in record
stringstream ss;
ss << "mismatch ofs:" << hex << i << "\tfilemap:" << setw(2) << (unsigned) w[i] << "\tprivmap:" << setw(2) << (unsigned) p[i];
if( p[i] > 32 && p[i] <= 126 )
@@ -324,7 +381,7 @@ namespace mongo {
}
if( low != 0xffffffff ) {
std::stringstream ss;
- ss << "dur error warning views mismatch " << mmf->filename() << ' ' << (hex) << low << ".." << high << " len:" << high-low+1;
+ ss << "journal error warning views mismatch " << mmf->filename() << ' ' << (hex) << low << ".." << high << " len:" << high-low+1;
log() << ss.str() << endl;
log() << "priv loc: " << (void*)(p+low) << ' ' << endl;
set<WriteIntent>& b = commitJob.writes();
@@ -357,6 +414,9 @@ namespace mongo {
Call within write lock.
*/
void _REMAPPRIVATEVIEW() {
+ // todo: Consider using ProcessInfo herein and watching for getResidentSize to drop. that could be a way
+ // to assure very good behavior here.
+
static unsigned startAt;
static unsigned long long lastRemap;
@@ -370,9 +430,11 @@ namespace mongo {
// remapping.
unsigned long long now = curTimeMicros64();
double fraction = (now-lastRemap)/2000000.0;
+ if( cmdLine.durOptions & CmdLine::DurAlwaysRemap )
+ fraction = 1;
lastRemap = now;
- rwlock lk(MongoFile::mmmutex, false);
+ RWLockRecursive::Shared lk(MongoFile::mmmutex);
set<MongoFile*>& files = MongoFile::getAllFiles();
unsigned sz = files.size();
if( sz == 0 )
@@ -422,11 +484,79 @@ namespace mongo {
stats.curr->_remapPrivateViewMicros += t.micros();
}
+ // lock order: dbMutex first, then this
mutex groupCommitMutex("groupCommit");
- /** locking: in read lock when called. */
+ bool _groupCommitWithLimitedLocks() {
+ scoped_ptr<readlocktry> lk1( new readlocktry("", 500) );
+ if( !lk1->got() )
+ return false;
+
+ scoped_lock lk2(groupCommitMutex);
+
+ commitJob.beginCommit();
+
+ if( !commitJob.hasWritten() ) {
+ // getlasterror request could have came after the data was already committed
+ commitJob.notifyCommitted();
+ return true;
+ }
+ JSectHeader h;
+ PREPLOGBUFFER(h);
+
+ RWLockRecursive::Shared lk3(MongoFile::mmmutex);
+
+ unsigned abLen = commitJob._ab.len();
+ commitJob.reset(); // must be reset before allowing anyone to write
+ DEV assert( !commitJob.hasWritten() );
+
+ // release the readlock -- allowing others to now write while we are writing to the journal (etc.)
+ lk1.reset();
+
+ // ****** now other threads can do writes ******
+ WRITETOJOURNAL(h, commitJob._ab);
+ assert( abLen == commitJob._ab.len() ); // a check that no one touched the builder while we were doing work. if so, our locking is wrong.
+
+ // data is now in the journal, which is sufficient for acknowledging getLastError.
+ // (ok to crash after that)
+ commitJob.notifyCommitted();
+
+ WRITETODATAFILES(h, commitJob._ab);
+ assert( abLen == commitJob._ab.len() ); // check again wasn't modded
+ commitJob._ab.reset();
+
+ // can't : dbMutex._remapPrivateViewRequested = true;
+
+ return true;
+ }
+
+ /** @return true if committed; false if lock acquisition timed out (we only try for a read lock herein and only wait for a certain duration). */
+ bool groupCommitWithLimitedLocks() {
+ try {
+ return _groupCommitWithLimitedLocks();
+ }
+ catch(DBException& e ) {
+ log() << "dbexception in groupCommitLL causing immediate shutdown: " << e.toString() << endl;
+ mongoAbort("dur1");
+ }
+ catch(std::ios_base::failure& e) {
+ log() << "ios_base exception in groupCommitLL causing immediate shutdown: " << e.what() << endl;
+ mongoAbort("dur2");
+ }
+ catch(std::bad_alloc& e) {
+ log() << "bad_alloc exception in groupCommitLL causing immediate shutdown: " << e.what() << endl;
+ mongoAbort("dur3");
+ }
+ catch(std::exception& e) {
+ log() << "exception in dur::groupCommitLL causing immediate shutdown: " << e.what() << endl;
+ mongoAbort("dur4");
+ }
+ return false;
+ }
+
+ /** locking: in read lock when called. */
static void _groupCommit() {
- stats.curr->_commits++;
+ commitJob.beginCommit();
if( !commitJob.hasWritten() ) {
// getlasterror request could have came after the data was already committed
@@ -438,20 +568,23 @@ namespace mongo {
// (and we are only read locked in the dbMutex, so it could happen)
scoped_lock lk(groupCommitMutex);
- PREPLOGBUFFER();
+ JSectHeader h;
+ PREPLOGBUFFER(h);
// todo : write to the journal outside locks, as this write can be slow.
// however, be careful then about remapprivateview as that cannot be done
// if new writes are then pending in the private maps.
- WRITETOJOURNAL(commitJob._ab);
+ WRITETOJOURNAL(h, commitJob._ab);
// data is now in the journal, which is sufficient for acknowledging getLastError.
// (ok to crash after that)
commitJob.notifyCommitted();
- WRITETODATAFILES();
+ WRITETODATAFILES(h, commitJob._ab);
+ debugValidateAllMapsMatch();
commitJob.reset();
+ commitJob._ab.reset();
// REMAPPRIVATEVIEW
//
@@ -463,7 +596,7 @@ namespace mongo {
// this needs done in a write lock (as there is a short window during remapping when each view
// might not exist) thus we do it on the next acquisition of that instead of here (there is no
// rush if you aren't writing anyway -- but it must happen, if it is done, before any uncommitted
- // writes occur). If desired, perhpas this can be eliminated on posix as it may be that the remap
+ // writes occur). If desired, perhaps this can be eliminated on posix as it may be that the remap
// is race-free there.
//
dbMutex._remapPrivateViewRequested = true;
@@ -478,7 +611,8 @@ namespace mongo {
}
}
- /** locking in read lock when called
+ /** locking: in read lock when called
+ or, for early commits (commitIfNeeded), in write lock
@see MongoMMF::close()
*/
static void groupCommit() {
@@ -491,29 +625,33 @@ namespace mongo {
}
catch(DBException& e ) {
log() << "dbexception in groupCommit causing immediate shutdown: " << e.toString() << endl;
- abort();
+ mongoAbort("gc1");
}
catch(std::ios_base::failure& e) {
log() << "ios_base exception in groupCommit causing immediate shutdown: " << e.what() << endl;
- abort();
+ mongoAbort("gc2");
}
catch(std::bad_alloc& e) {
log() << "bad_alloc exception in groupCommit causing immediate shutdown: " << e.what() << endl;
- abort();
+ mongoAbort("gc3");
}
catch(std::exception& e) {
log() << "exception in dur::groupCommit causing immediate shutdown: " << e.what() << endl;
- abort(); // based on myTerminate()
+ mongoAbort("gc4");
}
}
static void go() {
- if( !commitJob.hasWritten() ){
- commitJob.notifyCommitted();
- return;
+ const int N = 10;
+ static int n;
+ if( privateMapBytes < UncommittedBytesLimit && ++n % N && (cmdLine.durOptions&CmdLine::DurAlwaysRemap)==0 ) {
+ // limited locks version doesn't do any remapprivateview at all, so only try this if privateMapBytes
+ // is in an acceptable range. also every Nth commit, we do everything so we can do some remapping;
+ // remapping a lot all at once could cause jitter from a large amount of copy-on-writes all at once.
+ if( groupCommitWithLimitedLocks() )
+ return;
}
-
- {
+ else {
readlocktry lk("", 1000);
if( lk.got() ) {
groupCommit();
@@ -542,45 +680,53 @@ namespace mongo {
else {
assert( inShutdown() );
if( commitJob.hasWritten() ) {
- log() << "dur warning files are closing outside locks with writes pending" << endl;
+ log() << "journal warning files are closing outside locks with writes pending" << endl;
}
}
}
- CodeBlock durThreadMain;
+ filesystem::path getJournalDir();
void durThread() {
- Client::initThread("dur");
- const int HowOftenToGroupCommitMs = 90;
+ Client::initThread("journal");
+
+ bool samePartition = true;
+ try {
+ const string dbpathDir = boost::filesystem::path(dbpath).native_directory_string();
+ samePartition = onSamePartition(getJournalDir().string(), dbpathDir);
+ }
+ catch(...) {
+ }
+
while( !inShutdown() ) {
- sleepmillis(10);
- CodeBlock::Within w(durThreadMain);
+ unsigned ms = cmdLine.journalCommitInterval;
+ if( ms == 0 ) {
+ // use default
+ ms = samePartition ? 100 : 30;
+ }
+
+ unsigned oneThird = (ms / 3) + 1; // +1 so never zero
+
try {
- int millis = HowOftenToGroupCommitMs;
- {
- stats.rotate();
- {
- Timer t;
- journalRotate(); // note we do this part outside of mongomutex
- millis -= t.millis();
- assert( millis <= HowOftenToGroupCommitMs );
- if( millis < 5 )
- millis = 5;
- }
+ stats.rotate();
- // we do this in a couple blocks, which makes it a tiny bit faster (only a little) on throughput,
- // but is likely also less spiky on our cpu usage, which is good:
- sleepmillis(millis/2);
- commitJob.wi()._deferred.invoke();
- sleepmillis(millis/2);
+ // we do this in a couple blocks (the invoke()), which makes it a tiny bit faster (only a little) on throughput,
+ // but is likely also less spiky on our cpu usage, which is good.
+
+ // commit sooner if one or more getLastError j:true is pending
+ sleepmillis(oneThird);
+ for( unsigned i = 1; i <= 2; i++ ) {
+ if( commitJob._notify.nWaiting() )
+ break;
commitJob.wi()._deferred.invoke();
+ sleepmillis(oneThird);
}
go();
}
catch(std::exception& e) {
log() << "exception in durThread causing immediate shutdown: " << e.what() << endl;
- abort(); // based on myTerminate()
+ mongoAbort("exception in durThread");
}
}
cc().shutdown();
@@ -604,6 +750,19 @@ namespace mongo {
if( !cmdLine.dur )
return;
+#if defined(_DURABLEDEFAULTON)
+ DEV {
+ if( time(0) & 1 ) {
+ cmdLine.durOptions |= CmdLine::DurAlwaysCommit;
+ log() << "_DEBUG _DURABLEDEFAULTON : forcing DurAlwaysCommit mode for this run" << endl;
+ }
+ if( time(0) & 2 ) {
+ cmdLine.durOptions |= CmdLine::DurAlwaysRemap;
+ log() << "_DEBUG _DURABLEDEFAULTON : forcing DurAlwaysRemap mode for this run" << endl;
+ }
+ }
+#endif
+
DurableInterface::enableDurability();
journalMakeDir();
@@ -623,6 +782,13 @@ namespace mongo {
void DurableImpl::syncDataAndTruncateJournal() {
dbMutex.assertWriteLocked();
+ // a commit from the commit thread won't begin while we are in the write lock,
+ // but it may already be in progress and the end of that work is done outside
+ // (dbMutex) locks. This line waits for that to complete if already underway.
+ {
+ scoped_lock lk(groupCommitMutex);
+ }
+
groupCommit();
MongoFile::flushAll(true);
journalCleanup();