diff options
Diffstat (limited to 'db/dur.cpp')
-rw-r--r-- | db/dur.cpp | 292 |
1 files changed, 229 insertions, 63 deletions
@@ -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(); |