mds/AnchorClient.o\
mds/LogEvent.o\
mds/IdAllocator.o\
+ mds/ClientMap.o\
mds/MDLog.o
OSD_OBJS= \
+- change same_inst_since to align with "in" set
+- tag MClientRequest with mdsmap v
+- push new mdsmap to clients on send_message_client, based on the tag?
+ - hrm, what about exports and stale caps wonkiness... there's a race with the REAP. hmm.
+
+
some smallish projects:
- crush rewrite in C
- xml import/export?
- ?
+- pg monitor service
+ - to support statfs?
+ - general pg health
+ - some sort of (throttled) osd status reporting
+ - dynamic pg creation (eventually!)
+
- SimpleMessenger
- clean up/merge Messenger/Dispatcher interfaces
- auto close idle connections
- generalize monitor client?
- throttle message resend attempts
-- paxos layer work
- - integrate leasing into paxos framework
- - carefully interface design...
+- ENOSPC on client, OSD
+
code cleanup
- endian portability
- word size
- - clean up all encoded structures
+ - clean up all encoded structures
general kernel planning
- soft consistency on (kernel) lookup?
+- accurate reconstruction of (syscall) path?
-
-
+software raid layer for EBOFS?
+- actually, we just need software raid _awareness_ in the allocator, so
+ that we can write only full stripes, without fear of clobbering things on
+ failure. then use MD or similar layer provided by kernel.
sage doc
sage mds
-- journal+recovery
- - local rename
- - how to notify replicas...
-/ - stray purge
- - stray reintegration
- - remote link
- - impl remote inode xlock
- - ESlaveUpdate replay, resolution, etc.
- - remote unlink
- - remote rename
- - file capabilities i/o
+
+- hmm, should we move ESubtreeMap out of the journal?
+ that would avoid all the icky weirdness in shutdown, with periodic logging, etc.
+
+- extend/clean up filepath to allow paths relative to an ino
+ - fix path_traverse
+ - fix reconnect/rejoin open file weirdness
+
+- stray reintegration
+- stray purge on shutdown
+ - need to export stray crap to another mds..
+- verify stray is empty on shutdown
+
- dirfrag split/merge
- client readdir for dirfrags
- consistency points/snapshots
- dentry versions vs dirfrags...
- statfs?
-- finish multistage rejoin
-- trim_on_rejoin
-
- more testing of failures + thrashing.
- is export prep dir open deadlock properly fixed by forge_replica_dir()?
- failures during recovery stages (resolve, rejoin)... make sure rejoin still works!
we break commit()'s preconditions when it fetches an incomplete dir.
- detect and deal with client failure
-
-- recovering open files
- - recovery will either have inode (from EOpen), or will provide path+cap to reassert open state.
- - path+cap window will require some fetching of metadata from disk before doing the rejoin
- - failures during migration.. what about client stale/reap stuff and misplaced WR caps?
+ - failure during reconnect vs clientmap. although probalby the whole thing needs a larger overhaul...
- inode.max_size
+- inode.allocated_size
- real chdir (directory "open")
- relative metadata ops
-
-
- osd needs a set_floor_and_read op for safe failover/STOGITH-like semantics.
-- incremental mdsmaps?
-
- EMetablob should return 'expired' if they have higher versions (and are thus described by a newer journal entry)
-- dir version/committed/etc versus migration, log expires.
- - DOCUMENT.
- fix rmdir empty exported dirfrag race
- export all frags <= 1 item? then we ensure freezing before empty, avoiding any last unlink + export vs rmdir race.
- need to move state from replicas to auth. simplelock doesn't currently support that.
- ScatterLock or something? hrm.
-- test open_remote_ino
-
- FIXME how to journal root and stray inode content?
- in particular, i care about dirfragtree.. get it on rejoin?
- and dir sizes, if i add that... also on rejoin?
-
-
-
-
-foreign rename
-- question: can we generalize foreign and local rename?
-- initiated by dest.
- - if we get into race with lock acquisition, drop locks and forward to new dest.
-- how to do pre-auth pinning?
- - is it sufficient to wait on, then grab, all local auth pins, _then_ do foreign locks?
- - local auth pins can hold subtrees in freezing state, preventing exports, and additional auth_pins.
- - so, wait, then grab all local auth_pins,
- - then work on locks in proper order (*),
- - if we detect we are missing a local auth_pin (i.e. migration race), drop all auth_pins and wait/restart
- - need to more carefully look at lock dependencies to avoid deadlock...
- - establish a complete full ordering on locks, based on any lock dependencies?
- - is it possible to "leak" locks, e.g. get inode_hard lock, work on something else, but inode moves and we dont notice?
- - pin paths for those locks?
- - can we pin when we choose order, so that locks are sure to proceed?
-- we can change active_requests to key of reqid (also unique), and use the same key for foreign locks
- - clean up dentry_xlock_request.. just merge it into destroy_xlock_start, if !is_auth().
-- renamer will
- - check preconditions (i.e. i am dest)
- - grab all locks (avoiding deadlock)
- - verify preconditions are still true, else forward/retry (actually, this already happens w/ the way we structure the lock acquisition code...)
- - prepare foreign bits (using foreign request_auth_pins, locks, etc.)
- - source unlink,
- - anchortable update (if source is anchored),
- - dest nlink-- (if dest is remote link on foreign host)
- - make sure replicas have either both source+dest pinned in cache (or neither...)
- - use foreign request_pins?
- - log update
- - do update locally
- - async commit + unlock
-- rejoin will need to explicitly resolve uncommitted items.
- - fully implement link/unlink first, and use that as a model?
-
osdmon
- allow fresh replacement osds. add osd_created in osdmap, probably
- monitor needs to monitor some osds...
- pg_num changes
- report crashed pgs?
+messenger
+- fix messenger shutdown.. we shouldn't delete messenger, since the caller may be referencing it, etc.
+
simplemessenger
- close idle connections
- buffer sent messages until a receive is acknowledged (handshake!)
if (in->dn) {
dout(12) << " had ino " << in->inode.ino
- << " linked at wrong position, unlinking"
+ << " not linked or linked at the right position, relinking"
<< endl;
- dn = relink(in->dn, dir, dname);
+ dn = relink(dir, dname, in);
} else {
// link
dout(12) << " had ino " << in->inode.ino
// pick mds
if (!diri || g_conf.client_use_random_mds) {
// no root info, pick a random MDS
- mds = rand() % mdsmap->get_num_mds();
+ mds = mdsmap->get_random_in_mds();
+ if (mds < 0) mds = 0;
+
+ if (0) {
+ mds = 0;
+ dout(0) << "hack: sending all requests to mds" << mds << endl;
+ }
} else {
if (req->auth_is_best()) {
// pick the actual auth (as best we can)
// assign a unique tid
tid_t tid = ++last_tid;
req->set_tid(tid);
+
if (!mds_requests.empty())
req->set_oldest_client_tid(mds_requests.begin()->first);
+ else
+ req->set_oldest_client_tid(tid); // this one is the oldest.
// make note
MetaRequest request(req, tid);
// open a session?
if (mds_sessions.count(mds) == 0) {
Cond cond;
+
if (waiting_for_session.count(mds) == 0) {
dout(10) << "opening session to mds" << mds << endl;
- messenger->send_message(new MClientSession(MClientSession::OP_OPEN),
+ messenger->send_message(new MClientSession(MClientSession::OP_REQUEST_OPEN),
mdsmap->get_inst(mds), MDS_PORT_SERVER);
}
int from = m->get_source().num();
switch (m->op) {
- case MClientSession::OP_OPEN_ACK:
- mds_sessions.insert(from);
+ case MClientSession::OP_OPEN:
+ assert(mds_sessions.count(from) == 0);
+ mds_sessions[from] = 0;
break;
- case MClientSession::OP_CLOSE_ACK:
+ case MClientSession::OP_CLOSE:
mds_sessions.erase(from);
// FIXME: kick requests (hard) so that they are redirected. or fail.
break;
<< " " << cap_string(p->second->caps[mds].caps)
<< " wants " << cap_string(p->second->file_caps_wanted())
<< endl;
- m->add_inode_caps(p->first,
- p->second->caps[mds].caps,
- p->second->caps[mds].seq,
- p->second->file_caps_wanted(),
- p->second->inode.size,
- p->second->inode.mtime, p->second->inode.atime);
+ p->second->caps[mds].seq = 0; // reset seq.
+ m->add_inode_caps(p->first, // ino
+ p->second->file_caps_wanted(), // wanted
+ p->second->caps[mds].caps, // issued
+ p->second->inode.size, p->second->inode.mtime, p->second->inode.atime);
string path;
p->second->make_path(path);
dout(10) << " path on " << p->first << " is " << path << endl;
dout(10) << " clearing stale caps on " << p->first << endl;
p->second->stale_caps.erase(mds); // hrm, is this right?
}
- }
+ }
+
+ // reset my cap seq number
+ mds_sessions[mds] = 0;
} else {
dout(10) << " i had no session with this mds";
m->closed = true;
m->clear_payload(); // for if/when we send back to MDS
+ // note push seq increment
+ assert(mds_sessions.count(mds));
+ mds_sessions[mds]++;
+
// reap?
- if (m->get_special() == MClientFileCaps::OP_REAP) {
+ if (m->get_op() == MClientFileCaps::OP_REAP) {
int other = m->get_mds();
if (in && in->stale_caps.count(other)) {
assert(in);
// stale?
- if (m->get_special() == MClientFileCaps::OP_STALE) {
+ if (m->get_op() == MClientFileCaps::OP_STALE) {
dout(5) << "handle_file_caps on ino " << m->get_ino() << " seq " << m->get_seq() << " from mds" << mds << " now stale" << endl;
// move to stale list
}
// release?
- if (m->get_special() == MClientFileCaps::OP_RELEASE) {
+ if (m->get_op() == MClientFileCaps::OP_RELEASE) {
dout(5) << "handle_file_caps on ino " << m->get_ino() << " from mds" << mds << " release" << endl;
assert(in->caps.count(mds));
in->caps.erase(mds);
<< ", which we don't want caps for, releasing." << endl;
m->set_caps(0);
m->set_wanted(0);
- messenger->send_message(m, m->get_source_inst(), m->get_source_port());
+ messenger->send_message(m, m->get_source_inst(), MDS_PORT_LOCKER);
return;
}
in->file_wr_size = 0;
}
- messenger->send_message(m, m->get_source_inst(), m->get_source_port());
+ messenger->send_message(m, m->get_source_inst(), MDS_PORT_LOCKER);
}
// release (some of?) these caps
it->second.caps = retain & it->second.caps;
// note: tell mds _full_ wanted; it'll filter/behave based on what it is allowed to do
- MClientFileCaps *m = new MClientFileCaps(in->inode,
+ MClientFileCaps *m = new MClientFileCaps(MClientFileCaps::OP_ACK,
+ in->inode,
it->second.seq,
it->second.caps,
in->file_caps_wanted());
for (map<int,InodeCap>::iterator it = in->caps.begin();
it != in->caps.end();
it++) {
- MClientFileCaps *m = new MClientFileCaps(in->inode,
+ MClientFileCaps *m = new MClientFileCaps(MClientFileCaps::OP_ACK,
+ in->inode,
it->second.seq,
it->second.caps,
in->file_caps_wanted());
}
// send session closes!
- for (set<int>::iterator p = mds_sessions.begin();
+ for (map<int,version_t>::iterator p = mds_sessions.begin();
p != mds_sessions.end();
++p) {
- dout(2) << "sending client_session close to mds" << *p << endl;
- messenger->send_message(new MClientSession(MClientSession::OP_CLOSE),
- mdsmap->get_inst(*p), MDS_PORT_SERVER);
+ dout(2) << "sending client_session close to mds" << p->first << " seq " << p->second << endl;
+ messenger->send_message(new MClientSession(MClientSession::OP_REQUEST_CLOSE,
+ p->second),
+ mdsmap->get_inst(p->first), MDS_PORT_SERVER);
}
// send unmount!
Dir *open_dir() {
if (!dir) {
if (dn) dn->get(); // pin dentry
- get();
+ get(); // pin inode
dir = new Dir(this);
}
return dir;
MonMap *monmap;
// mds sessions
- set<int> mds_sessions;
+ map<int, version_t> mds_sessions; // mds -> push seq
map<int, list<Cond*> > waiting_for_session;
void handle_client_session(MClientSession *m);
delete in->dir;
in->dir = 0;
- put_inode(in);
+ put_inode(in); // unpin inode
}
int get_cache_size() { return lru.lru_get_size(); }
in->dn = dn;
in->get();
+ if (in->dir) dn->get(); // dir -> dn pin
+
lru.lru_insert_mid(dn); // mid or top?
return dn;
}
Inode *in = dn->inode;
// unlink from inode
+ if (dn->inode->dir) dn->put(); // dir -> dn pin
dn->inode = 0;
in->dn = 0;
put_inode(in);
delete dn;
}
- Dentry *relink(Dentry *dn, Dir *dir, const string& name) {
- // first link new dn to dir
- /*
- char *oldname = (char*)dn->name;
- dn->name = new char[name.length()+1];
- strcpy((char*)dn->name, name.c_str());
- dir->dentries[dn->name] = dn;
- */
- //cout << "relink dir " << dir->parent_inode->inode.ino << " '" << name << "' -> inode " << dn->inode->inode.ino << endl;
+ Dentry *relink(Dir *dir, const string& name, Inode *in) {
+ Dentry *olddn = in->dn;
+ Dir *olddir = olddn->dir; // note: might == dir!
- dir->dentries[name] = dn;
+ // newdn, attach to inode. don't touch inode ref.
+ Dentry *newdn = new Dentry;
+ newdn->name = name;
+ newdn->inode = in;
+ newdn->dir = dir;
+ in->dn = newdn;
- // unlink from old dir
- dn->dir->dentries.erase(dn->name);
- //delete[] oldname;
- if (dn->dir->is_empty())
- close_dir(dn->dir);
+ if (in->dir) { // dir -> dn pin
+ newdn->get();
+ olddn->put();
+ }
- // fix up dn
- dn->name = name;
- dn->dir = dir;
+ // unlink old dn from dir
+ olddir->dentries.erase(olddn->name);
+ olddn->inode = 0;
+ olddn->dir = 0;
+ lru.lru_remove(olddn);
+
+ // link new dn to dir
+ dir->dentries[name] = newdn;
+ lru.lru_insert_mid(newdn);
+
+ // olddir now empty? (remember, olddir might == dir)
+ if (olddir->is_empty())
+ close_dir(olddir);
- return dn;
+ return newdn;
}
// move dentry to top of lru
void SyntheticClient::foo()
{
+ if (1) {
+ // open some files
+ srand(0);
+ for (int i=0; i<20; i++) {
+ int s = 5;
+ int a = rand() % s;
+ int b = rand() % s;
+ int c = rand() % s;
+ char src[80];
+ sprintf(src, "syn.0.0/dir.%d/dir.%d/file.%d", a, b, c);
+ int fd = client->open(src, O_RDONLY);
+ }
+
+ return;
+ }
+
+ if (0) {
+ // rename fun
+ for (int i=0; i<100; i++) {
+ int s = 5;
+ int a = rand() % s;
+ int b = rand() % s;
+ int c = rand() % s;
+ int d = rand() % s;
+ int e = rand() % s;
+ int f = rand() % s;
+ char src[80];
+ char dst[80];
+ sprintf(src, "syn.0.0/dir.%d/dir.%d/file.%d", a, b, c);
+ sprintf(dst, "syn.0.0/dir.%d/dir.%d/file.%d", d, e, f);
+ client->rename(src, dst);
+ }
+ return;
+ }
+
+ if (1) {
+ // link fun
+ srand(0);
+ for (int i=0; i<100; i++) {
+ int s = 5;
+ int a = rand() % s;
+ int b = rand() % s;
+ int c = rand() % s;
+ int d = rand() % s;
+ int e = rand() % s;
+ int f = rand() % s;
+ char src[80];
+ char dst[80];
+ sprintf(src, "syn.0.0/dir.%d/dir.%d/file.%d", a, b, c);
+ sprintf(dst, "syn.0.0/dir.%d/dir.%d/newlink.%d", d, e, f);
+ client->link(src, dst);
+ }
+ srand(0);
+ for (int i=0; i<100; i++) {
+ int s = 5;
+ int a = rand() % s;
+ int b = rand() % s;
+ int c = rand() % s;
+ int d = rand() % s;
+ int e = rand() % s;
+ int f = rand() % s;
+ char src[80];
+ char dst[80];
+ sprintf(src, "syn.0.0/dir.%d/dir.%d/file.%d", a, b, c);
+ sprintf(dst, "syn.0.0/dir.%d/dir.%d/newlink.%d", d, e, f);
+ client->unlink(dst);
+ }
+
+
+ return;
+ }
+
// link fun
client->mknod("one", 0755);
client->mknod("two", 0755);
<< endl;
if (time_to_stop()) return 0;
+
+ for (int k=0; k<n; k++) {
+
+ if (rand() % 10 == 0) {
+ // rename some directories. whee!
+ int dep = (rand() % depth) + 1;
+ string src = basedir;
+ {
+ char t[80];
+ for (int d=0; d<dep; d++) {
+ int a = rand() % dirs;
+ sprintf(t, "/dir.%d", a);
+ src += t;
+ }
+ }
+ string dst = basedir;
+ {
+ char t[80];
+ for (int d=0; d<dep; d++) {
+ int a = rand() % dirs;
+ sprintf(t, "/dir.%d", a);
+ dst += t;
+ }
+ }
+
+ if (client->rename(dst.c_str(), "/tmp") == 0) {
+ client->rename(src.c_str(), dst.c_str());
+ client->rename("/tmp", src.c_str());
+ }
+ continue;
+ }
+
+ // pick a dest dir
+ string src = basedir;
+ {
+ char t[80];
+ for (int d=0; d<depth; d++) {
+ int a = rand() % dirs;
+ sprintf(t, "/dir.%d", a);
+ src += t;
+ }
+ int a = rand() % files;
+ sprintf(t, "/file.%d", a);
+ src += t;
+ }
+ string dst = basedir;
+ {
+ char t[80];
+ for (int d=0; d<depth; d++) {
+ int a = rand() % dirs;
+ sprintf(t, "/dir.%d", a);
+ dst += t;
+ }
+ int a = rand() % files;
+ sprintf(t, "/file.%d", a);
+ dst += t;
+ }
+
+ int o = rand() % 4;
+ switch (o) {
+ case 0:
+ client->mknod(src.c_str(), 0755);
+ client->rename(src.c_str(), dst.c_str());
+ break;
+ case 1:
+ client->mknod(src.c_str(), 0755);
+ client->unlink(dst.c_str());
+ client->link(src.c_str(), dst.c_str());
+ break;
+ case 2: client->unlink(src.c_str()); break;
+ case 3: client->unlink(dst.c_str()); break;
+ //case 4: client->mknod(src.c_str(), 0755); break;
+ //case 5: client->mknod(dst.c_str(), 0755); break;
+ }
+ }
+ return 0;
// now link shit up
for (int i=0; i<n; i++) {
FileLayout g_OSD_MDLogLayout( 1<<20, 1, 1<<20, pg_t::TYPE_REP, 2 ); // 1M objects
FileLayout g_OSD_MDAnchorTableLayout( 1<<20, 1, 1<<20, pg_t::TYPE_REP, 2 ); // 1M objects. (a lie, just object layout policy)
+#include <msg/msg_types.h>
+
// fake osd failures: osd -> time
+std::map<entity_name_t,float> g_fake_kill_after;
std::map<int,float> g_fake_osd_down;
std::map<int,float> g_fake_osd_out;
debug_ns: 0,
debug_ms: 0,
debug_mon: 0,
+ debug_paxos: 0,
debug_after: 0,
// --- journaler ---
journaler_allow_split_entries: true,
journaler_safe: false, // wait for COMMIT on journal writes
+ journaler_write_head_interval: 15,
// --- mds ---
mds_cache_size: MDS_CACHE_SIZE,
mds_decay_halflife: 30,
- mds_beacon_interval: 5.0,
- mds_beacon_grace: 10.0,
+ mds_beacon_interval: 5, //30.0,
+ mds_beacon_grace: 15, //60*60.0,
mds_log: true,
mds_log_max_len: MDS_CACHE_SIZE / 3,
mds_log_read_inc: 1<<20,
mds_log_pad_entry: 128,//256,//64,
mds_log_flush_on_shutdown: true,
- mds_log_import_map_interval: 1024*1024, // frequency (in bytes) of EImportMap in log
+ mds_log_subtree_map_interval: 128*1024, // frequency (in bytes) of EImportMap in log
mds_log_eopen_size: 100, // # open inodes per log entry
mds_bal_replicate_threshold: 2000,
mds_thrash_exports: 0,
mds_dump_cache_on_map: false,
+ mds_dump_cache_after_rejoin: true,
// --- osd ---
osd_rep: OSD_REP_PRIMARY,
g_conf.fake_osd_mttf = atoi(args[++i]);
else if (strcmp(args[i], "--fake_osd_mttr") == 0)
g_conf.fake_osd_mttr = atoi(args[++i]);
+
+ else if (strcmp(args[i], "--fake_kill_osd_after") == 0) {
+ g_fake_kill_after[entity_name_t(entity_name_t::TYPE_OSD, atoi(args[i+1]))] = atof(args[i+2]);
+ i += 2;
+ }
+ else if (strcmp(args[i], "--fake_kill_mds_after") == 0) {
+ g_fake_kill_after[entity_name_t(entity_name_t::TYPE_MDS, atoi(args[i+1]))] = atof(args[i+2]);
+ i += 2;
+ }
+ else if (strcmp(args[i], "--fake_kill_mon_after") == 0) {
+ g_fake_kill_after[entity_name_t(entity_name_t::TYPE_MON, atoi(args[i+1]))] = atof(args[i+2]);
+ i += 2;
+ }
+ else if (strcmp(args[i], "--fake_kill_client_after") == 0) {
+ g_fake_kill_after[entity_name_t(entity_name_t::TYPE_CLIENT, atoi(args[i+1]))] = atof(args[i+2]);
+ i += 2;
+ }
+
else if (strcmp(args[i], "--fake_osd_down") == 0) {
int osd = atoi(args[++i]);
float when = atof(args[++i]);
int debug_ns;
int debug_ms;
int debug_mon;
+ int debug_paxos;
int debug_after;
// journaler
bool journaler_allow_split_entries;
bool journaler_safe;
+ int journaler_write_head_interval;
// mds
int mds_cache_size;
int mds_log_read_inc;
int mds_log_pad_entry;
bool mds_log_flush_on_shutdown;
- off_t mds_log_import_map_interval;
+ off_t mds_log_subtree_map_interval;
int mds_log_eopen_size;
float mds_bal_replicate_threshold;
int mds_thrash_exports;
bool mds_dump_cache_on_map;
+ bool mds_dump_cache_after_rejoin;
// osd
int osd_rep;
{
bio->done = true;
if (bio->cond) {
+ lock.Lock(); // hmm?
bio->cond->Signal();
+ lock.Unlock();
}
else if (bio->cb) {
bio->cb->finish((ioh_t)bio, bio->rval);
assert(off + len <= EBOFS_BLOCK_SIZE);
// trim any existing that overlaps
- for (map<off_t, bufferlist>::iterator i = partial.begin();
- i != partial.end();
- ) {
- if (i->first + i->second.length() <= off) { // before
+ map<off_t, bufferlist>::iterator i = partial.begin();
+ while (i != partial.end()) {
+ // is [off,off+len)...
+ // past i?
+ if (off >= i->first + i->second.length()) {
i++;
continue;
}
- if (i->first >= off+len) break; // past affected area.
-
- // overlap all?
- if (off <= i->first && i->first + i->second.length() <= off+len) {
+ // before i?
+ if (i->first >= off+len) break;
+
+ // does [off,off+len)...
+ // overlap all of i?
+ if (off <= i->first && off+len >= i->first + i->second.length()) {
// erase it and move on.
- off_t dead = i->first;
- i++;
- partial.erase(dead);
+ partial.erase(i++);
continue;
}
- // overlap tail?
- else if (i->first < off && off < i->first + i->second.length()) {
- // shorten.
- unsigned newlen = off - i->first;
+ // overlap tail of i?
+ if (off > i->first && off < i->first + i->second.length()) {
+ // shorten i.
bufferlist o;
o.claim( i->second );
- i->second.substr_of(o, 0, newlen);
+ unsigned taillen = off - i->first;
+ i->second.substr_of(o, 0, taillen);
i++;
continue;
}
- // overlap head?
- else if (off < i->first && off+len < i->first + i->second.length()) {
- // move.
- off_t oldoff = i->first;
- off_t newoff = off+len;
- unsigned trim = newoff - oldoff;
- partial[newoff].substr_of(i->second, trim, i->second.length()-trim);
- i++; // should be at newoff!
- partial.erase( oldoff );
+ // overlap head of i?
+ if (off < i->first && off+len < i->first + i->second.length()) {
+ // move i (make new tail).
+ off_t tailoff = off+len;
+ unsigned trim = tailoff - i->first;
+ partial[tailoff].substr_of(i->second, trim, i->second.length()-trim);
+ partial.erase(i++); // should now be at tailoff
i++;
continue;
- } else
- assert(0);
+ }
+ // split i?
+ if (off > i->first && off+len < i->first + i->second.length()) {
+ bufferlist o;
+ o.claim( i->second );
+ // shorten head
+ unsigned headlen = off - i->first;
+ i->second.substr_of(o, 0, headlen);
+ // new tail
+ unsigned tailoff = off+len - i->first;
+ unsigned taillen = o.length() - len - headlen;
+ partial[off+len].substr_of(o, tailoff, taillen);
+ break;
+ }
+ assert(0);
}
// insert
if (e < super_epoch) {
dout(-3) << "mount replay: skipping old entry in epoch " << e << " < " << super_epoch << endl;
+ continue;
}
if (e == super_epoch+1) {
super_epoch++;
-void Ebofs::apply_write(Onode *on, off_t off, size_t len, bufferlist& bl)
+void Ebofs::apply_write(Onode *on, off_t off, size_t len, const bufferlist& bl)
{
ObjectCache *oc = on->get_oc(&bc);
-int Ebofs::_write(object_t oid, off_t offset, size_t length, bufferlist& bl)
+int Ebofs::_write(object_t oid, off_t offset, size_t length, const bufferlist& bl)
{
dout(7) << "_write " << oid << " " << offset << "~" << length << endl;
assert(bl.length() == length);
int Ebofs::write(object_t oid,
off_t off, size_t len,
- bufferlist& bl, Context *onsafe)
+ const bufferlist& bl, Context *onsafe)
{
ebofs_lock.Lock();
assert(len > 0);
if (journal) {
Transaction t;
t.write(oid, off, len, bl);
- bufferlist bl;
- t._encode(bl);
- if (journal->submit_entry(bl, onsafe)) break;
+ bufferlist tbl;
+ t._encode(tbl);
+ if (journal->submit_entry(tbl, onsafe)) break;
}
if (onsafe) commit_waiters[super_epoch].push_back(onsafe);
break;
block_t start, block_t len,
interval_set<block_t>& alloc,
block_t& old_bfirst, block_t& old_blast);
- void apply_write(Onode *on, off_t off, size_t len, bufferlist& bl);
+ void apply_write(Onode *on, off_t off, size_t len, const bufferlist& bl);
bool attempt_read(Onode *on, off_t off, size_t len, bufferlist& bl,
Cond *will_wait_on, bool *will_wait_on_bool);
int read(object_t, off_t off, size_t len, bufferlist& bl);
int is_cached(object_t oid, off_t off, size_t len);
- int write(object_t oid, off_t off, size_t len, bufferlist& bl, Context *onsafe);
+ int write(object_t oid, off_t off, size_t len, const bufferlist& bl, Context *onsafe);
void trim_from_cache(object_t oid, off_t off, size_t len);
int truncate(object_t oid, off_t size, Context *onsafe=0);
int truncate_front(object_t oid, off_t size, Context *onsafe=0);
int _getattrs(object_t oid, map<string,bufferptr> &aset);
bool _write_will_block();
- int _write(object_t oid, off_t off, size_t len, bufferlist& bl);
+ int _write(object_t oid, off_t off, size_t len, const bufferlist& bl);
void _trim_from_cache(object_t oid, off_t off, size_t len);
int _truncate(object_t oid, off_t size);
int _truncate_front(object_t oid, off_t size);
#include "config.h"
#undef dout
-#define dout(x) if (true || x <= g_conf.debug_ebofs) cout << "ebofs(" << ebofs->dev.get_device_name() << ").journal "
+#define dout(x) if (x <= g_conf.debug_ebofs) cout << "ebofs(" << ebofs->dev.get_device_name() << ").journal "
#define derr(x) if (x <= g_conf.debug_ebofs) cerr << "ebofs(" << ebofs->dev.get_device_name() << ").journal "
dout(1) << "open " << fn << " " << st.st_size << " bytes" << endl;
// write empty header
+ memset(&header, 0, sizeof(header));
header.clear();
header.fsid = ebofs->get_fsid();
header.max_size = st.st_size;
<< endl;
full = true;
print_header();
- return false;
+ return false;
}
} else {
// we haven't wrapped.
block_t off = nodeid_offset(*i);
block_t b = region_loc[region].start + off;
- if (1) { // sanity check debug FIXME
+ if (0) { // sanity check debug FIXME
assert(didb.count(b) == 0);
didb.insert(b);
}
vector<char*> args;
argv_to_vec(argc, argv, args);
+ // stop on our own (by default)
+ g_conf.mon_stop_on_last_unmount = true;
+ g_conf.mon_stop_with_last_mds = true;
+
parse_config_options(args);
int start = 0;
if (g_conf.clock_tare) g_clock.tare();
+
MonMap *monmap = new MonMap(g_conf.num_mon);
entity_addr_t a;
for (int i=0; i<g_conf.num_mon; i++) {
}
};
- Context *new_sub() {
- num++;
- waitfor.insert(num);
- return new C_GatherSub(this, num);
- }
-
private:
Context *onfinish;
std::set<int> waitfor;
int num;
public:
- C_Gather(Context *f) : onfinish(f), num(0) {
+ C_Gather(Context *f=0) : onfinish(f), num(0) {
//cout << "C_Gather new " << this << endl;
}
~C_Gather() {
//cout << "C_Gather delete " << this << endl;
assert(!onfinish);
}
+
+ void set_finisher(Context *c) {
+ assert(!onfinish);
+ onfinish = c;
+ }
+ Context *new_sub() {
+ num++;
+ waitfor.insert(num);
+ return new C_GatherSub(this, num);
+ }
+
+ bool empty() { return num == 0; }
+ int get_num() { return num; }
+
void finish(int r) {
- // nobody should ever call me.
- assert(0);
+ assert(0); // nobody should ever call me.
}
};
char *c_str() { assert(_raw); return _raw->data + _off; }
unsigned length() const { return _len; }
unsigned offset() const { return _off; }
- unsigned unused_tail_length() const { return _raw->len - (_off+_len); }
+ unsigned start() const { return _off; }
+ unsigned end() const { return _off + _len; }
+ unsigned unused_tail_length() const {
+ if (_raw)
+ return _raw->len - (_off+_len);
+ else
+ return 0;
+ }
const char& operator[](unsigned n) const {
assert(_raw);
assert(n < _len);
// my private bits
std::list<ptr> _buffers;
unsigned _len;
+ ptr append_buffer; // where i put small appends.
public:
// cons/des
const std::list<ptr>& buffers() const { return _buffers; }
unsigned length() const {
-#if 0
+#if 1
// DEBUG: verify _len
unsigned len = 0;
- for (std::list<ptr>::iterator it = _buffers.begin();
+ for (std::list<ptr>::const_iterator it = _buffers.begin();
it != _buffers.end();
it++) {
len += (*it).length();
void append(const char *data, unsigned len) {
- if (len == 0) return;
-
- unsigned alen = 0;
-
- // copy into the tail buffer?
- if (!_buffers.empty()) {
- unsigned avail = _buffers.back().unused_tail_length();
- if (avail > 0) {
- //std::cout << "copying up to " << len << " into tail " << avail << " bytes of tail buf " << _buffers.back() << std::endl;
- if (avail > len)
- avail = len;
- _buffers.back().append(data, avail);
- _len += avail;
- data += avail;
- len -= avail;
+ while (len > 0) {
+ // put what we can into the existing append_buffer.
+ if (append_buffer.unused_tail_length() > 0) {
+ unsigned gap = append_buffer.unused_tail_length();
+ if (gap > len) gap = len;
+ append_buffer.append(data, gap);
+ append(append_buffer, append_buffer.end() - gap, gap); // add segment to the list
+ len -= gap;
+ data += gap;
}
- alen = _buffers.back().length();
+ if (len == 0) break; // done!
+
+ // make a new append_buffer!
+ unsigned alen = BUFFER_PAGE_SIZE * (((len-1) / BUFFER_PAGE_SIZE) + 1);
+ append_buffer = create_page_aligned(alen);
+ append_buffer.set_length(0); // unused, so far.
}
- if (len == 0) return;
-
- // just add another buffer.
- // alloc a bit extra, in case we do a bunch of appends. FIXME be smarter!
- if (alen < 4096) alen = 4096;
- ptr bp = create(alen);
- bp.set_length(len);
- bp.copy_in(0, len, data);
- push_back(bp);
}
void append(ptr& bp) {
push_back(bp);
push_back(tempbp);
}
void append(const list& bl) {
- list temp(bl); // copy list
- claim_append(temp); // and append
+ _len += bl._len;
+ for (std::list<ptr>::const_iterator p = bl._buffers.begin();
+ p != bl._buffers.end();
+ ++p)
+ _buffers.push_back(*p);
}
}
}
- void substr_of(list& other, unsigned off, unsigned len) {
+ void substr_of(const list& other, unsigned off, unsigned len) {
assert(off + len <= other.length());
clear();
// skip off
- std::list<ptr>::iterator curbuf = other._buffers.begin();
+ std::list<ptr>::const_iterator curbuf = other._buffers.begin();
while (off > 0) {
assert(curbuf != _buffers.end());
if (off >= (*curbuf).length()) {
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation. See file COPYING.
+ *
+ */
+
+#ifndef __ENCODABLE_H
+#define __ENCODABLE_H
+
+#include "buffer.h"
+
+#include <set>
+#include <map>
+#include <deque>
+#include <vector>
+#include <string>
+#include <ext/hash_map>
+
+// list
+template<class T>
+inline void _encode_complex(const std::list<T>& ls, bufferlist& bl)
+{
+ uint32_t n = ls.size();
+ _encoderaw(n, bl);
+ for (typename std::list<T>::const_iterator p = ls.begin(); p != ls.end(); ++p)
+ _encode_complex(*p, bl);
+}
+template<class T>
+inline void _decode_complex(std::list<T>& ls, bufferlist& bl, int& off)
+{
+ uint32_t n;
+ _decoderaw(n, bl, off);
+ ls.clear();
+ while (n--) {
+ T v;
+ _decode_complex(v, bl, off);
+ ls.push_back(v);
+ }
+}
+
+// deque
+template<class T>
+inline void _encode_complex(const std::deque<T>& ls, bufferlist& bl)
+{
+ uint32_t n = ls.size();
+ _encoderaw(n, bl);
+ for (typename std::deque<T>::const_iterator p = ls.begin(); p != ls.end(); ++p)
+ _encode_complex(*p, bl);
+}
+template<class T>
+inline void _decode_complex(std::deque<T>& ls, bufferlist& bl, int& off)
+{
+ uint32_t n;
+ _decoderaw(n, bl, off);
+ ls.clear();
+ while (n--) {
+ T v;
+ _decode_complex(v, bl, off);
+ ls.push_back(v);
+ }
+}
+
+// set
+template<class T>
+inline void _encode_complex(const std::set<T>& s, bufferlist& bl)
+{
+ uint32_t n = s.size();
+ _encoderaw(n, bl);
+ for (typename std::set<T>::const_iterator p = s.begin(); p != s.end(); ++p)
+ _encode_complex(*p, bl);
+}
+template<class T>
+inline void _decode_complex(std::set<T>& s, bufferlist& bl, int& off)
+{
+ uint32_t n;
+ _decoderaw(n, bl, off);
+ s.clear();
+ while (n--) {
+ T v;
+ _decode_complex(v, bl, off);
+ s.insert(v);
+ }
+}
+
+// vector
+template<class T>
+inline void _encode_complex(const std::vector<T>& v, bufferlist& bl)
+{
+ uint32_t n = v.size();
+ _encoderaw(n, bl);
+ for (typename std::vector<T>::const_iterator p = v.begin(); p != v.end(); ++p)
+ _encode_complex(*p, bl);
+}
+template<class T>
+inline void _decode_complex(std::vector<T>& v, bufferlist& bl, int& off)
+{
+ uint32_t n;
+ _decoderaw(n, bl, off);
+ v.resize(n);
+ for (uint32_t i=0; i<n; i++)
+ _decode_complex(v[i], bl, off);
+}
+
+// map
+template<class T, class U>
+inline void _encode_complex(const std::map<T,U>& m, bufferlist& bl)
+{
+ uint32_t n = m.size();
+ _encoderaw(n, bl);
+ for (typename std::map<T,U>::const_iterator p = m.begin(); p != m.end(); ++p) {
+ _encode(p->first, bl);
+ _encode_complex(p->second, bl);
+ }
+}
+template<class T, class U>
+inline void _decode_complex(std::map<T,U>& m, bufferlist& bl, int& off)
+{
+ uint32_t n;
+ _decoderaw(n, bl, off);
+ m.clear();
+ while (n--) {
+ T k;
+ _decode(k, bl, off);
+ _decode_complex(m[k], bl, off);
+ }
+}
+
+// hash_map
+template<class T, class U>
+inline void _encode_complex(const __gnu_cxx::hash_map<T,U>& m, bufferlist& bl)
+{
+ uint32_t n = m.size();
+ _encoderaw(n, bl);
+ for (typename __gnu_cxx::hash_map<T,U>::const_iterator p = m.begin(); p != m.end(); ++p) {
+ _encode(p->first, bl);
+ _encode_complex(p->second, bl);
+ }
+}
+template<class T, class U>
+inline void _decode_complex(__gnu_cxx::hash_map<T,U>& m, bufferlist& bl, int& off)
+{
+ uint32_t n;
+ _decoderaw(n, bl, off);
+ m.clear();
+ while (n--) {
+ T k;
+ _decode(k, bl, off);
+ _decode_complex(m[k], bl, off);
+ }
+}
+
+// base case
+template<class T>
+inline void _encode_complex(const T& t, bufferlist& bl)
+{
+ t._encode(bl);
+}
+template<class T>
+inline void _decode_complex(T& t, bufferlist& bl, int& off)
+{
+ t._decode(bl, off);
+}
+
+#endif
bool anchored; // auth only?
// file (data access)
- off_t size, max_size;
+ off_t size, max_size, allocated_size;
utime_t mtime; // file data modify time.
utime_t atime; // file data access time.
#include <math.h>
#include <sys/time.h>
+#include <time.h>
// --------
// utime_t
// ostream
inline std::ostream& operator<<(std::ostream& out, const utime_t& t)
{
- //return out << t.sec() << "." << t.usec();
- out << (long)t.sec() << ".";
out.setf(std::ios::right);
out.fill('0');
+ if (t.sec() < ((time_t)(60*60*24*365*10))) {
+ // raw seconds. this looks like a relative time.
+ out << (long)t.sec();
+ } else {
+ // localtime. this looks like an absolute time.
+ struct tm bdt;
+ time_t tt = t.sec();
+ localtime_r(&tt, &bdt);
+ out << std::setw(2) << (bdt.tm_year-100) // 2007 -> '07'
+ << std::setw(2) << bdt.tm_mon
+ << std::setw(2) << bdt.tm_mday
+ << "."
+ << std::setw(2) << bdt.tm_hour
+ << std::setw(2) << bdt.tm_min
+ << std::setw(2) << bdt.tm_sec;
+ }
+ out << ".";
out << std::setw(6) << t.usec();
out.unsetf(std::ios::right);
return out;
-
- //return out << (long)t.sec << "." << ios::setf(ios::right) << ios::fill('0') << t.usec() << ios::usetf();
}
#endif
void AnchorTable::dec(inodeno_t ino)
{
dout(7) << "dec " << ino << endl;
-
assert(anchor_map.count(ino));
- Anchor &anchor = anchor_map[ino];
while (true) {
+ Anchor &anchor = anchor_map[ino];
anchor.nref--;
if (anchor.nref == 0) {
if (ino == 0) break;
if (anchor_map.count(ino) == 0) break;
- anchor = anchor_map[ino];
}
}
#undef dout
#define dout(x) if (x <= g_conf.debug || x <= g_conf.debug_mds) cout << g_clock.now() << " mds" << dir->cache->mds->get_nodeid() << ".cache.den(" << dir->ino() << " " << name << ") "
+ostream& CDentry::print_db_line_prefix(ostream& out)
+{
+ return out << g_clock.now() << " mds" << dir->cache->mds->get_nodeid() << ".cache.den(" << dir->ino() << " " << name << ") ";
+}
+
// CDentry
out << " inode=" << dn.get_inode();
+ if (dn.is_new()) out << " state=new";
+
if (dn.get_num_ref()) {
out << " |";
dn.print_pin_set(out);
}
+void CDentry::add_waiter(int tag, Context *c)
+{
+ // wait on the directory?
+ if (tag & (WAIT_AUTHPINNABLE|WAIT_SINGLEAUTH)) {
+ dir->add_waiter(tag, c);
+ return;
+ }
+ MDSCacheObject::add_waiter(tag, c);
+}
+
+
version_t CDentry::pre_dirty(version_t min)
{
projected_version = dir->pre_dirty(min);
// state+pin
if (!state_test(STATE_DIRTY)) {
state_set(STATE_DIRTY);
+ dir->inc_num_dirty();
get(PIN_DIRTY);
}
}
dir->mark_dirty(pv);
}
-void CDentry::mark_clean() {
+
+void CDentry::mark_clean()
+{
dout(10) << " mark_clean " << *this << endl;
assert(is_dirty());
assert(version <= dir->get_version());
- // this happens on export.
- //assert(version <= dir->get_last_committed_version());
-
// state+pin
state_clear(STATE_DIRTY);
+ dir->dec_num_dirty();
put(PIN_DIRTY);
+
+ if (state_test(STATE_NEW))
+ state_clear(STATE_NEW);
}
+void CDentry::mark_new()
+{
+ dout(10) << " mark_new " << *this << endl;
+ state_set(STATE_NEW);
+}
void CDentry::make_path(string& s)
{
s += name;
}
+void CDentry::make_path(string& s, inodeno_t tobase)
+{
+ assert(dir);
+
+ if (dir->inode->is_root()) {
+ s += "/"; // make it an absolute path (no matter what) if we hit the root.
+ }
+ else if (dir->inode->get_parent_dn() &&
+ dir->inode->ino() != tobase) {
+ dir->inode->get_parent_dn()->make_path(s, tobase);
+ s += "/";
+ }
+ s += name;
+}
+
/** make_anchor_trace
* construct an anchor trace for this dentry, as if it were linked to *in.
*/
}
+// ----------------------------
+// auth pins
+
+bool CDentry::can_auth_pin()
+{
+ assert(dir);
+ return dir->can_auth_pin();
+}
+
+void CDentry::auth_pin()
+{
+ assert(dir);
+ dir->auth_pin();
+}
+
+void CDentry::auth_unpin()
+{
+ assert(dir);
+ dir->auth_unpin();
+}
+
+
// ----------------------------
// locking
-void CDentry::set_mlock_info(MLock *m)
+void CDentry::set_object_info(MDSCacheObjectInfo &info)
{
- m->set_dn(dir->dirfrag(), name);
+ info.dirfrag = dir->dirfrag();
+ info.dname = name;
}
void CDentry::encode_lock_state(int type, bufferlist& bl)
{
-
+ // null, ino, or remote_ino?
+ int c;
+ if (is_primary()) {
+ c = 1;
+ ::_encode(c, bl);
+ ::_encode(inode->inode.ino, bl);
+ }
+ else if (is_remote()) {
+ c = 2;
+ ::_encode(c, bl);
+ ::_encode(remote_ino, bl);
+ }
+ else if (is_null()) {
+ // encode nothing.
+ }
+ else assert(0);
}
void CDentry::decode_lock_state(int type, bufferlist& bl)
-{
+{
+ if (bl.length() == 0) {
+ // null
+ assert(is_null());
+ return;
+ }
+ int off = 0;
+ char c;
+ inodeno_t ino;
+ ::_decode(c, bl, off);
+
+ switch (c) {
+ case 1:
+ case 2:
+ _decode(ino, bl, off);
+ // newly linked?
+ if (is_null() && !is_auth()) {
+ // force trim from cache!
+ dout(10) << "decode_lock_state replica dentry null -> non-null, must trim" << endl;
+ //assert(get_num_ref() == 0);
+ } else {
+ // verify?
+
+ }
+ break;
+ default:
+ assert(0);
+ }
}
class CDentry : public MDSCacheObject, public LRUObject {
public:
// -- state --
+ static const int STATE_NEW = 1;
// -- pins --
static const int PIN_INODEPIN = 1; // linked inode is pinned
// -- wait --
static const int WAIT_LOCK_OFFSET = 8;
+ void add_waiter(int tag, Context *c);
static const int EXPORT_NONCE = 1;
void last_put() {
lru_unpin();
}
+
+ // auth pins
+ bool can_auth_pin();
+ void auth_pin();
+ void auth_unpin();
// dentry type is primary || remote || null
// misc
void make_path(string& p);
+ void make_path(string& p, inodeno_t tobase);
void make_anchor_trace(vector<class Anchor>& trace, CInode *in);
// -- version --
void mark_dirty(version_t projected_dirv);
void mark_clean();
+ void mark_new();
+ bool is_new() { return state_test(STATE_NEW); }
// -- replication
CDentryDiscover *replicate_to(int rep);
assert(type == LOCK_OTYPE_DN);
return &lock;
}
- void set_mlock_info(MLock *m);
+ void set_object_info(MDSCacheObjectInfo &info);
void encode_lock_state(int type, bufferlist& bl);
void decode_lock_state(int type, bufferlist& bl);
+ ostream& print_db_line_prefix(ostream& out);
void print(ostream& out);
friend class CDir;
int replica_nonce;
int lockstate;
- inodeno_t ino;
inodeno_t remote_ino;
public:
CDentryDiscover(CDentry *dn, int nonce) :
dname(dn->get_name()), replica_nonce(nonce),
lockstate(dn->lock.get_replica_state()),
- ino(dn->get_ino()),
remote_ino(dn->get_remote_ino()) { }
string& get_dname() { return dname; }
void update_dentry(CDentry *dn) {
dn->set_replica_nonce( replica_nonce );
- if (remote_ino)
- dn->set_remote_ino(remote_ino);
}
- void update_new_dentry(CDentry *dn) {
- update_dentry(dn);
+ void init_dentry_lock(CDentry *dn) {
dn->lock.set_state( lockstate );
}
void _encode(bufferlist& bl) {
::_encode(dname, bl);
- bl.append((char*)&replica_nonce, sizeof(replica_nonce));
- bl.append((char*)&lockstate, sizeof(lockstate));
+ ::_encode(remote_ino, bl);
+ ::_encode(replica_nonce, bl);
+ ::_encode(lockstate, bl);
}
void _decode(bufferlist& bl, int& off) {
::_decode(dname, bl, off);
- bl.copy(off, sizeof(replica_nonce), (char*)&replica_nonce);
- off += sizeof(replica_nonce);
- bl.copy(off, sizeof(lockstate), (char*)&lockstate);
- off += sizeof(lockstate);
+ ::_decode(remote_ino, bl, off);
+ ::_decode(replica_nonce, bl, off);
+ ::_decode(lockstate, bl, off);
}
};
out << " v=" << dir.get_version();
out << " cv=" << dir.get_committing_version();
out << "/" << dir.get_committed_version();
+ out << "/" << dir.get_committed_version_equivalent();
} else {
out << " rep@" << dir.authority();
if (dir.get_replica_nonce() > 1)
if (dir.state_test(CDir::STATE_IMPORTBOUND)) out << "|importbound";
out << " sz=" << dir.get_nitems() << "+" << dir.get_nnull();
+ if (dir.get_num_dirty())
+ out << " dirty=" << dir.get_num_dirty();
+
if (dir.get_num_ref()) {
out << " |";
return out << "]";
}
+
void CDir::print(ostream& out)
{
out << *this;
//#define dout(x) if (x <= g_conf.debug || x <= g_conf.debug_mds) cout << g_clock.now() << " mds" << cache->mds->get_nodeid() << ".cache." << *this << " "
+ostream& CDir::print_db_line_prefix(ostream& out)
+{
+ return out << g_clock.now() << " mds" << cache->mds->get_nodeid() << ".cache.dir(" << get_inode()->inode.ino << ") ";
+}
+
+
+
// -------------------------------------------------------------------
// CDir
nitems = 0;
nnull = 0;
+ num_dirty = 0;
+
state = STATE_INITIAL;
projected_version = version = 0;
* linking fun
*/
-CDentry* CDir::add_dentry( const string& dname, inodeno_t ino, bool auth)
+CDentry* CDir::add_dentry( const string& dname, inodeno_t ino)
{
// foreign
assert(lookup(dname) == 0);
// create dentry
CDentry* dn = new CDentry(dname, ino);
- if (auth)
+ if (is_auth())
dn->state_set(CDentry::STATE_AUTH);
cache->lru.lru_insert_mid(dn);
}
-CDentry* CDir::add_dentry( const string& dname, CInode *in, bool auth )
+CDentry* CDir::add_dentry( const string& dname, CInode *in)
{
// primary
assert(lookup(dname) == 0);
// create dentry
CDentry* dn = new CDentry(dname, in);
- if (auth)
+ if (is_auth())
dn->state_set(CDentry::STATE_AUTH);
cache->lru.lru_insert_mid(dn);
//assert(null_items.count(dn->name) == 0);
items[dn->name] = dn;
-
+
if (in) {
link_inode_work( dn, in );
} else {
assert(items.count(dn->name) == 1);
items.erase(dn->name);
+ // adjust dirty counter?
+ if (dn->state_test(CDentry::STATE_DIRTY))
+ num_dirty--;
+
cache->lru.lru_remove(dn);
delete dn;
}
+void CDir::try_remove_unlinked_dn(CDentry *dn)
+{
+ assert(dn->dir == this);
+
+ if (dn->is_new() && dn->is_dirty() &&
+ dn->get_num_ref() == 1) {
+ dout(10) << "try_remove_unlinked_dn " << *dn << " in " << *this << endl;
+ dn->mark_clean();
+ remove_dentry(dn);
+
+ if (version == projected_version &&
+ committing_version == committed_version &&
+ num_dirty == 0) {
+ dout(10) << "try_remove_unlinked_dn committed_equivalent now " << version
+ << " vs committed " << committed_version
+ << endl;
+ committed_version_equivalent = committed_version;
+ }
+ }
+}
+
+
CDirDiscover *CDir::replicate_to(int mds)
void CDir::finish_waiting(int mask, int result)
{
- dout(11) << "finish_waiting mask " << mask << " result " << result << " on " << *this << endl;
+ dout(11) << "finish_waiting mask " << hex << mask << dec << " result " << result << " on " << *this << endl;
list<Context*> finished;
take_waiting(mask, finished);
// parse out inode
inode_t inode;
- bl.copy(off, sizeof(inode), (char*)&inode);
- off += sizeof(inode);
+ ::_decode(inode, bl, off);
string symlink;
if (inode.is_symlink())
::_decode(symlink, bl, off);
fragtree_t fragtree;
- fragtree._decode(bl,off);
+ fragtree._decode(bl, off);
if (dn) {
if (dn->get_inode() == 0) {
// marker, name, ino
bl.append( "L", 1 ); // remote link
- bl.append( it->first.c_str(), it->first.length() + 1);
- bl.append((char*)&ino, sizeof(ino));
+ ::_encode(it->first, bl);
+ ::_encode(ino, bl);
} else {
// primary link
CInode *in = dn->get_inode();
// marker, name, inode, [symlink string]
bl.append( "I", 1 ); // inode
- bl.append( it->first.c_str(), it->first.length() + 1);
- bl.append( (char*) &in->inode, sizeof(inode_t));
+ ::_encode(it->first, bl);
+ ::_encode(in->inode, bl);
if (in->is_symlink()) {
// include symlink destination!
dout(18) << " inlcuding symlink ptr " << in->symlink << endl;
- bl.append( (char*) in->symlink.c_str(), in->symlink.length() + 1);
+ ::_encode(in->symlink, bl);
}
in->dirfragtree._encode(bl);
static const int WAIT_DENTRY = (1<<0); // wait for item to be in cache
static const int WAIT_COMPLETE = (1<<1); // wait for complete dir contents
static const int WAIT_FREEZEABLE = (1<<2); // hard_pins removed
- static const int WAIT_UNFREEZE = (1<<3); // unfreeze
- static const int WAIT_AUTHPINNABLE = WAIT_UNFREEZE;
- static const int WAIT_IMPORTED = (1<<4); // import finish
- //static const int WAIT_SINGLEAUTH = (1<<5);
+ static const int WAIT_UNFREEZE = WAIT_AUTHPINNABLE; // unfreeze
+ static const int WAIT_IMPORTED = (1<<3); // import finish
- static const int WAIT_DNLOCK_OFFSET = 6;
+ static const int WAIT_DNLOCK_OFFSET = 4;
static const int WAIT_ANY = (0xffffffff);
static const int WAIT_ATFREEZEROOT = (WAIT_AUTHPINNABLE|WAIT_UNFREEZE);
return dirfrag() < ((const CDir*)r)->dirfrag();
}
- protected:
+protected:
// contents
CDir_map_t items; // non-null AND null
size_t nitems; // # non-null
size_t nnull; // # null
+ int num_dirty;
+
// state
version_t version;
version_t committing_version;
version_t committed_version;
+ version_t committed_version_equivalent; // in case of, e.g., temporary file
version_t projected_version;
// lock nesting, freeze
}
size_t get_nitems() { return nitems; }
size_t get_nnull() { return nnull; }
-
- /*
- float get_popularity() {
- return popularity[0].get();
- }
- */
+ void inc_num_dirty() { num_dirty++; }
+ void dec_num_dirty() {
+ assert(num_dirty > 0);
+ num_dirty--;
+ }
+ int get_num_dirty() {
+ return num_dirty;
+ }
+
+ void try_remove_unlinked_dn(CDentry *dn);
// -- dentries and inodes --
public:
return iter->second;
}
- CDentry* add_dentry( const string& dname, CInode *in=0, bool auth=true );
- CDentry* add_dentry( const string& dname, inodeno_t ino, bool auth=true );
+ CDentry* add_dentry( const string& dname, CInode *in=0 );
+ CDentry* add_dentry( const string& dname, inodeno_t ino );
void remove_dentry( CDentry *dn ); // delete dentry
void link_inode( CDentry *dn, inodeno_t ino );
void link_inode( CDentry *dn, CInode *in );
version_t get_projected_version() { return projected_version; }
version_t get_committing_version() { return committing_version; }
version_t get_committed_version() { return committed_version; }
+ version_t get_committed_version_equivalent() { return committed_version_equivalent; }
void set_committed_version(version_t v) { committed_version = v; }
version_t pre_dirty(version_t min=0);
+ ostream& print_db_line_prefix(ostream& out);
void print(ostream& out);
};
#include "messages/MLock.h"
#include <string>
-#include <sstream>
+#include <stdio.h>
#include "config.h"
#undef dout
}
+inode_t *CInode::project_inode()
+{
+ if (projected_inode.empty()) {
+ projected_inode.push_back(new inode_t(inode));
+ } else {
+ projected_inode.push_back(new inode_t(*projected_inode.back()));
+ }
+ dout(15) << "project_inode " << projected_inode.back() << endl;
+ return projected_inode.back();
+}
+
+void CInode::pop_and_dirty_projected_inode()
+{
+ assert(!projected_inode.empty());
+ dout(15) << "pop_and_dirty_projected_inode " << projected_inode.front()
+ << " v" << projected_inode.front()->version << endl;
+ mark_dirty(projected_inode.front()->version);
+ inode = *projected_inode.front();
+ delete projected_inode.front();
+ projected_inode.pop_front();
+}
+
+
// ====== CInode =======
// dirfrags
ls.push_back(p->second);
}
+
+CDir *CInode::get_or_open_dirfrag(MDCache *mdcache, frag_t fg)
+{
+ assert(is_dir());
+
+ // have it?
+ CDir *dir = get_dirfrag(fg);
+ if (dir) return dir;
+
+ // create it.
+ assert(is_auth());
+ dir = dirfrags[fg] = new CDir(this, fg, mdcache, true);
+ return dir;
+}
+
+CDir *CInode::add_dirfrag(CDir *dir)
+{
+ assert(dirfrags.count(dir->dirfrag().frag) == 0);
+ dirfrags[dir->dirfrag().frag] = dir;
+ return dir;
+}
+
+void CInode::close_dirfrag(frag_t fg)
+{
+ dout(14) << "close_dirfrag " << fg << endl;
+ assert(dirfrags.count(fg));
+
+ CDir *dir = dirfrags[fg];
+ dir->remove_null_dentries();
+
+ // clear dirty flag
+ if (dir->is_dirty())
+ dir->mark_clean();
+
+ // dump any remaining dentries, for debugging purposes
+ for (map<string,CDentry*>::iterator p = dir->items.begin();
+ p != dir->items.end();
+ ++p)
+ dout(14) << "close_dirfrag LEFTOVER dn " << *p->second << endl;
+
+ assert(dir->get_num_ref() == 0);
+ delete dir;
+ dirfrags.erase(fg);
+}
+
+void CInode::close_dirfrags()
+{
+ while (!dirfrags.empty())
+ close_dirfrag(dirfrags.begin()->first);
+}
+
+bool CInode::has_subtree_root_dirfrag()
+{
+ for (map<frag_t,CDir*>::iterator p = dirfrags.begin();
+ p != dirfrags.end();
+ ++p)
+ if (p->second->is_subtree_root())
+ return true;
+ return false;
+}
+
+
+
+
// pins
void CInode::first_get()
return NULL;
}
-CDir *CInode::get_or_open_dirfrag(MDCache *mdcache, frag_t fg)
-{
- assert(is_dir());
-
- // have it?
- CDir *dir = get_dirfrag(fg);
- if (dir) return dir;
-
- // create it.
- assert(is_auth());
- dir = dirfrags[fg] = new CDir(this, fg, mdcache, true);
- return dir;
-}
-
-CDir *CInode::add_dirfrag(CDir *dir)
-{
- assert(dirfrags.count(dir->dirfrag().frag) == 0);
- dirfrags[dir->dirfrag().frag] = dir;
- return dir;
-}
-
-void CInode::close_dirfrag(frag_t fg)
-{
- dout(14) << "close_dirfrag " << fg << endl;
- assert(dirfrags.count(fg));
-
- CDir *dir = dirfrags[fg];
- dir->remove_null_dentries();
-
- // clear dirty flag
- if (dir->is_dirty())
- dir->mark_clean();
-
- // dump any remaining dentries, for debugging purposes
- for (map<string,CDentry*>::iterator p = dir->items.begin();
- p != dir->items.end();
- ++p)
- dout(14) << "close_dirfrag LEFTOVER dn " << *p->second << endl;
-
- assert(dir->get_num_ref() == 0);
- delete dir;
- dirfrags.erase(fg);
-}
-
-void CInode::close_dirfrags()
-{
- while (!dirfrags.empty())
- close_dirfrag(dirfrags.begin()->first);
-}
-
void CInode::make_path(string& s)
s = ""; // root
}
else if (is_stray()) {
- s = "~";
+ s = "~stray";
+ char n[10];
+ sprintf(n, "%d", (int)(ino()-MDS_INO_STRAY_OFFSET));
+ s += n;
}
else {
s = "(dangling)"; // dangling
dout(10) << "make_anchor_trace added " << trace.back() << endl;
}
else
- assert(is_root());
+ assert(is_root() || is_stray());
}
void CInode::name_stray_dentry(string& dname)
{
- stringstream ss;
- ss << inode.ino;
- ss >> dname;
+ char s[20];
+ sprintf(s, "%ld", inode.ino.val);
+ dname = s;
}
// ------------------
// locking
-void CInode::set_mlock_info(MLock *m)
+void CInode::set_object_info(MDSCacheObjectInfo &info)
{
- m->set_ino(ino());
+ info.ino = ino();
}
void CInode::encode_lock_state(int type, bufferlist& bl)
{
switch (type) {
case LOCK_OTYPE_IAUTH:
- ::_encode(inode.ctime, bl);
- ::_encode(inode.mode, bl);
- ::_encode(inode.uid, bl);
- ::_encode(inode.gid, bl);
+ _encode(inode.ctime, bl);
+ _encode(inode.mode, bl);
+ _encode(inode.uid, bl);
+ _encode(inode.gid, bl);
break;
case LOCK_OTYPE_ILINK:
- ::_encode(inode.ctime, bl);
- ::_encode(inode.nlink, bl);
- ::_encode(inode.anchored, bl);
+ _encode(inode.ctime, bl);
+ _encode(inode.nlink, bl);
+ _encode(inode.anchored, bl);
break;
case LOCK_OTYPE_IDIRFRAGTREE:
break;
case LOCK_OTYPE_IFILE:
- ::_encode(inode.size, bl);
- ::_encode(inode.mtime, bl);
- ::_encode(inode.atime, bl);
+ _encode(inode.size, bl);
+ _encode(inode.mtime, bl);
+ _encode(inode.atime, bl);
break;
case LOCK_OTYPE_IDIR:
- ::_encode(inode.mtime, bl);
- {
+ _encode(inode.mtime, bl);
+ if (0) {
map<frag_t,int> dfsz;
for (map<frag_t,CDir*>::iterator p = dirfrags.begin();
p != dirfrags.end();
++p)
if (p->second->is_auth())
dfsz[p->first] = p->second->get_nitems();
- ::_encode(dfsz, bl);
+ _encode(dfsz, bl);
}
break;
switch (type) {
case LOCK_OTYPE_IAUTH:
- ::_decode(tm, bl, off);
+ _decode(tm, bl, off);
if (inode.ctime < tm) inode.ctime = tm;
- ::_decode(inode.mode, bl, off);
- ::_decode(inode.uid, bl, off);
- ::_decode(inode.gid, bl, off);
+ _decode(inode.mode, bl, off);
+ _decode(inode.uid, bl, off);
+ _decode(inode.gid, bl, off);
break;
case LOCK_OTYPE_ILINK:
- ::_decode(tm, bl, off);
+ _decode(tm, bl, off);
if (inode.ctime < tm) inode.ctime = tm;
- ::_decode(inode.nlink, bl, off);
- ::_decode(inode.anchored, bl, off);
+ _decode(inode.nlink, bl, off);
+ _decode(inode.anchored, bl, off);
break;
case LOCK_OTYPE_IDIRFRAGTREE:
break;
case LOCK_OTYPE_IFILE:
- ::_decode(inode.size, bl, off);
- ::_decode(inode.mtime, bl, off);
- ::_decode(inode.atime, bl, off);
+ _decode(inode.size, bl, off);
+ _decode(inode.mtime, bl, off);
+ _decode(inode.atime, bl, off);
break;
case LOCK_OTYPE_IDIR:
//::_decode(inode.size, bl, off);
- ::_decode(tm, bl, off);
- if (inode.mtime < tm) inode.mtime = tm;
- {
+ _decode(tm, bl, off);
+ if (inode.mtime < tm) {
+ inode.mtime = tm;
+ dirlock.set_updated();
+ }
+ if (0) {
map<frag_t,int> dfsz;
::_decode(dfsz, bl, off);
// hmm which to keep?
void CInode::add_waiter(int tag, Context *c)
{
// wait on the directory?
- if (tag & WAIT_AUTHPINNABLE) {
- parent->dir->add_waiter(CDir::WAIT_AUTHPINNABLE, c);
- return;
- }
- if (tag & WAIT_SINGLEAUTH) {
- parent->dir->add_waiter(CDir::WAIT_SINGLEAUTH, c);
+ if (tag & (WAIT_AUTHPINNABLE|WAIT_SINGLEAUTH)) {
+ parent->dir->add_waiter(tag, c);
return;
}
MDSCacheObject::add_waiter(tag, c);
pair<int,int> CInode::authority()
{
- if (is_root())
- return CDIR_AUTH_ROOTINODE; // root _inode_ is locked to mds0.
+ //if (is_root())
+ //return CDIR_AUTH_ROOTINODE; // root _inode_ is locked to mds0.
+ if (force_auth.first >= 0)
+ return force_auth;
if (parent)
return parent->dir->authority();
#include "SimpleLock.h"
#include "FileLock.h"
#include "ScatterLock.h"
+#include "LocalLock.h"
#include "Capability.h"
static const int PIN_OPENINGDIR = 14;
static const int PIN_REMOTEPARENT = 15;
static const int PIN_BATCHOPENJOURNAL = 16;
+ static const int PIN_SCATTERED = 17;
const char *pin_name(int p) {
switch (p) {
case PIN_OPENINGDIR: return "openingdir";
case PIN_REMOTEPARENT: return "remoteparent";
case PIN_BATCHOPENJOURNAL: return "batchopenjournal";
+ case PIN_SCATTERED: return "scattered";
default: return generic_pin_name(p);
}
}
// -- state --
- static const int STATE_ROOT = (1<<2);
- //static const int STATE_DANGLING = (1<<4); // delete me when i expire; i have no dentry
- static const int STATE_EXPORTING = (1<<6); // on nonauth bystander.
- static const int STATE_ANCHORING = (1<<7);
- static const int STATE_UNANCHORING = (1<<8);
- static const int STATE_OPENINGDIR = (1<<9);
+ static const int STATE_ROOT = (1<<1);
+ static const int STATE_EXPORTING = (1<<2); // on nonauth bystander.
+ static const int STATE_ANCHORING = (1<<3);
+ static const int STATE_UNANCHORING = (1<<4);
+ static const int STATE_OPENINGDIR = (1<<5);
+ static const int STATE_REJOINUNDEF = (1<<6); // inode contents undefined.
// -- waiters --
- static const int WAIT_SLAVEAGREE = (1<<0);
- static const int WAIT_AUTHPINNABLE = (1<<1);
- static const int WAIT_DIR = (1<<2);
- static const int WAIT_ANCHORED = (1<<3);
- static const int WAIT_UNANCHORED = (1<<4);
- static const int WAIT_CAPS = (1<<5);
+ //static const int WAIT_SLAVEAGREE = (1<<0);
+ static const int WAIT_DIR = (1<<1);
+ static const int WAIT_ANCHORED = (1<<2);
+ static const int WAIT_UNANCHORED = (1<<3);
+ static const int WAIT_CAPS = (1<<4);
- static const int WAIT_AUTHLOCK_OFFSET = 6;
- static const int WAIT_LINKLOCK_OFFSET = 6 + SimpleLock::WAIT_BITS;
- static const int WAIT_DIRFRAGTREELOCK_OFFSET = 6 + 2*SimpleLock::WAIT_BITS;
- static const int WAIT_FILELOCK_OFFSET = 6 + 3*SimpleLock::WAIT_BITS;
- static const int WAIT_DIRLOCK_OFFSET = 6 + 4*SimpleLock::WAIT_BITS;
+ static const int WAIT_AUTHLOCK_OFFSET = 5;
+ static const int WAIT_LINKLOCK_OFFSET = 5 + SimpleLock::WAIT_BITS;
+ static const int WAIT_DIRFRAGTREELOCK_OFFSET = 5 + 2*SimpleLock::WAIT_BITS;
+ static const int WAIT_FILELOCK_OFFSET = 5 + 3*SimpleLock::WAIT_BITS;
+ static const int WAIT_DIRLOCK_OFFSET = 5 + 4*SimpleLock::WAIT_BITS;
+ static const int WAIT_VERSIONLOCK_OFFSET = 5 + 5*SimpleLock::WAIT_BITS;
static const int WAIT_ANY = 0xffffffff;
off_t last_open_journaled; // log offset for the last journaled EOpen
+ // projected values (only defined while dirty)
+ list<inode_t*> projected_inode;
+ list<fragtree_t> projected_dirfragtree;
+
+ version_t get_projected_version() {
+ if (projected_inode.empty())
+ return inode.version;
+ else
+ return projected_inode.back()->version;
+ }
+
+ inode_t *project_inode();
+ void pop_and_dirty_projected_inode();
+
+
// -- cache infrastructure --
map<frag_t,CDir*> dirfrags; // cached dir fragments
frag_t pick_dirfrag(const string &dn);
+ bool has_dirfrags() { return !dirfrags.empty(); }
CDir* get_dirfrag(frag_t fg) {
if (dirfrags.count(fg))
return dirfrags[fg];
CDir *add_dirfrag(CDir *dir);
void close_dirfrag(frag_t fg);
void close_dirfrags();
+ bool has_subtree_root_dirfrag();
protected:
// parent dentries in cache
CDentry *parent; // primary link
set<CDentry*> remote_parents; // if hard linked
+ pair<int,int> force_auth;
// -- distributed state --
protected:
CInode(MDCache *c, bool auth=true) :
mdcache(c),
last_open_journaled(0),
- parent(0),
+ parent(0), force_auth(CDIR_AUTH_DEFAULT),
replica_caps_wanted(0),
auth_pins(0), nested_auth_pins(0),
+ versionlock(this, LOCK_OTYPE_IVERSION, WAIT_VERSIONLOCK_OFFSET),
authlock(this, LOCK_OTYPE_IAUTH, WAIT_AUTHLOCK_OFFSET),
linklock(this, LOCK_OTYPE_ILINK, WAIT_LINKLOCK_OFFSET),
dirfragtreelock(this, LOCK_OTYPE_IDIRFRAGTREE, WAIT_DIRFRAGTREELOCK_OFFSET),
void mark_clean();
-
-
CInodeDiscover* replicate_to(int rep);
// -- locks --
public:
+ LocalLock versionlock;
SimpleLock authlock;
SimpleLock linklock;
SimpleLock dirfragtreelock;
default: assert(0);
}
}
- void set_mlock_info(MLock *m);
+ void set_object_info(MDSCacheObjectInfo &info);
void encode_lock_state(int type, bufferlist& bl);
void decode_lock_state(int type, bufferlist& bl);
return &client_caps[client];
return 0;
}
+ void reconnect_cap(int client, inode_caps_reconnect_t& icr) {
+ Capability *cap = get_client_cap(client);
+ if (cap) {
+ cap->merge(icr.wanted, icr.issued);
+ } else {
+ Capability newcap(icr.wanted, 0);
+ newcap.issue(icr.issued);
+ add_client_cap(client, newcap);
+ }
+ inode.size = MAX(inode.size, icr.size);
+ inode.mtime = MAX(inode.mtime, icr.mtime);
+ inode.atime = MAX(inode.atime, icr.atime);
+ }
/*
void set_client_caps(map<int,Capability>& cl) {
if (client_caps.empty() && !cl.empty())
client_caps = cl;
}
*/
- void take_client_caps(map<int,Capability>& cl) {
+ void take_client_caps(map<int,Capability::Export>& cl) {
if (!client_caps.empty())
put(PIN_CAPS);
- cl = client_caps;
+ for (map<int,Capability>::iterator it = client_caps.begin();
+ it != client_caps.end();
+ it++) {
+ cl[it->first] = it->second.make_export();
+ }
client_caps.clear();
}
- void merge_client_caps(map<int,Capability>& cl, set<int>& new_client_caps) {
+ void merge_client_caps(map<int,Capability::Export>& cl, set<int>& new_client_caps) {
if (client_caps.empty() && !cl.empty())
get(PIN_CAPS);
- for (map<int,Capability>::iterator it = cl.begin();
+
+ for (map<int,Capability::Export>::iterator it = cl.begin();
it != cl.end();
it++) {
new_client_caps.insert(it->first);
client_caps[it->first].merge(it->second);
} else {
// new
- client_caps[it->first] = it->second;
+ client_caps[it->first] = Capability(it->second);
}
}
}
// -- reference counting --
-
- /* these can be pinned any # of times, and are
- linked to an active_request, so they're automatically cleaned
- up when a request is finished. pin at will! */
- void request_pin_get() {
- get(PIN_REQUEST);
- }
- void request_pin_put() {
- put(PIN_REQUEST);
- }
-
void bad_put(int by) {
dout(7) << " bad put " << *this << " by " << by << " " << pin_name(by) << " was " << ref << " (" << ref_set << ")" << endl;
assert(ref_set.count(by) == 1);
// -- hierarchy stuff --
-private:
- //void get_parent();
- //void put_parent();
-
public:
void set_primary_parent(CDentry *p) {
assert(parent == 0);
in->inode = inode;
in->symlink = symlink;
in->dirfragtree = dirfragtree;
-
in->replica_nonce = replica_nonce;
+ }
+ void init_inode_locks(CInode *in) {
in->authlock.set_state(authlock_state);
in->linklock.set_state(linklock_state);
in->dirfragtreelock.set_state(dirfragtreelock_state);
fragtree_t dirfragtree;
map<int,int> replicas;
- map<int,Capability> cap_map;
+ map<int,Capability::Export> cap_map;
bufferlist locks;
void update_inode(CInode *in, set<int>& new_client_caps) {
// treat scatterlocked mtime special, since replica may have newer info
if (in->dirlock.get_state() == LOCK_SCATTER ||
- in->dirlock.get_state() == LOCK_GSYNCS)
+ in->dirlock.get_state() == LOCK_GLOCKC ||
+ in->dirlock.get_state() == LOCK_GTEMPSYNCC)
st.inode.mtime = MAX(in->inode.mtime, st.inode.mtime);
in->inode = st.inode;
dirfragtree._encode(bl);
::_encode(replicas, bl);
::_encode(locks, bl);
-
- // caps
- for (map<int,Capability>::iterator it = cap_map.begin();
- it != cap_map.end();
- it++) {
- bl.append((char*)&it->first, sizeof(it->first));
- it->second._encode(bl);
- }
+ ::_encode(cap_map, bl);
}
int _decode(bufferlist& bl, int off = 0) {
dirfragtree._decode(bl, off);
::_decode(replicas, bl, off);
::_decode(locks, bl, off);
-
- // caps
- for (int i=0; i<st.num_caps; i++) {
- int c;
- bl.copy(off, sizeof(c), (char*)&c);
- off += sizeof(c);
- cap_map[c]._decode(bl, off);
- }
+ ::_decode(cap_map, bl, off);
return off;
}
return s;
}
+typedef uint32_t capseq_t;
class Capability {
+public:
+ struct Export {
+ int wanted;
+ int issued;
+ int pending;
+ Export() {}
+ Export(int w, int i, int p) : wanted(w), issued(i), pending(p) {}
+ };
+
+private:
int wanted_caps; // what the client wants (ideally)
-
- map<long, int> cap_history; // seq -> cap
- long last_sent, last_recv;
-
+
+ map<capseq_t, int> cap_history; // seq -> cap
+ capseq_t last_sent, last_recv;
+
bool suppress;
public:
- Capability(int want=0, long s=0) :
+ Capability(int want=0, capseq_t s=0) :
wanted_caps(want),
last_sent(s),
last_recv(s),
suppress(false) {
//cap_history[last_sent] = 0;
}
-
+ Capability(Export& other) :
+ wanted_caps(other.wanted),
+ last_sent(0), last_recv(0) {
+ // issued vs pending
+ if (other.issued & ~other.pending)
+ issue(other.issued);
+ issue(other.pending);
+ }
bool is_suppress() { return suppress; }
void set_suppress(bool b) { suppress = b; }
bool is_null() { return cap_history.empty() && wanted_caps == 0; }
// most recently issued caps.
- int pending() {
+ int pending() {
if (cap_history.count(last_sent))
return cap_history[ last_sent ];
return 0;
// caps potentially issued
int issued() {
int c = 0;
- for (long seq = last_recv; seq <= last_sent; seq++) {
+ for (capseq_t seq = last_recv; seq <= last_sent; seq++) {
if (cap_history.count(seq)) {
c |= cap_history[seq];
dout(10) << " cap issued: seq " << seq << " " << cap_string(cap_history[seq]) << " -> " << cap_string(c) << endl;
int issued_conflicts() { return conflicts(issued()); }
// issue caps; return seq number.
- long issue(int c) {
+ capseq_t issue(int c) {
//int was = pending();
//no! if (c == was && last_sent) return -1; // repeat of previous?
*/
return last_sent;
}
- long get_last_seq() { return last_sent; }
+ capseq_t get_last_seq() { return last_sent; }
- void merge(Capability& other) {
+ Export make_export() {
+ return Export(wanted_caps, issued(), pending());
+ }
+ void merge(Export& other) {
+ // issued + pending
+ int newpending = other.pending | pending();
+ if (other.issued & ~newpending)
+ issue(other.issued | newpending);
+ issue(newpending);
+
+ // wanted
+ wanted_caps = wanted_caps | other.wanted;
+ }
+ void merge(int otherwanted, int otherissued) {
// issued + pending
- int newpending = other.pending() | pending();
- if (other.issued() & ~newpending)
- issue(other.issued() | newpending);
+ int newpending = pending();
+ if (otherissued & ~newpending)
+ issue(otherissued | newpending);
issue(newpending);
// wanted
- wanted_caps = wanted_caps | other.wanted();
+ wanted_caps = wanted_caps | otherwanted;
}
// confirm receipt of a previous sent/issued seq.
- int confirm_receipt(long seq, int caps) {
+ int confirm_receipt(capseq_t seq, int caps) {
int r = 0;
// old seqs
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation. See file COPYING.
+ *
+ */
+
+
+
+#define DBLEVEL 20
+
+#include "include/types.h"
+
+#include "MDS.h"
+#include "ClientMap.h"
+
+#include "osdc/Filer.h"
+
+#include "config.h"
+#undef dout
+#define dout(x) if (x <= g_conf.debug_mds) cout << g_clock.now() << " mds" << mds->get_nodeid() << ".clientmap "
+
+
+
+void ClientMap::init_inode()
+{
+ memset(&inode, 0, sizeof(inode));
+ inode.ino = MDS_INO_CLIENTMAP_OFFSET + mds->get_nodeid();
+ inode.layout = g_OSD_FileLayout;
+}
+
+
+// ----------------
+// LOAD
+
+class C_CM_Load : public Context {
+ ClientMap *clientmap;
+public:
+ bufferlist bl;
+ C_CM_Load(ClientMap *cm) : clientmap(cm) {}
+ void finish(int r) {
+ clientmap->_load_finish(bl);
+ }
+};
+
+void ClientMap::load(Context *onload)
+{
+ dout(10) << "load" << endl;
+
+ init_inode();
+
+ if (onload)
+ waiting_for_load.push_back(onload);
+
+ C_CM_Load *c = new C_CM_Load(this);
+ mds->filer->read(inode,
+ 0, inode.layout.stripe_unit,
+ &c->bl,
+ c);
+
+}
+
+void ClientMap::_load_finish(bufferlist &bl)
+{
+ int off = 0;
+ decode(bl, off);
+ dout(10) << "_load_finish v " << version
+ << ", " << client_inst.size() << " clients, "
+ << bl.length() << " bytes"
+ << endl;
+ projected = committing = committed = version;
+ finish_contexts(waiting_for_load);
+}
+
+
+// ----------------
+// SAVE
+
+class C_CM_Save : public Context {
+ ClientMap *clientmap;
+ version_t version;
+public:
+ C_CM_Save(ClientMap *cm, version_t v) : clientmap(cm), version(v) {}
+ void finish(int r) {
+ clientmap->_save_finish(version);
+ }
+};
+
+void ClientMap::save(Context *onsave, version_t needv)
+{
+ dout(10) << "save needv " << needv << ", v " << version << endl;
+ commit_waiters[version].push_back(onsave);
+
+ if (needv && committing >= needv) return;
+
+ bufferlist bl;
+
+ init_inode();
+ encode(bl);
+ committing = version;
+ mds->filer->write(inode,
+ 0, bl.length(), bl,
+ 0,
+ 0, new C_CM_Save(this, version));
+}
+
+void ClientMap::_save_finish(version_t v)
+{
+ dout(10) << "_save_finish v" << v << endl;
+ committed = v;
+
+ finish_contexts(commit_waiters[v]);
+ commit_waiters.erase(v);
+}
#include <ext/hash_map>
using namespace __gnu_cxx;
+#include "include/Context.h"
+#include "mdstypes.h"
+
+class MDS;
/*
* this structure is used by the MDS purely so that
*/
class ClientMap {
private:
+ MDS *mds;
+
version_t version;
version_t projected;
version_t committing;
map<version_t, list<Context*> > commit_waiters;
public:
- ClientMap() : version(0), projected(0), committing(0), committed(0) {}
+ ClientMap(MDS *m) : mds(m),
+ version(0), projected(0), committing(0), committed(0) {}
version_t get_version() { return version; }
version_t get_projected() { return projected; }
void set_committing(version_t v) { committing = v; }
void set_committed(version_t v) { committed = v; }
- void add_commit_waiter(Context *c) {
- commit_waiters[committing].push_back(c);
- }
- void take_commit_waiters(version_t v, list<Context*>& ls) {
- ls.swap(commit_waiters[v]);
- commit_waiters.erase(v);
- }
-
private:
- // effects version
+ // affects version
hash_map<int,entity_inst_t> client_inst;
+
+ // does not affect version
set<int> sessions;
set<int> opening;
set<int> closing;
version++;
}
+private:
+ // -- push sequence --
+ hash_map<int,version_t> client_push_seq; // seq # for messages pushed to client.
+
+public:
+ version_t inc_push_seq(int client) {
+ return ++client_push_seq[client];
+ }
+ version_t get_push_seq(int client) {
+ return client_push_seq[client];
+ }
+
+
private:
// -- completed requests --
// client id -> tid -> result code
projected = committing = committed = version;
}
+
+ // -- loading, saving --
+ inode_t inode;
+ list<Context*> waiting_for_load;
+
+ void init_inode();
+ void load(Context *onload);
+ void _load_finish(bufferlist &bl);
+ void save(Context *onsave, version_t needv=0);
+ void _save_finish(version_t v);
};
#endif
}
return 0;
}
-
+ void export_twiddle() {
+ clear_gather();
+ state = get_replica_state();
+ }
// read/write access
bool can_rdlock(MDRequest *mdr) {
- if (!parent->is_auth())
- return (state == LOCK_SYNC);
- if (state == LOCK_LOCK && mdr && xlock_by == mdr)
- return true;
- if (state == LOCK_LOCK && !xlock_by)
- return true;
- return (state == LOCK_SYNC) || (state == LOCK_GMIXEDR)
- || (state == LOCK_GLOCKR);
+ if (!parent->is_auth()) return (state == LOCK_SYNC);
+ //if (state == LOCK_LOCK && mdr && xlock_by == mdr) return true;
+ if (state == LOCK_LOCK && !xlock_by) return true;
+ return
+ (state == LOCK_SYNC) ||
+ (state == LOCK_GMIXEDR) ||
+ (state == LOCK_GLOCKR);
}
bool can_rdlock_soon() {
if (parent->is_auth())
#include "IdAllocator.h"
#include "MDS.h"
#include "MDLog.h"
-#include "events/EAlloc.h"
#include "osdc/Filer.h"
}
-idno_t IdAllocator::alloc_id(bool replay)
+inodeno_t IdAllocator::alloc_id()
{
assert(is_active());
// pick one
- idno_t id = free.start();
+ inodeno_t id = free.start();
free.erase(id);
dout(10) << "idalloc " << this << ": alloc id " << id << endl;
version++;
// log it
+ /*
if (!replay)
mds->mdlog->submit_entry(new EAlloc(IDTYPE_INO, id, EALLOC_EV_ALLOC, version));
-
+ */
+
return id;
}
-void IdAllocator::reclaim_id(idno_t id, bool replay)
+void IdAllocator::reclaim_id(inodeno_t id)
{
assert(is_active());
version++;
+ /*
if (!replay)
mds->mdlog->submit_entry(new EAlloc(IDTYPE_INO, id, EALLOC_EV_FREE, version));
+ */
}
{
init_inode();
+ // use generic range. FIXME THIS IS CRAP
free.clear();
-
- // use generic range FIXME THIS IS CRAP
- free.insert((long long)0x1000000 * (long long)(mds->get_nodeid()+1),
- (long long)0x1000000 * (long long)(mds->get_nodeid()+2) - 1LL);
- //free[ID_INO].dump();
-
- //free[ID_FH].map_insert(10000000LL * (mds->get_nodeid()+1),
- //10000000LL * (mds->get_nodeid()+2) - 1);
- //free[ID_FH].dump();
+ free.insert((uint64_t)0x10000000000 * (uint64_t)(mds->get_nodeid()+1),
+ (uint64_t)0x10000000000 * (uint64_t)(mds->get_nodeid()+2) - (uint64_t)1);
state = STATE_ACTIVE;
}
class MDS;
-#define IDTYPE_INO 1
-typedef inodeno_t idno_t;
-
class IdAllocator {
MDS *mds;
inode_t inode;
version_t version, committing_version, committed_version;
- interval_set<idno_t> free; // unused ids
+ interval_set<inodeno_t> free; // unused ids
map<version_t, list<Context*> > waitfor_save;
void init_inode();
// alloc or reclaim ids
- idno_t alloc_id(bool replay=false);
- void reclaim_id(idno_t id, bool replay=false);
+ inodeno_t alloc_id();
+ void reclaim_id(inodeno_t ino);
version_t get_version() { return version; }
version_t get_committed_version() { return committed_version; }
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation. See file COPYING.
+ *
+ */
+
+
+#ifndef __LOCALLOCK_H
+#define __LOCALLOCK_H
+
+#include "SimpleLock.h"
+
+class LocalLock : public SimpleLock {
+protected:
+ int num_wrlock;
+
+public:
+ LocalLock(MDSCacheObject *o, int t, int wo) :
+ SimpleLock(o, t, wo),
+ num_wrlock(0) {
+ set_state(LOCK_LOCK); // always.
+ }
+
+ bool can_wrlock() {
+ return !is_xlocked();
+ }
+ void get_wrlock() {
+ assert(can_wrlock());
+ if (num_wrlock == 0) parent->get(MDSCacheObject::PIN_LOCK);
+ ++num_wrlock;
+ }
+ void put_wrlock() {
+ --num_wrlock;
+ if (num_wrlock == 0) parent->put(MDSCacheObject::PIN_LOCK);
+ }
+ bool is_wrlocked() { return num_wrlock > 0; }
+ int get_num_wrlocks() { return num_wrlock; }
+
+
+ void print(ostream& out) {
+ out << "(";
+ out << get_lock_type_name(get_type());
+ if (is_xlocked())
+ out << " x=" << get_xlocked_by();
+ if (is_wrlocked())
+ out << " wr=" << get_num_wrlocks();
+ out << ")";
+ }
+
+};
+
+
+#endif
#include "messages/MInodeFileCaps.h"
-#include "messages/MInodeLink.h"
-#include "messages/MInodeLinkAck.h"
-#include "messages/MInodeUnlink.h"
-#include "messages/MInodeUnlinkAck.h"
-
#include "messages/MLock.h"
#include "messages/MDentryUnlink.h"
#include "messages/MClientRequest.h"
#include "messages/MClientFileCaps.h"
+#include "messages/MMDSSlaveRequest.h"
+
#include <errno.h>
#include <assert.h>
set<SimpleLock*> &wrlocks,
set<SimpleLock*> &xlocks)
{
+ if (mdr->done_locking) {
+ dout(10) << "acquire_locks " << *mdr << " -- done locking" << endl;
+ return true; // at least we had better be!
+ }
dout(10) << "acquire_locks " << *mdr << endl;
- // sort everything we will lock
- set<SimpleLock*, SimpleLock::ptr_lt> sorted;
+ set<SimpleLock*, SimpleLock::ptr_lt> sorted; // sort everything we will lock
+ set<SimpleLock*> mustpin = xlocks; // items to authpin
+
+ // xlocks
+ for (set<SimpleLock*>::iterator p = xlocks.begin(); p != xlocks.end(); ++p) {
+ dout(20) << " must xlock " << **p << " " << *(*p)->get_parent() << endl;
+ sorted.insert(*p);
+
+ // augment xlock with a versionlock?
+ if ((*p)->get_type() > LOCK_OTYPE_IVERSION) {
+ // inode version lock?
+ CInode *in = (CInode*)(*p)->get_parent();
+ if (mdr->is_master()) {
+ // master. wrlock versionlock so we can pipeline inode updates to journal.
+ wrlocks.insert(&in->versionlock);
+ } else {
+ // slave. exclusively lock the inode version (i.e. block other journal updates)
+ xlocks.insert(&in->versionlock);
+ sorted.insert(&in->versionlock);
+ }
+ }
+ }
+
+ // wrlocks
+ for (set<SimpleLock*>::iterator p = wrlocks.begin(); p != wrlocks.end(); ++p) {
+ dout(20) << " must wrlock " << **p << " " << *(*p)->get_parent() << endl;
+ sorted.insert(*p);
+ if ((*p)->get_parent()->is_auth())
+ mustpin.insert(*p);
+ else if ((*p)->get_type() == LOCK_OTYPE_IDIR &&
+ !(*p)->get_parent()->is_auth() && !((ScatterLock*)(*p))->can_wrlock()) { // we might have to request a scatter
+ dout(15) << " will also auth_pin " << *(*p)->get_parent() << " in case we need to request a scatter" << endl;
+ mustpin.insert(*p);
+ }
+ }
- // (local) AUTH PINS
+ // rdlocks
+ for (set<SimpleLock*>::iterator p = rdlocks.begin();
+ p != rdlocks.end();
+ ++p) {
+ dout(20) << " must rdlock " << **p << " " << *(*p)->get_parent() << endl;
+ sorted.insert(*p);
+ }
- // make list of items to authpin
- set<SimpleLock*> mustpin = xlocks;
- for (set<SimpleLock*>::iterator p = wrlocks.begin(); p != wrlocks.end(); ++p)
- mustpin.insert(*p);
+
+ // AUTH PINS
+ map<int, set<MDSCacheObject*> > mustpin_remote; // mds -> (object set)
// can i auth pin them all now?
for (set<SimpleLock*>::iterator p = mustpin.begin();
p != mustpin.end();
++p) {
- dout(10) << "must authpin " << **p << " " << *(*p)->get_parent() << endl;
+ MDSCacheObject *object = (*p)->get_parent();
- // sort in
- sorted.insert(*p);
+ dout(10) << " must authpin " << *object << endl;
- if ((*p)->get_type() == LOCK_OTYPE_DN) {
- CDir *dir = ((CDentry*)(*p)->get_parent())->dir;
- dout(10) << "might auth_pin " << *dir << endl;
-
- if (!dir->is_auth()) continue;
- if (!mdr->is_auth_pinned(dir) &&
- !dir->can_auth_pin()) {
+ if (mdr->is_auth_pinned(object))
+ continue;
+
+ if (!object->is_auth()) {
+ if (object->is_ambiguous_auth()) {
// wait
- dir->add_waiter(CDir::WAIT_AUTHPINNABLE, new C_MDS_RetryRequest(mdcache, mdr));
- mdcache->request_drop_locks(mdr);
- mdr->drop_auth_pins();
- return false;
- }
- } else {
- CInode *in = (CInode*)(*p)->get_parent();
- if (!in->is_auth()) continue;
- if (!mdr->is_auth_pinned(in) &&
- !in->can_auth_pin()) {
- in->add_waiter(CInode::WAIT_AUTHPINNABLE, new C_MDS_RetryRequest(mdcache, mdr));
- mdcache->request_drop_locks(mdr);
- mdr->drop_auth_pins();
+ dout(10) << " ambiguous auth, waiting to authpin " << *object << endl;
+ object->add_waiter(MDSCacheObject::WAIT_SINGLEAUTH, new C_MDS_RetryRequest(mdcache, mdr));
+ mds->locker->drop_locks(mdr);
+ mdr->drop_local_auth_pins();
return false;
}
+ mustpin_remote[object->authority().first].insert(object);
+ continue;
+ }
+ if (!object->can_auth_pin()) {
+ // wait
+ dout(10) << " can't auth_pin (freezing?), waiting to authpin " << *object << endl;
+ object->add_waiter(MDSCacheObject::WAIT_AUTHPINNABLE, new C_MDS_RetryRequest(mdcache, mdr));
+ mds->locker->drop_locks(mdr);
+ mdr->drop_local_auth_pins();
+ return false;
}
}
- // ok, grab the auth pins
+ // ok, grab local auth pins
for (set<SimpleLock*>::iterator p = mustpin.begin();
p != mustpin.end();
++p) {
- if ((*p)->get_type() == LOCK_OTYPE_DN) {
- CDir *dir = ((CDentry*)(*p)->get_parent())->dir;
- if (!dir->is_auth()) continue;
- dout(10) << "auth_pinning " << *dir << endl;
- mdr->auth_pin(dir);
- } else {
- CInode *in = (CInode*)(*p)->get_parent();
- if (!in->is_auth()) continue;
- dout(10) << "auth_pinning " << *in << endl;
- mdr->auth_pin(in);
+ MDSCacheObject *object = (*p)->get_parent();
+ if (mdr->is_auth_pinned(object)) {
+ dout(10) << " already auth_pinned " << *object << endl;
+ } else if (object->is_auth()) {
+ dout(10) << " auth_pinning " << *object << endl;
+ mdr->auth_pin(object);
}
}
- // sort in rdlocks too
- for (set<SimpleLock*>::iterator p = rdlocks.begin();
- p != rdlocks.end();
- ++p) {
- dout(10) << "will rdlock " << **p << " " << *(*p)->get_parent() << endl;
- sorted.insert(*p);
+ // request remote auth_pins
+ if (!mustpin_remote.empty()) {
+ for (map<int, set<MDSCacheObject*> >::iterator p = mustpin_remote.begin();
+ p != mustpin_remote.end();
+ ++p) {
+ dout(10) << "requesting remote auth_pins from mds" << p->first << endl;
+
+ MMDSSlaveRequest *req = new MMDSSlaveRequest(mdr->reqid, MMDSSlaveRequest::OP_AUTHPIN);
+ for (set<MDSCacheObject*>::iterator q = p->second.begin();
+ q != p->second.end();
+ ++q) {
+ dout(10) << " req remote auth_pin of " << **q << endl;
+ MDSCacheObjectInfo info;
+ (*q)->set_object_info(info);
+ req->get_authpins().push_back(info);
+ }
+ mds->send_message_mds(req, p->first, MDS_PORT_SERVER);
+
+ // put in waiting list
+ assert(mdr->waiting_on_slave.count(p->first) == 0);
+ mdr->waiting_on_slave.insert(p->first);
+ }
+ return false;
}
// acquire locks.
// already locked?
if (existing != mdr->locks.end() && *existing == *p) {
// right kind?
- SimpleLock *had = *existing;
- if (xlocks.count(*p) == mdr->xlocks.count(*p) &&
- wrlocks.count(*p) == mdr->wrlocks.count(*p) &&
- rdlocks.count(*p) == mdr->rdlocks.count(*p)) {
- dout(10) << "acquire_locks already locked " << *had << " " << *had->get_parent() << endl;
- existing++;
- continue;
+ SimpleLock *have = *existing;
+ existing++;
+ if (xlocks.count(*p) && mdr->xlocks.count(*p)) {
+ dout(10) << " already xlocked " << *have << " " << *have->get_parent() << endl;
}
+ else if (wrlocks.count(*p) && mdr->wrlocks.count(*p)) {
+ dout(10) << " already wrlocked " << *have << " " << *have->get_parent() << endl;
+ }
+ else if (rdlocks.count(*p) && mdr->rdlocks.count(*p)) {
+ dout(10) << " already rdlocked " << *have << " " << *have->get_parent() << endl;
+ }
+ else assert(0);
+ continue;
}
// hose any stray locks
while (existing != mdr->locks.end()) {
- SimpleLock *had = *existing;
+ SimpleLock *stray = *existing;
existing++;
- dout(10) << "acquire_locks unlocking out-of-order " << **existing
- << " " << *(*existing)->get_parent() << endl;
- if (mdr->xlocks.count(had))
- xlock_finish(had, mdr);
- else if (mdr->wrlocks.count(had))
- wrlock_finish(had, mdr);
+ dout(10) << " unlocking out-of-order " << *stray << " " << *stray->get_parent() << endl;
+ if (mdr->xlocks.count(stray))
+ xlock_finish(stray, mdr);
+ else if (mdr->wrlocks.count(stray))
+ wrlock_finish(stray, mdr);
else
- rdlock_finish(had, mdr);
+ rdlock_finish(stray, mdr);
}
// lock
if (xlocks.count(*p)) {
if (!xlock_start(*p, mdr))
return false;
- dout(10) << "acquire_locks got xlock on " << **p << " " << *(*p)->get_parent() << endl;
+ dout(10) << " got xlock on " << **p << " " << *(*p)->get_parent() << endl;
} else if (wrlocks.count(*p)) {
if (!wrlock_start(*p, mdr))
return false;
- dout(10) << "acquire_locks got wrlock on " << **p << " " << *(*p)->get_parent() << endl;
+ dout(10) << " got wrlock on " << **p << " " << *(*p)->get_parent() << endl;
} else {
if (!rdlock_start(*p, mdr))
return false;
- dout(10) << "acquire_locks got rdlock on " << **p << " " << *(*p)->get_parent() << endl;
+ dout(10) << " got rdlock on " << **p << " " << *(*p)->get_parent() << endl;
}
}
// any extra unneeded locks?
while (existing != mdr->locks.end()) {
- dout(10) << "acquire_locks unlocking " << *existing
- << " " << *(*existing)->get_parent() << endl;
- if (mdr->xlocks.count(*existing))
- xlock_finish(*existing, mdr);
- else if (mdr->wrlocks.count(*existing))
- wrlock_finish(*existing, mdr);
+ SimpleLock *stray = *existing;
+ existing++;
+ dout(10) << " unlocking extra " << *stray << " " << *stray->get_parent() << endl;
+ if (mdr->xlocks.count(stray))
+ xlock_finish(stray, mdr);
+ else if (mdr->wrlocks.count(stray))
+ wrlock_finish(stray, mdr);
else
- rdlock_finish(*existing, mdr);
+ rdlock_finish(stray, mdr);
}
return true;
}
+void Locker::drop_locks(MDRequest *mdr)
+{
+ // leftover locks
+ while (!mdr->xlocks.empty())
+ xlock_finish(*mdr->xlocks.begin(), mdr);
+ while (!mdr->rdlocks.empty())
+ rdlock_finish(*mdr->rdlocks.begin(), mdr);
+ while (!mdr->wrlocks.empty())
+ wrlock_finish(*mdr->wrlocks.begin(), mdr);
+}
+
// generics
switch (lock->get_type()) {
case LOCK_OTYPE_IDIR:
return scatter_wrlock_start((ScatterLock*)lock, mdr);
+ case LOCK_OTYPE_IVERSION:
+ return local_wrlock_start((LocalLock*)lock, mdr);
default:
assert(0);
}
switch (lock->get_type()) {
case LOCK_OTYPE_IDIR:
return scatter_wrlock_finish((ScatterLock*)lock, mdr);
+ case LOCK_OTYPE_IVERSION:
+ return local_wrlock_finish((LocalLock*)lock, mdr);
default:
assert(0);
}
switch (lock->get_type()) {
case LOCK_OTYPE_IFILE:
return file_xlock_start((FileLock*)lock, mdr);
+ case LOCK_OTYPE_IVERSION:
+ return local_xlock_start((LocalLock*)lock, mdr);
case LOCK_OTYPE_IDIR:
assert(0);
default:
switch (lock->get_type()) {
case LOCK_OTYPE_IFILE:
return file_xlock_finish((FileLock*)lock, mdr);
+ case LOCK_OTYPE_IVERSION:
+ return local_xlock_finish((LocalLock*)lock, mdr);
case LOCK_OTYPE_IDIR:
assert(0);
default:
+/** rejoin_set_state
+ * @lock the lock
+ * @s the new state
+ * @waiters list for anybody waiting on this lock
+ */
+void Locker::rejoin_set_state(SimpleLock *lock, int s, list<Context*>& waiters)
+{
+ if (!lock->is_stable()) {
+ lock->set_state(s);
+ lock->get_parent()->auth_unpin();
+ } else {
+ lock->set_state(s);
+ }
+ lock->take_waiting(SimpleLock::WAIT_ALL, waiters);
+}
+
if (in->is_auth()) {
// [auth] twiddle mode?
- file_eval(&in->filelock);
+ if (in->filelock.is_stable())
+ file_eval(&in->filelock);
} else {
// [replica] tell auth about any new caps wanted
request_inode_file_caps(in);
if (seq > 0 &&
!it->second.is_suppress()) {
dout(7) << " sending MClientFileCaps to client" << it->first << " seq " << it->second.get_last_seq() << " new pending " << cap_string(it->second.pending()) << " was " << cap_string(before) << endl;
- mds->messenger->send_message(new MClientFileCaps(in->inode,
- it->second.get_last_seq(),
- it->second.pending(),
- it->second.wanted()),
- mds->clientmap.get_inst(it->first),
- 0, MDS_PORT_LOCKER);
+ mds->send_message_client(new MClientFileCaps(MClientFileCaps::OP_GRANT,
+ in->inode,
+ it->second.get_last_seq(),
+ it->second.pending(),
+ it->second.wanted()),
+ it->first);
}
}
}
else
in->mds_caps_wanted.erase(m->get_from());
- file_eval(&in->filelock);
+ if (in->filelock.is_stable())
+ try_file_eval(&in->filelock); // ** may or may not be auth_pinned **
delete m;
}
request_inode_file_caps(in);
// tell client.
- MClientFileCaps *r = new MClientFileCaps(in->inode,
- 0, 0, 0,
- MClientFileCaps::OP_RELEASE);
- mds->messenger->send_message(r, m->get_source_inst(), 0, MDS_PORT_LOCKER);
+ MClientFileCaps *r = new MClientFileCaps(MClientFileCaps::OP_RELEASE,
+ in->inode,
+ 0, 0, 0);
+ mds->send_message_client(r, m->get_source_inst());
}
// merge in atime?
}
// reevaluate, waiters
- file_eval(&in->filelock);
- in->finish_waiting(CInode::WAIT_CAPS, 0);
+ if (!in->filelock.is_stable())
+ file_eval_gather(&in->filelock);
+ else
+ file_eval(&in->filelock);
+
+ //in->finish_waiting(CInode::WAIT_CAPS, 0); // note: any users for this?
delete m;
}
// locks ----------------------------------------------------------------
-/*
-
-
-INODES:
-
-= two types of inode metadata:
- hard - uid/gid, mode
- file - mtime, size
- ? atime - atime (*) <-- we want a lazy update strategy?
-
-= correspondingly, two types of inode locks:
- hardlock - hard metadata
- filelock - file metadata
-
- -> These locks are completely orthogonal!
-
-= metadata ops and how they affect inode metadata:
- sma=size mtime atime
- HARD FILE OP
- files:
- R RRR stat
- RW chmod/chown
- R W touch ?ctime
- R openr
- W read atime
- R openw
- Wc openwc ?ctime
- WW write size mtime
- close
-
- dirs:
- R W readdir atime
- RRR ( + implied stats on files)
- Rc WW mkdir (ctime on new dir, size+mtime on parent dir)
- R WW link/unlink/rename/rmdir (size+mtime on dir)
-
-
-
-= relationship to client (writers):
-
- - ops in question are
- - stat ... need reasonable value for mtime (+ atime?)
- - maybe we want a "quicksync" type operation instead of full lock
- - truncate ... need to stop writers for the atomic truncate operation
- - need a full lock
-
-
-
-
-= modes
- - SYNC
- Rauth Rreplica Wauth Wreplica
- sync
-
-
-
-
-
-ALSO:
-
- dirlock - no dir changes (prior to unhashing)
- denlock - dentry lock (prior to unlink, rename)
-
-
-*/
-
-
-void Locker::handle_lock(MLock *m)
+SimpleLock *Locker::get_lock(int lock_type, MDSCacheObjectInfo &info)
{
- // nobody should be talking to us during recovery.
- assert(mds->is_rejoin() || mds->is_active() || mds->is_stopping());
-
- switch (m->get_otype()) {
+ switch (lock_type) {
case LOCK_OTYPE_DN:
{
- CDir *dir = mdcache->get_dirfrag(m->get_dirfrag());
+ CDir *dir = mdcache->get_dirfrag(info.dirfrag);
CDentry *dn = 0;
if (dir)
- dn = dir->lookup(m->get_dn());
+ dn = dir->lookup(info.dname);
if (!dn) {
- dout(7) << "dont' have dn " << m->get_dirfrag() << " " << m->get_dn() << endl;
- delete m;
- return;
+ dout(7) << "get_lock don't have dn " << info.dirfrag << " " << info.dname << endl;
+ return 0;
}
-
- handle_simple_lock(&dn->lock, m);
+ return &dn->lock;
}
- break;
case LOCK_OTYPE_IAUTH:
case LOCK_OTYPE_ILINK:
case LOCK_OTYPE_IDIRFRAGTREE:
case LOCK_OTYPE_IFILE:
+ case LOCK_OTYPE_IDIR:
{
- CInode *in = mdcache->get_inode(m->get_ino());
+ CInode *in = mdcache->get_inode(info.ino);
if (!in) {
- dout(7) << "dont' have ino " << m->get_ino() << endl;
- delete m;
- return;
+ dout(7) << "get_lock don't have ino " << info.ino << endl;
+ return 0;
}
- switch (m->get_otype()) {
- case LOCK_OTYPE_IAUTH:
- handle_simple_lock(&in->authlock, m);
- break;
- case LOCK_OTYPE_ILINK:
- handle_simple_lock(&in->linklock, m);
- break;
- case LOCK_OTYPE_IDIRFRAGTREE:
- handle_simple_lock(&in->dirfragtreelock, m);
- break;
- case LOCK_OTYPE_IFILE:
- handle_file_lock(&in->filelock, m);
- break;
+ switch (lock_type) {
+ case LOCK_OTYPE_IAUTH: return &in->authlock;
+ case LOCK_OTYPE_ILINK: return &in->linklock;
+ case LOCK_OTYPE_IDIRFRAGTREE: return &in->dirfragtreelock;
+ case LOCK_OTYPE_IFILE: return &in->filelock;
+ case LOCK_OTYPE_IDIR: return &in->dirlock;
}
}
+
+ default:
+ dout(7) << "get_lock don't know lock_type " << lock_type << endl;
+ assert(0);
break;
+ }
+
+ return 0;
+}
+
+
+void Locker::handle_lock(MLock *m)
+{
+ // nobody should be talking to us during recovery.
+ assert(mds->is_rejoin() || mds->is_active() || mds->is_stopping());
+
+ SimpleLock *lock = get_lock(m->get_lock_type(), m->get_object_info());
+ if (!lock) {
+ dout(10) << "don't have object " << m->get_object_info() << ", must have trimmed, dropping" << endl;
+ delete m;
+ return;
+ }
+ switch (lock->get_type()) {
+ case LOCK_OTYPE_DN:
+ case LOCK_OTYPE_IAUTH:
+ case LOCK_OTYPE_ILINK:
+ case LOCK_OTYPE_IDIRFRAGTREE:
+ handle_simple_lock(lock, m);
+ break;
+
+ case LOCK_OTYPE_IFILE:
+ handle_file_lock((FileLock*)lock, m);
+ break;
+
case LOCK_OTYPE_IDIR:
- {
- CInode *in = mdcache->get_inode(m->get_ino());
- if (!in) {
- dout(7) << "dont' have ino " << m->get_ino() << endl;
- delete m;
- return;
- }
- handle_scatter_lock(&in->dirlock, m);
- }
+ handle_scatter_lock((ScatterLock*)lock, m);
break;
default:
- dout(7) << "handle_lock got otype " << m->get_otype() << endl;
+ dout(7) << "handle_lock got otype " << m->get_lock_type() << endl;
assert(0);
break;
}
lock->decode_locked_state(m->get_data());
lock->set_state(LOCK_SYNC);
lock->finish_waiters(SimpleLock::WAIT_RD|SimpleLock::WAIT_STABLE);
+
+ // special case: trim replica no-longer-null dentry?
+ if (lock->get_type() == LOCK_OTYPE_DN) {
+ CDentry *dn = (CDentry*)lock->get_parent();
+ if (dn->is_null() && m->get_data().length() > 0) {
+ dout(10) << "handle_simple_lock replica dentry null -> non-null, must trim "
+ << *dn << endl;
+ assert(dn->get_num_ref() == 0);
+ map<int, MCacheExpire*> expiremap;
+ mdcache->trim_dentry(dn, expiremap);
+ mdcache->send_expire_messages(expiremap);
+ }
+ }
break;
case LOCK_AC_LOCK:
break;
- case LOCK_AC_REQXLOCKACK:
- dout(7) << "handle_simple_lock got remote xlock on "
- << *lock << " " << *lock->get_parent() << endl;
- {
- MDRequest *mdr = mdcache->request_get(m->get_reqid());
- mdr->xlocks.insert(lock);
- mdr->locks.insert(lock);
- lock->set_state(LOCK_REMOTEXLOCK);
- lock->finish_waiters(SimpleLock::WAIT_REMOTEXLOCK);
- }
- break;
-
// -- auth --
case LOCK_AC_LOCKACK:
assert(lock->get_state() == LOCK_GLOCKR);
} else {
dout(7) << "handle_simple_lock " << *lock << " on " << *lock->get_parent() << " from " << from
<< ", last one" << endl;
- simple_eval(lock);
+ simple_eval_gather(lock);
}
break;
- case LOCK_AC_REQXLOCK:
- assert(lock->get_parent()->is_auth());
- {
- // register request
- MDRequest *mdr = mdcache->request_start(m);
-
- dout(7) << "handle_simple_lock " << m->get_source() << " " << *mdr << " requesting xlock "
- << *lock << " on " << *lock->get_parent()
- << endl;
-
- if (!simple_xlock_start(lock, mdr))
- return;
-
- // ack
- MLock *m = new MLock(lock, LOCK_AC_REQXLOCKACK, mds->get_nodeid());
- mds->send_message_mds(m, mdr->request->get_source().num(), MDS_PORT_LOCKER);
- }
- return;
-
- case LOCK_AC_UNXLOCK:
- assert(lock->get_parent()->is_auth());
- {
- // get request
- MDRequest *mdr = mdcache->request_get(m->get_reqid());
-
- dout(7) << "handle_simple_lock " << m->get_source() << " " << *mdr << " dropping xlock "
- << *lock << " on " << *lock->get_parent()
-
- << endl;
-
- simple_xlock_finish(lock, mdr);
-
- if (mdr->locks.empty())
- mdcache->request_finish(mdr);
-
- }
- return;
-
}
delete m;
}
+/* unused, currently.
class C_Locker_SimpleEval : public Context {
Locker *locker;
public:
C_Locker_SimpleEval(Locker *l, SimpleLock *lk) : locker(l), lock(lk) {}
void finish(int r) {
- locker->simple_eval(lock);
+ locker->try_simple_eval(lock);
}
};
-void Locker::simple_eval(SimpleLock *lock)
+void Locker::try_simple_eval(SimpleLock *lock)
{
// unstable and ambiguous auth?
if (!lock->is_stable() &&
return;
}
- // finished remote xlock?
- if (lock->get_state() == LOCK_REMOTEXLOCK &&
- !lock->is_xlocked()) {
- // tell auth
- assert(!lock->get_parent()->is_auth()); // should be auth_pinned on the auth
- dout(7) << "simple_eval releasing remote xlock on " << *lock->get_parent() << endl;
- int auth = lock->get_parent()->authority().first;
- if (mds->mdsmap->get_state(auth) >= MDSMap::STATE_REJOIN)
- mds->send_message_mds(new MLock(lock, LOCK_AC_UNXLOCK, mds->get_nodeid()),
- auth, MDS_PORT_LOCKER);
- lock->set_state(LOCK_LOCK);
+ if (!lock->get_parent()->is_auth()) {
+ dout(7) << "try_simple_eval not auth for " << *lock->get_parent() << endl;
+ return;
+ }
+
+ if (!lock->get_parent()->can_auth_pin()) {
+ dout(7) << "try_simple_eval can't auth_pin, waiting on " << *lock->get_parent() << endl;
+ //if (!lock->get_parent()->is_waiter(MDSCacheObject::WAIT_SINGLEAUTH))
+ lock->get_parent()->add_waiter(MDSCacheObject::WAIT_AUTHPINNABLE, new C_Locker_SimpleEval(this, lock));
+ return;
}
+ if (lock->is_stable())
+ simple_eval(lock);
+}
+*/
+
+void Locker::simple_eval_gather(SimpleLock *lock)
+{
+ dout(10) << "simple_eval_gather " << *lock << " on " << *lock->get_parent() << endl;
+
// finished gathering?
if (lock->get_state() == LOCK_GLOCKR &&
!lock->is_gathering() &&
lock->set_state(LOCK_LOCK);
lock->finish_waiters(SimpleLock::WAIT_STABLE|SimpleLock::WAIT_WR);
+
+ if (lock->get_parent()->is_auth()) {
+ lock->get_parent()->auth_unpin();
+
+ // re-eval?
+ simple_eval(lock);
+ }
}
+}
+
+void Locker::simple_eval(SimpleLock *lock)
+{
+ dout(10) << "simple_eval " << *lock << " on " << *lock->get_parent() << endl;
+
+ assert(lock->get_parent()->is_auth());
+ assert(lock->is_stable());
// stable -> sync?
- if (lock->get_parent()->is_auth() &&
- lock->is_stable() &&
+ if (!lock->is_xlocked() &&
lock->get_state() != LOCK_SYNC &&
!lock->is_waiter_for(SimpleLock::WAIT_WR)) {
dout(7) << "simple_eval stable, syncing " << *lock
{
dout(7) << "simple_sync on " << *lock << " on " << *lock->get_parent() << endl;
assert(lock->get_parent()->is_auth());
+ assert(lock->is_stable());
// check state
if (lock->get_state() == LOCK_SYNC)
return; // already sync
- if (lock->get_state() == LOCK_GLOCKR)
- assert(0); // um... hmm!
assert(lock->get_state() == LOCK_LOCK);
// sync.
{
dout(7) << "simple_lock on " << *lock << " on " << *lock->get_parent() << endl;
assert(lock->get_parent()->is_auth());
+ assert(lock->is_stable());
// check state
- if (lock->get_state() == LOCK_LOCK ||
- lock->get_state() == LOCK_GLOCKR)
- return; // already lock or locking
+ if (lock->get_state() == LOCK_LOCK) return;
assert(lock->get_state() == LOCK_SYNC);
if (lock->get_parent()->is_replicated()) {
// change lock
lock->set_state(LOCK_GLOCKR);
lock->init_gather();
+ lock->get_parent()->auth_pin();
} else {
lock->set_state(LOCK_LOCK);
}
// last one?
if (!lock->is_rdlocked())
- simple_eval(lock);
+ simple_eval_gather(lock);
}
bool Locker::simple_xlock_start(SimpleLock *lock, MDRequest *mdr)
}
} else {
// replica
-
+ // this had better not be a remote xlock attempt!
+ assert(!mdr->slave_request);
+
// wait for single auth
if (lock->get_parent()->is_ambiguous_auth()) {
lock->get_parent()->add_waiter(MDSCacheObject::WAIT_SINGLEAUTH,
new C_MDS_RetryRequest(mdcache, mdr));
return false;
}
- int auth = lock->get_parent()->authority().first;
-
- // wait for sync.
- // (???????????)
- if (lock->get_state() != LOCK_SYNC) {
- lock->add_waiter(SimpleLock::WAIT_RD, new C_MDS_RetryRequest(mdcache, mdr));
- return false;
- }
// send lock request
- MLock *m = new MLock(lock, LOCK_AC_REQXLOCK, mds->get_nodeid());
- mds->send_message_mds(m, auth, MDS_PORT_LOCKER);
-
+ int auth = lock->get_parent()->authority().first;
+ mdr->slaves.insert(auth);
+ MMDSSlaveRequest *r = new MMDSSlaveRequest(mdr->reqid, MMDSSlaveRequest::OP_XLOCK);
+ r->set_lock_type(lock->get_type());
+ lock->get_parent()->set_object_info(r->get_object_info());
+ mds->send_message_mds(r, auth, MDS_PORT_SERVER);
+
// wait
lock->add_waiter(SimpleLock::WAIT_REMOTEXLOCK, new C_MDS_RetryRequest(mdcache, mdr));
return false;
void Locker::simple_xlock_finish(SimpleLock *lock, MDRequest *mdr)
{
+ dout(7) << "simple_xlock_finish on " << *lock << " on " << *lock->get_parent() << endl;
+
// drop ref
assert(lock->can_xlock(mdr));
lock->put_xlock();
assert(mdr);
mdr->xlocks.erase(lock);
mdr->locks.erase(lock);
- dout(7) << "simple_xlock_finish on " << *lock << " on " << *lock->get_parent() << endl;
+
+ // remote xlock?
+ if (!lock->get_parent()->is_auth()) {
+ // tell auth
+ dout(7) << "simple_xlock_finish releasing remote xlock on " << *lock->get_parent() << endl;
+ int auth = lock->get_parent()->authority().first;
+ if (mds->mdsmap->get_state(auth) >= MDSMap::STATE_REJOIN) {
+ MMDSSlaveRequest *slavereq = new MMDSSlaveRequest(mdr->reqid, MMDSSlaveRequest::OP_UNXLOCK);
+ slavereq->set_lock_type(lock->get_type());
+ lock->get_parent()->set_object_info(slavereq->get_object_info());
+ mds->send_message_mds(slavereq, auth, MDS_PORT_SERVER);
+ }
+ }
// others waiting?
lock->finish_waiters(SimpleLock::WAIT_WR, 0);
- // eval
- simple_eval(lock);
+ // eval?
+ if (lock->get_parent()->is_auth())
+ simple_eval(lock);
}
* see if we can _anonymously_ rdlock an entire trace.
* if not, and req is specified, wait and retry that message.
*/
-bool Locker::dentry_can_rdlock_trace(vector<CDentry*>& trace, MClientRequest *req)
+bool Locker::dentry_can_rdlock_trace(vector<CDentry*>& trace)
{
// verify dentries are rdlockable.
// we do this because
it++) {
CDentry *dn = *it;
if (!dn->lock.can_rdlock(0)) {
- if (req) {
- dout(10) << "can_rdlock_trace can't rdlock " << *dn << ", waiting" << endl;
- dn->lock.add_waiter(SimpleLock::WAIT_RD, new C_MDS_RetryMessage(mds, req));
- } else {
- dout(10) << "can_rdlock_trace can't rdlock " << *dn << endl;
- }
+ dout(10) << "can_rdlock_trace can't rdlock " << *dn << endl;
return false;
}
}
// grab dentry rdlocks
for (vector<CDentry*>::iterator it = trace.begin();
it != trace.end();
- it++)
+ it++) {
+ dout(10) << "dentry_anon_rdlock_trace_start rdlocking " << (*it)->lock << " " << **it << endl;
(*it)->lock.get_rdlock();
+ }
}
dout(7) << "scatter_rdlock_start on " << *lock
<< " on " << *lock->get_parent() << endl;
+ // read on stable scattered replica?
+ if (lock->get_state() == LOCK_SCATTER &&
+ !lock->get_parent()->is_auth()) {
+ dout(7) << "scatter_rdlock_start scatterlock read on a stable scattered replica, fw to auth" << endl;
+ mdcache->request_forward(mdr, lock->get_parent()->authority().first);
+ return false;
+ }
+
// pre-twiddle?
if (lock->get_state() == LOCK_SCATTER &&
lock->get_parent()->is_auth() &&
}
// wait for read.
- lock->add_waiter(SimpleLock::WAIT_RD, new C_MDS_RetryRequest(mdcache, mdr));
+ lock->add_waiter(SimpleLock::WAIT_RD|SimpleLock::WAIT_STABLE, new C_MDS_RetryRequest(mdcache, mdr));
- // initiate sync?
- if (lock->get_state() == LOCK_SCATTER &&
- lock->get_parent()->is_auth())
- scatter_sync(lock);
+ // initiate sync or tempsync?
+ if (lock->is_stable() &&
+ lock->get_parent()->is_auth()) {
+ if (lock->get_parent()->is_replicated())
+ scatter_tempsync(lock);
+ else
+ scatter_sync(lock);
+ }
return false;
}
mdr->locks.erase(lock);
}
- scatter_eval(lock);
+ scatter_eval_gather(lock);
}
<< " on " << *lock->get_parent() << endl;
// pre-twiddle?
- if (lock->get_state() == LOCK_SYNC &&
- lock->get_parent()->is_auth() &&
+ if (lock->get_parent()->is_auth() &&
!lock->get_parent()->is_replicated() &&
- !lock->is_rdlocked())
- scatter_scatter(lock);
+ !lock->is_rdlocked() &&
+ !lock->is_xlocked() &&
+ lock->get_state() == LOCK_SYNC)
+ scatter_lock(lock);
// can wrlock?
if (lock->can_wrlock()) {
}
// wait for write.
- lock->add_waiter(SimpleLock::WAIT_WR, new C_MDS_RetryRequest(mdcache, mdr));
-
- // initiate scatter?
- if (lock->get_state() == LOCK_SYNC &&
- lock->get_parent()->is_auth())
- scatter_scatter(lock);
+ lock->add_waiter(SimpleLock::WAIT_WR|SimpleLock::WAIT_STABLE, new C_MDS_RetryRequest(mdcache, mdr));
+
+ // initiate scatter or lock?
+ if (lock->is_stable()) {
+ if (lock->get_parent()->is_auth()) {
+ // auth. scatter or lock?
+ if (((CInode*)lock->get_parent())->has_subtree_root_dirfrag())
+ scatter_scatter(lock);
+ else
+ scatter_lock(lock);
+ } else {
+ // replica.
+ // auth should be auth_pinned (see acquire_locks wrlock weird mustpin case).
+ int auth = lock->get_parent()->authority().first;
+ dout(10) << "requesting scatter from auth on "
+ << *lock << " on " << *lock->get_parent() << endl;
+ mds->send_message_mds(new MLock(lock, LOCK_AC_REQSCATTER, mds->get_nodeid()),
+ auth, MDS_PORT_LOCKER);
+ }
+ }
return false;
}
mdr->locks.erase(lock);
}
- scatter_eval(lock);
+ scatter_eval_gather(lock);
}
public:
C_Locker_ScatterEval(Locker *l, ScatterLock *lk) : locker(l), lock(lk) {}
void finish(int r) {
- locker->scatter_eval(lock);
+ locker->try_scatter_eval(lock);
}
};
-void Locker::scatter_eval(ScatterLock *lock)
+
+void Locker::try_scatter_eval(ScatterLock *lock)
{
// unstable and ambiguous auth?
if (!lock->is_stable() &&
return;
}
+ if (!lock->get_parent()->is_auth()) {
+ dout(7) << "try_scatter_eval not auth for " << *lock->get_parent() << endl;
+ return;
+ }
+
+ if (!lock->get_parent()->can_auth_pin()) {
+ dout(7) << "try_scatter_eval can't auth_pin, waiting on " << *lock->get_parent() << endl;
+ //if (!lock->get_parent()->is_waiter(MDSCacheObject::WAIT_SINGLEAUTH))
+ lock->get_parent()->add_waiter(MDSCacheObject::WAIT_AUTHPINNABLE, new C_Locker_ScatterEval(this, lock));
+ return;
+ }
+
+ if (lock->is_stable())
+ scatter_eval(lock);
+}
+
+
+void Locker::scatter_eval_gather(ScatterLock *lock)
+{
+ dout(10) << "scatter_eval_gather " << *lock << " on " << *lock->get_parent() << endl;
+
if (!lock->get_parent()->is_auth()) {
// REPLICA
- if (lock->get_state() == LOCK_GSYNCS &&
+ if (lock->get_state() == LOCK_GLOCKC &&
!lock->is_wrlocked()) {
- dout(10) << "scatter_eval no wrlocks, acking sync" << endl;
+ dout(10) << "scatter_eval no wrlocks, acking lock" << endl;
int auth = lock->get_parent()->authority().first;
if (mds->mdsmap->get_state(auth) >= MDSMap::STATE_REJOIN) {
bufferlist data;
lock->encode_locked_state(data);
- mds->send_message_mds(new MLock(lock, LOCK_AC_SYNCACK, mds->get_nodeid(), data),
+ mds->send_message_mds(new MLock(lock, LOCK_AC_LOCKACK, mds->get_nodeid(), data),
auth, MDS_PORT_LOCKER);
}
- lock->set_state(LOCK_SYNC);
- lock->finish_waiters(ScatterLock::WAIT_STABLE); // ?
+ lock->set_state(LOCK_LOCK);
+ //lock->get_parent()->put(CInode::PIN_SCATTERED);
}
-
+
} else {
// AUTH
-
- // gsyncs -> sync?
- if (lock->get_state() == LOCK_GSYNCS &&
+
+ // glocks|glockt -> lock?
+ if ((lock->get_state() == LOCK_GLOCKS ||
+ lock->get_state() == LOCK_GLOCKT) &&
!lock->is_gathering() &&
- !lock->is_wrlocked()) {
- dout(7) << "scatter_eval finished gather/un-wrlock on " << *lock
+ !lock->is_rdlocked()) {
+ dout(7) << "scatter_eval finished lock gather/un-rdlock on " << *lock
<< " on " << *lock->get_parent() << endl;
- lock->set_state(LOCK_SYNC);
- lock->finish_waiters(ScatterLock::WAIT_STABLE|ScatterLock::WAIT_RD);
+ lock->set_state(LOCK_LOCK);
+ lock->finish_waiters(ScatterLock::WAIT_XLOCK|ScatterLock::WAIT_STABLE);
+ lock->get_parent()->auth_unpin();
}
- // gscatters -> scatter?
- if (lock->get_state() == LOCK_GSCATTERS &&
- !lock->is_rdlocked()) {
- assert(lock->get_parent()->is_auth());
+ // glockc -> lock?
+ else if (lock->get_state() == LOCK_GLOCKC &&
+ !lock->is_gathering() &&
+ !lock->is_wrlocked()) {
+ if (lock->is_updated()) {
+ scatter_writebehind(lock);
+ } else {
+ dout(7) << "scatter_eval finished lock gather/un-wrlock on " << *lock
+ << " on " << *lock->get_parent() << endl;
+ lock->set_state(LOCK_LOCK);
+ //lock->get_parent()->put(CInode::PIN_SCATTERED);
+ lock->finish_waiters(ScatterLock::WAIT_XLOCK|ScatterLock::WAIT_STABLE);
+ lock->get_parent()->auth_unpin();
+ }
+ }
+
+ // gSyncL -> sync?
+ else if (lock->get_state() == LOCK_GSYNCL &&
+ !lock->is_wrlocked()) {
+ dout(7) << "scatter_eval finished sync un-wrlock on " << *lock
+ << " on " << *lock->get_parent() << endl;
+ if (lock->get_parent()->is_replicated()) {
+ // encode and bcast
+ bufferlist data;
+ lock->encode_locked_state(data);
+ send_lock_message(lock, LOCK_AC_SYNC, data);
+ }
+ lock->set_state(LOCK_SYNC);
+ lock->finish_waiters(ScatterLock::WAIT_RD|ScatterLock::WAIT_STABLE);
+ lock->get_parent()->auth_unpin();
+ }
+
+ // gscattert|gscatters -> scatter?
+ else if ((lock->get_state() == LOCK_GSCATTERT ||
+ lock->get_state() == LOCK_GSCATTERS) &&
+ !lock->is_gathering() &&
+ !lock->is_rdlocked()) {
+ dout(7) << "scatter_eval finished scatter un-rdlock(/gather) on " << *lock
+ << " on " << *lock->get_parent() << endl;
if (lock->get_parent()->is_replicated()) {
// encode and bcast
bufferlist data;
lock->encode_locked_state(data);
send_lock_message(lock, LOCK_AC_SCATTER, data);
- }
-
+ }
lock->set_state(LOCK_SCATTER);
- lock->get_wrlock();
- lock->finish_waiters(ScatterLock::WAIT_WR|ScatterLock::WAIT_STABLE);
- lock->put_wrlock();
+ //lock->get_parent()->get(CInode::PIN_SCATTERED);
+ lock->finish_waiters(ScatterLock::WAIT_WR|ScatterLock::WAIT_STABLE);
+ lock->get_parent()->auth_unpin();
}
-
- // waiting for rd?
- if (lock->get_state() == LOCK_SCATTER &&
- !lock->is_wrlocked() &&
- lock->is_waiter_for(ScatterLock::WAIT_RD)) {
- dout(10) << "scatter_eval no wrlocks, read waiter, syncing" << endl;
- scatter_sync(lock);
+
+ // gTempsyncC|gTempsyncL -> tempsync
+ else if ((lock->get_state() == LOCK_GTEMPSYNCC ||
+ lock->get_state() == LOCK_GTEMPSYNCL) &&
+ !lock->is_gathering() &&
+ !lock->is_wrlocked()) {
+ if (lock->is_updated()) {
+ scatter_writebehind(lock);
+ } else {
+ dout(7) << "scatter_eval finished tempsync gather/un-wrlock on " << *lock
+ << " on " << *lock->get_parent() << endl;
+ lock->set_state(LOCK_TEMPSYNC);
+ lock->finish_waiters(ScatterLock::WAIT_RD|ScatterLock::WAIT_STABLE);
+ lock->get_parent()->auth_unpin();
+ }
}
-
- // re-scatter?
- if (lock->get_state() == LOCK_SYNC &&
- !lock->is_rdlocked()) {
- dout(10) << "scatter_eval no rdlocks, scattering" << endl;
+
+
+ // re-eval?
+ if (lock->is_stable()) // && lock->get_parent()->can_auth_pin())
+ scatter_eval(lock);
+ }
+}
+
+void Locker::scatter_writebehind(ScatterLock *lock)
+{
+ CInode *in = (CInode*)lock->get_parent();
+ dout(10) << "scatter_writebehind on " << *lock << " on " << *in << endl;
+
+ // journal write-behind.
+ inode_t *pi = in->project_inode();
+ pi->version = in->pre_dirty();
+
+ EUpdate *le = new EUpdate("dir.mtime writebehind");
+ le->metablob.add_dir_context(in->get_parent_dn()->get_dir());
+ le->metablob.add_primary_dentry(in->get_parent_dn(), true, 0, pi);
+
+ mds->mdlog->submit_entry(le);
+ mds->mdlog->wait_for_sync(new C_Locker_ScatterWB(this, lock));
+}
+
+void Locker::scatter_writebehind_finish(ScatterLock *lock)
+{
+ CInode *in = (CInode*)lock->get_parent();
+ dout(10) << "scatter_writebehind_finish on " << *lock << " on " << *in << endl;
+ in->pop_and_dirty_projected_inode();
+ lock->clear_updated();
+ scatter_eval_gather(lock);
+}
+
+void Locker::scatter_eval(ScatterLock *lock)
+{
+ dout(10) << "scatter_eval " << *lock << " on " << *lock->get_parent() << endl;
+
+ assert(lock->get_parent()->is_auth());
+ assert(lock->is_stable());
+
+ if (((CInode*)lock->get_parent())->has_subtree_root_dirfrag()) {
+ // i _should_ be scattered.
+ if (!lock->is_rdlocked() &&
+ !lock->is_xlocked()) {
+ dout(10) << "scatter_eval no rdlocks|xlocks, am subtree root inode, scattering" << endl;
scatter_scatter(lock);
}
+ } else {
+ // i _should_ be sync.
+ if (!lock->is_wrlocked() &&
+ !lock->is_xlocked()) {
+ dout(10) << "scatter_eval no wrlocks|xlocks, not subtree root inode, syncing" << endl;
+ scatter_sync(lock);
+ }
}
}
dout(10) << "scatter_sync " << *lock
<< " on " << *lock->get_parent() << endl;
assert(lock->get_parent()->is_auth());
+ assert(lock->is_stable());
+
+ switch (lock->get_state()) {
+ case LOCK_SYNC:
+ return; // already sync.
+
+ case LOCK_TEMPSYNC:
+ break; // just do it.
+
+ case LOCK_LOCK:
+ if (lock->is_wrlocked() || lock->is_xlocked()) {
+ lock->set_state(LOCK_GSYNCL);
+ lock->get_parent()->auth_pin();
+ return;
+ }
+ break; // do it.
+
+ case LOCK_SCATTER:
+ // lock first. this is the slow way, incidentally.
+ if (lock->get_parent()->is_replicated()) {
+ send_lock_message(lock, LOCK_AC_LOCK);
+ lock->init_gather();
+ } else {
+ if (!lock->is_wrlocked()) {
+ //lock->get_parent()->put(CInode::PIN_SCATTERED);
+ break; // do it now, we're fine
+ }
+ }
+ lock->set_state(LOCK_GLOCKC);
+ lock->get_parent()->auth_pin();
+ return;
+
+ default:
+ assert(0);
+ }
- if (lock->get_state() == LOCK_SYNC) return;
- assert(lock->get_state() == LOCK_SCATTER);
-
- // bcast
+ // do sync
if (lock->get_parent()->is_replicated()) {
- send_lock_message(lock, LOCK_AC_SYNC);
- lock->set_state(LOCK_GSYNCS);
- lock->init_gather();
- }
- else if (lock->is_wrlocked()) {
- lock->set_state(LOCK_GSYNCS);
- }
- else {
- lock->set_state(LOCK_SYNC);
- lock->finish_waiters(ScatterLock::WAIT_RD|ScatterLock::WAIT_STABLE);
+ // encode and bcast
+ bufferlist data;
+ lock->encode_locked_state(data);
+ send_lock_message(lock, LOCK_AC_SYNC, data);
}
-}
+ lock->set_state(LOCK_SYNC);
+ lock->finish_waiters(ScatterLock::WAIT_RD|ScatterLock::WAIT_STABLE);
+}
void Locker::scatter_scatter(ScatterLock *lock)
{
dout(10) << "scatter_scatter " << *lock
<< " on " << *lock->get_parent() << endl;
assert(lock->get_parent()->is_auth());
+ assert(lock->is_stable());
- if (lock->get_state() == LOCK_SCATTER) return;
- assert(lock->get_state() == LOCK_SYNC);
-
- if (lock->is_rdlocked()) {
+ switch (lock->get_state()) {
+ case LOCK_SYNC:
+ if (!lock->is_rdlocked() &&
+ !lock->get_parent()->is_replicated())
+ break; // do it
+ if (lock->get_parent()->is_replicated()) {
+ send_lock_message(lock, LOCK_AC_LOCK);
+ lock->init_gather();
+ }
lock->set_state(LOCK_GSCATTERS);
- } else {
+ lock->get_parent()->auth_pin();
+ return;
+
+ case LOCK_LOCK:
+ if (lock->is_xlocked())
+ return; // sorry
+ break; // do it.
+
+ case LOCK_SCATTER:
+ return; // did it.
+
+ case LOCK_TEMPSYNC:
+ if (lock->is_rdlocked()) {
+ lock->set_state(LOCK_GSCATTERT);
+ lock->get_parent()->auth_pin();
+ return;
+ }
+ break; // do it
+
+ default:
+ assert(0);
+ }
+
+ // do scatter
+ if (lock->get_parent()->is_replicated()) {
+ // encode and bcast
+ bufferlist data;
+ lock->encode_locked_state(data);
+ send_lock_message(lock, LOCK_AC_SCATTER, data);
+ }
+ lock->set_state(LOCK_SCATTER);
+ //lock->get_parent()->get(CInode::PIN_SCATTERED);
+ lock->finish_waiters(ScatterLock::WAIT_WR|ScatterLock::WAIT_STABLE);
+}
+
+void Locker::scatter_lock(ScatterLock *lock)
+{
+ dout(10) << "scatter_lock " << *lock
+ << " on " << *lock->get_parent() << endl;
+ assert(lock->get_parent()->is_auth());
+ assert(lock->is_stable());
+
+ switch (lock->get_state()) {
+ case LOCK_SYNC:
+ if (!lock->is_rdlocked() &&
+ !lock->get_parent()->is_replicated())
+ break; // do it.
+
if (lock->get_parent()->is_replicated()) {
- // encode and bcast
- bufferlist data;
- lock->encode_locked_state(data);
- send_lock_message(lock, LOCK_AC_SCATTER, data);
+ send_lock_message(lock, LOCK_AC_LOCK);
+ lock->init_gather();
}
- lock->set_state(LOCK_SCATTER);
- lock->finish_waiters(ScatterLock::WAIT_WR|ScatterLock::WAIT_STABLE);
+ lock->set_state(LOCK_GLOCKS);
+ lock->get_parent()->auth_pin();
+ return;
+
+ case LOCK_LOCK:
+ return; // done.
+
+ case LOCK_SCATTER:
+ if (!lock->is_wrlocked() &&
+ !lock->get_parent()->is_replicated()) {
+ //lock->get_parent()->put(CInode::PIN_SCATTERED);
+ break; // do it.
+ }
+
+ if (lock->get_parent()->is_replicated()) {
+ send_lock_message(lock, LOCK_AC_LOCK);
+ lock->init_gather();
+ }
+ lock->set_state(LOCK_GLOCKC);
+ lock->get_parent()->auth_pin();
+ return;
+
+ case LOCK_TEMPSYNC:
+ if (lock->is_rdlocked()) {
+ lock->set_state(LOCK_GLOCKT);
+ lock->get_parent()->auth_pin();
+ return;
+ }
+ break; // do it.
+ }
+
+ // do lock
+ lock->set_state(LOCK_LOCK);
+ lock->finish_waiters(ScatterLock::WAIT_XLOCK|ScatterLock::WAIT_WR|ScatterLock::WAIT_STABLE);
+}
+
+void Locker::scatter_tempsync(ScatterLock *lock)
+{
+ dout(10) << "scatter_tempsync " << *lock
+ << " on " << *lock->get_parent() << endl;
+ assert(lock->get_parent()->is_auth());
+ assert(lock->is_stable());
+
+ switch (lock->get_state()) {
+ case LOCK_SYNC:
+ break; // do it.
+
+ case LOCK_LOCK:
+ if (lock->is_wrlocked() ||
+ lock->is_xlocked()) {
+ lock->set_state(LOCK_GTEMPSYNCL);
+ lock->get_parent()->auth_pin();
+ return;
+ }
+ break; // do it.
+
+ case LOCK_SCATTER:
+ if (!lock->is_wrlocked() &&
+ !lock->get_parent()->is_replicated()) {
+ //lock->get_parent()->put(CInode::PIN_SCATTERED);
+ break; // do it.
+ }
+
+ if (lock->get_parent()->is_replicated()) {
+ send_lock_message(lock, LOCK_AC_LOCK);
+ lock->init_gather();
+ }
+ lock->set_state(LOCK_GTEMPSYNCC);
+ lock->get_parent()->auth_pin();
+ return;
+
+ case LOCK_TEMPSYNC:
+ return; // done
}
+
+ // do tempsync
+ lock->set_state(LOCK_TEMPSYNC);
+ lock->finish_waiters(ScatterLock::WAIT_RD|ScatterLock::WAIT_STABLE);
}
+
+
+
void Locker::handle_scatter_lock(ScatterLock *lock, MLock *m)
{
int from = m->get_asker();
+ dout(10) << "handle_scatter_lock " << *m << " on " << *lock << " on " << *lock->get_parent() << endl;
if (mds->is_rejoin()) {
if (lock->get_parent()->is_rejoining()) {
switch (m->get_action()) {
// -- replica --
case LOCK_AC_SYNC:
- assert(lock->get_state() == LOCK_SCATTER);
+ assert(lock->get_state() == LOCK_LOCK);
+
+ lock->set_state(LOCK_SYNC);
+ lock->decode_locked_state(m->get_data());
+ lock->clear_updated();
+ lock->finish_waiters(ScatterLock::WAIT_RD|ScatterLock::WAIT_STABLE);
+ break;
+ case LOCK_AC_LOCK:
+ assert(lock->get_state() == LOCK_SCATTER ||
+ lock->get_state() == LOCK_SYNC);
+
// wait for wrlocks to close?
if (lock->is_wrlocked()) {
+ assert(lock->get_state() == LOCK_SCATTER);
dout(7) << "handle_scatter_lock has wrlocks, waiting on " << *lock
<< " on " << *lock->get_parent() << endl;
- lock->set_state(LOCK_GSYNCS);
+ lock->set_state(LOCK_GLOCKC);
+ } else if (lock->is_rdlocked()) {
+ assert(lock->get_state() == LOCK_SYNC);
+ dout(7) << "handle_scatter_lock has rdlocks, waiting on " << *lock
+ << " on " << *lock->get_parent() << endl;
+ lock->set_state(LOCK_GLOCKS);
} else {
+ //if (lock->get_state() == LOCK_SCATTER)
+ //lock->get_parent()->put(CInode::PIN_SCATTERED);
+
// encode and reply
bufferlist data;
lock->encode_locked_state(data);
- mds->send_message_mds(new MLock(lock, LOCK_AC_SYNCACK, mds->get_nodeid(), data),
+ mds->send_message_mds(new MLock(lock, LOCK_AC_LOCKACK, mds->get_nodeid(), data),
from, MDS_PORT_LOCKER);
+ lock->set_state(LOCK_LOCK);
}
break;
case LOCK_AC_SCATTER:
- assert(lock->get_state() == LOCK_SYNC);
+ assert(lock->get_state() == LOCK_LOCK);
lock->decode_locked_state(m->get_data());
+ lock->clear_updated();
lock->set_state(LOCK_SCATTER);
+ //lock->get_parent()->get(CInode::PIN_SCATTERED);
lock->finish_waiters(ScatterLock::WAIT_WR|ScatterLock::WAIT_STABLE);
break;
// -- for auth --
- case LOCK_AC_SYNCACK:
- assert(lock->get_state() == LOCK_GSYNCS);
+ case LOCK_AC_LOCKACK:
+ assert(lock->get_state() == LOCK_GLOCKS ||
+ lock->get_state() == LOCK_GLOCKC ||
+ lock->get_state() == LOCK_GSCATTERS);
assert(lock->is_gathering(from));
lock->remove_gather(from);
lock->decode_locked_state(m->get_data());
dout(7) << "handle_scatter_lock " << *lock << " on " << *lock->get_parent()
<< " from " << from << ", last one"
<< endl;
- scatter_eval(lock);
+ scatter_eval_gather(lock);
+ }
+ break;
+
+ case LOCK_AC_REQSCATTER:
+ if (lock->is_stable()) {
+ dout(7) << "handle_scatter_lock got scatter request on " << *lock << " on " << *lock->get_parent()
+ << endl;
+ scatter_scatter(lock);
+ } else {
+ dout(7) << "handle_scatter_lock ignoring scatter request on " << *lock << " on " << *lock->get_parent()
+ << endl;
}
break;
+
}
delete m;
+
+
+// ==========================================================================
+// local lock
+
+
+bool Locker::local_wrlock_start(LocalLock *lock, MDRequest *mdr)
+{
+ dout(7) << "local_wrlock_start on " << *lock
+ << " on " << *lock->get_parent() << endl;
+
+ if (lock->can_wrlock()) {
+ lock->get_wrlock();
+ mdr->wrlocks.insert(lock);
+ mdr->locks.insert(lock);
+ return true;
+ } else {
+ lock->add_waiter(SimpleLock::WAIT_WR|SimpleLock::WAIT_STABLE, new C_MDS_RetryRequest(mdcache, mdr));
+ return false;
+ }
+}
+
+void Locker::local_wrlock_finish(LocalLock *lock, MDRequest *mdr)
+{
+ dout(7) << "local_wrlock_finish on " << *lock
+ << " on " << *lock->get_parent() << endl;
+ lock->put_wrlock();
+ mdr->wrlocks.erase(lock);
+ mdr->locks.erase(lock);
+}
+
+bool Locker::local_xlock_start(LocalLock *lock, MDRequest *mdr)
+{
+ dout(7) << "local_xlock_start on " << *lock
+ << " on " << *lock->get_parent() << endl;
+
+ if (lock->is_xlocked_by_other(mdr)) {
+ lock->add_waiter(SimpleLock::WAIT_WR|SimpleLock::WAIT_STABLE, new C_MDS_RetryRequest(mdcache, mdr));
+ return false;
+ }
+
+ lock->get_xlock(mdr);
+ mdr->xlocks.insert(lock);
+ mdr->locks.insert(lock);
+ return true;
+}
+
+void Locker::local_xlock_finish(LocalLock *lock, MDRequest *mdr)
+{
+ dout(7) << "local_xlock_finish on " << *lock
+ << " on " << *lock->get_parent() << endl;
+ lock->put_xlock();
+ mdr->xlocks.erase(lock);
+ mdr->locks.erase(lock);
+
+ lock->finish_waiters(SimpleLock::WAIT_STABLE|SimpleLock::WAIT_WR);
+}
+
+
+
// ==========================================================================
// file lock
void Locker::file_rdlock_finish(FileLock *lock, MDRequest *mdr)
{
+ dout(7) << "rdlock_finish on " << *lock << " on " << *lock->get_parent() << endl;
+
// drop ref
- assert(lock->can_rdlock(mdr));
lock->put_rdlock();
mdr->rdlocks.erase(lock);
mdr->locks.erase(lock);
- dout(7) << "rdlock_finish on " << *lock << " on " << *lock->get_parent() << endl;
-
- if (!lock->is_rdlocked())
- file_eval(lock);
+ if (!lock->is_rdlocked())
+ file_eval_gather(lock);
}
void Locker::file_xlock_finish(FileLock *lock, MDRequest *mdr)
{
+ dout(7) << "file_xlock_finish on " << *lock << " on " << *lock->get_parent() << endl;
+
// drop ref
assert(lock->can_xlock(mdr));
lock->put_xlock();
mdr->locks.erase(lock);
mdr->xlocks.erase(lock);
- dout(7) << "file_xlock_finish on " << *lock << " on " << *lock->get_parent() << endl;
assert(lock->get_parent()->is_auth()); // or implement remote xlocks
// others waiting?
lock->finish_waiters(SimpleLock::WAIT_WR, 0);
- //// drop lock?
- //if (!lock->is_waiter_for(SimpleLock::WAIT_STABLE))
- file_eval(lock);
+ if (lock->get_parent()->is_auth())
+ file_eval(lock);
}
public:
C_Locker_FileEval(Locker *l, FileLock *lk) : locker(l), lock(lk) {}
void finish(int r) {
- locker->file_eval(lock);
+ locker->try_file_eval(lock);
}
};
-
-void Locker::file_eval(FileLock *lock)
+void Locker::try_file_eval(FileLock *lock)
{
CInode *in = (CInode*)lock->get_parent();
// unstable and ambiguous auth?
if (!lock->is_stable() &&
in->is_ambiguous_auth()) {
- dout(7) << "file_eval not stable and ambiguous auth, waiting on " << *in << endl;
+ dout(7) << "try_file_eval not stable and ambiguous auth, waiting on " << *in << endl;
//if (!lock->get_parent()->is_waiter(MDSCacheObject::WAIT_SINGLEAUTH))
in->add_waiter(CInode::WAIT_SINGLEAUTH, new C_Locker_FileEval(this, lock));
return;
}
+ if (!lock->get_parent()->is_auth()) {
+ dout(7) << "try_file_eval not auth for " << *lock->get_parent() << endl;
+ return;
+ }
+
+ if (!lock->get_parent()->can_auth_pin()) {
+ dout(7) << "try_file_eval can't auth_pin, waiting on " << *in << endl;
+ //if (!lock->get_parent()->is_waiter(MDSCacheObject::WAIT_SINGLEAUTH))
+ in->add_waiter(CInode::WAIT_AUTHPINNABLE, new C_Locker_FileEval(this, lock));
+ return;
+ }
+
+ if (lock->is_stable())
+ file_eval(lock);
+}
+
+
+void Locker::file_eval_gather(FileLock *lock)
+{
+ CInode *in = (CInode*)lock->get_parent();
int issued = in->get_caps_issued();
+
+ dout(7) << "file_eval_gather issued " << cap_string(issued)
+ << " vs " << cap_string(lock->caps_allowed())
+ << " on " << *lock << " on " << *lock->get_parent()
+ << endl;
+ if (lock->is_stable())
+ return; // nothing for us to do here!
+
// [auth] finished gather?
if (in->is_auth() &&
- !lock->is_stable() &&
- !lock->is_gathering()) {
- dout(7) << "file_eval finished mds gather on " << *lock << " on " << *lock->get_parent() << endl;
-
+ !lock->is_gathering() &&
+ ((issued & ~lock->caps_allowed()) == 0)) {
+ dout(7) << "file_eval_gather finished gather" << endl;
+
switch (lock->get_state()) {
// to lock
case LOCK_GLOCKR:
lock->get_rdlock();
lock->finish_waiters(SimpleLock::WAIT_STABLE|SimpleLock::WAIT_WR|SimpleLock::WAIT_RD);
lock->put_rdlock();
+ lock->get_parent()->auth_unpin();
}
break;
if ((issued & ~(CAP_FILE_RD)) == 0) {
lock->set_state(LOCK_MIXED);
lock->finish_waiters(SimpleLock::WAIT_STABLE);
+ lock->get_parent()->auth_unpin();
}
break;
}
lock->finish_waiters(SimpleLock::WAIT_STABLE);
+ lock->get_parent()->auth_unpin();
}
break;
// to loner
case LOCK_GLONERR:
- if (issued == 0) {
+ if ((issued & ~lock->caps_allowed()) == 0) {
lock->set_state(LOCK_LONER);
lock->finish_waiters(SimpleLock::WAIT_STABLE);
+ lock->get_parent()->auth_unpin();
}
break;
if ((issued & ~CAP_FILE_WR) == 0) {
lock->set_state(LOCK_LONER);
lock->finish_waiters(SimpleLock::WAIT_STABLE);
+ lock->get_parent()->auth_unpin();
}
break;
lock->get_rdlock();
lock->finish_waiters(SimpleLock::WAIT_RD|SimpleLock::WAIT_STABLE);
lock->put_rdlock();
+ lock->get_parent()->auth_unpin();
}
break;
}
issue_caps(in);
+
+ // stable re-eval?
+ if (lock->is_stable()) //&& lock->get_parent()->can_auth_pin())
+ file_eval(lock);
}
// [replica] finished caps gather?
- if (!in->is_auth() &&
- !lock->is_stable()) {
+ if (!in->is_auth()) {
switch (lock->get_state()) {
case LOCK_GMIXEDR:
if ((issued & ~(CAP_FILE_RD)) == 0) {
}
}
- // !stable -> do nothing.
- if (!lock->is_stable()) return;
+}
- // stable.
+void Locker::file_eval(FileLock *lock)
+{
+ CInode *in = (CInode*)lock->get_parent();
+ int wanted = in->get_caps_wanted();
+ bool loner = (in->client_caps.size() == 1) && in->mds_caps_wanted.empty();
+ dout(7) << "file_eval wanted=" << cap_string(wanted)
+ << " filelock=" << *lock << " on " << *lock->get_parent()
+ << " loner=" << loner
+ << endl;
+
+ assert(lock->get_parent()->is_auth());
assert(lock->is_stable());
- if (in->is_auth() &&
- !lock->is_xlocked()) {
- // [auth]
- // and not xlocked!
- int wanted = in->get_caps_wanted();
- bool loner = (in->client_caps.size() == 1) && in->mds_caps_wanted.empty();
- dout(7) << "file_eval wanted=" << cap_string(wanted)
- << " filelock=" << *lock << " on " << *lock->get_parent()
- << " loner=" << loner
- << endl;
-
- // * -> loner?
- if (!lock->is_rdlocked() &&
- !lock->is_waiter_for(SimpleLock::WAIT_WR) &&
- (wanted & CAP_FILE_WR) &&
- loner &&
- lock->get_state() != LOCK_LONER) {
- dout(7) << "file_eval stable, bump to loner " << *lock << " on " << *lock->get_parent() << endl;
- file_loner(lock);
- }
-
- // * -> mixed?
- else if (!lock->is_rdlocked() &&
- !lock->is_waiter_for(SimpleLock::WAIT_WR) &&
- (wanted & CAP_FILE_RD) &&
- (wanted & CAP_FILE_WR) &&
- !(loner && lock->get_state() == LOCK_LONER) &&
- lock->get_state() != LOCK_MIXED) {
- dout(7) << "file_eval stable, bump to mixed " << *lock << " on " << *lock->get_parent() << endl;
- file_mixed(lock);
- }
-
- // * -> sync?
- else if (!in->filelock.is_waiter_for(SimpleLock::WAIT_WR) &&
- !(wanted & (CAP_FILE_WR|CAP_FILE_WRBUFFER)) &&
- ((wanted & CAP_FILE_RD) ||
- in->is_replicated() ||
- (!loner && lock->get_state() == LOCK_LONER)) &&
- lock->get_state() != LOCK_SYNC) {
- dout(7) << "file_eval stable, bump to sync " << *lock << " on " << *lock->get_parent() << endl;
- file_sync(lock);
- }
-
- // * -> lock? (if not replicated or open)
- else if (!in->is_replicated() &&
- wanted == 0 &&
- lock->get_state() != LOCK_LOCK) {
- file_lock(lock);
- }
+ // not xlocked!
+ if (lock->is_xlocked()) return;
+
+ // * -> loner?
+ if (!lock->is_rdlocked() &&
+ !lock->is_waiter_for(SimpleLock::WAIT_WR) &&
+ (wanted & CAP_FILE_WR) &&
+ loner &&
+ lock->get_state() != LOCK_LONER) {
+ dout(7) << "file_eval stable, bump to loner " << *lock << " on " << *lock->get_parent() << endl;
+ file_loner(lock);
+ }
+
+ // * -> mixed?
+ else if (!lock->is_rdlocked() &&
+ !lock->is_waiter_for(SimpleLock::WAIT_WR) &&
+ (wanted & CAP_FILE_RD) &&
+ (wanted & CAP_FILE_WR) &&
+ !(loner && lock->get_state() == LOCK_LONER) &&
+ lock->get_state() != LOCK_MIXED) {
+ dout(7) << "file_eval stable, bump to mixed " << *lock << " on " << *lock->get_parent() << endl;
+ file_mixed(lock);
+ }
+
+ // * -> sync?
+ else if (!in->filelock.is_waiter_for(SimpleLock::WAIT_WR) &&
+ !(wanted & (CAP_FILE_WR|CAP_FILE_WRBUFFER)) &&
+ ((wanted & CAP_FILE_RD) ||
+ in->is_replicated() ||
+ (!loner && lock->get_state() == LOCK_LONER)) &&
+ lock->get_state() != LOCK_SYNC) {
+ dout(7) << "file_eval stable, bump to sync " << *lock << " on " << *lock->get_parent() << endl;
+ file_sync(lock);
+ }
+
+ // * -> lock? (if not replicated or open)
+ else if (!in->is_replicated() &&
+ wanted == 0 &&
+ lock->get_state() != LOCK_LOCK) {
+ file_lock(lock);
}
}
dout(7) << "file_sync " << *lock << " on " << *lock->get_parent() << endl;
assert(in->is_auth());
-
- // check state
- if (lock->get_state() == LOCK_SYNC ||
- lock->get_state() == LOCK_GSYNCL ||
- lock->get_state() == LOCK_GSYNCM)
- return true;
-
assert(lock->is_stable());
int issued = in->get_caps_issued();
if (issued & CAP_FILE_WR) {
// gather client write caps
lock->set_state(LOCK_GSYNCM);
+ lock->get_parent()->auth_pin();
issue_caps(in);
} else {
// no writers, go straight to sync
if (issued & CAP_FILE_WR) {
// gather client write caps
lock->set_state(LOCK_GSYNCL);
+ lock->get_parent()->auth_pin();
issue_caps(in);
} else {
// no writers, go straight to sync
dout(7) << "inode_file_lock " << *lock << " on " << *lock->get_parent() << endl;
assert(in->is_auth());
-
- // check state
- if (lock->get_state() == LOCK_LOCK ||
- lock->get_state() == LOCK_GLOCKR ||
- lock->get_state() == LOCK_GLOCKM ||
- lock->get_state() == LOCK_GLOCKL)
- return; // lock or locking
-
assert(lock->is_stable());
int issued = in->get_caps_issued();
// change lock
lock->set_state(LOCK_GLOCKR);
+ lock->get_parent()->auth_pin();
// call back caps
if (issued)
if (issued) {
// call back caps
lock->set_state(LOCK_GLOCKR);
+ lock->get_parent()->auth_pin();
issue_caps(in);
} else {
lock->set_state(LOCK_LOCK);
// change lock
lock->set_state(LOCK_GLOCKM);
+ lock->get_parent()->auth_pin();
// call back caps
issue_caps(in);
if (issued) {
// change lock
lock->set_state(LOCK_GLOCKM);
+ lock->get_parent()->auth_pin();
// call back caps
issue_caps(in);
if (issued & CAP_FILE_WR) {
// change lock
lock->set_state(LOCK_GLOCKL);
-
+ lock->get_parent()->auth_pin();
+
// call back caps
issue_caps(in);
} else {
CInode *in = (CInode*)lock->get_parent();
assert(in->is_auth());
-
- // check state
- if (lock->get_state() == LOCK_GMIXEDR ||
- lock->get_state() == LOCK_GMIXEDL)
- return; // mixed or mixing
-
assert(lock->is_stable());
int issued = in->get_caps_issued();
lock->init_gather();
lock->set_state(LOCK_GMIXEDR);
+ lock->get_parent()->auth_pin();
+
issue_caps(in);
} else {
if (issued) {
lock->set_state(LOCK_GMIXEDR);
- issue_caps(in);
+ lock->get_parent()->auth_pin();
+ issue_caps(in);
} else {
lock->set_state(LOCK_MIXED);
}
if (issued & CAP_FILE_WRBUFFER) {
// gather up WRBUFFER caps
lock->set_state(LOCK_GMIXEDL);
+ lock->get_parent()->auth_pin();
issue_caps(in);
}
else if (in->is_replicated()) {
dout(7) << "inode_file_loner " << *lock << " on " << *lock->get_parent() << endl;
assert(in->is_auth());
-
- // check state
- if (lock->get_state() == LOCK_LONER ||
- lock->get_state() == LOCK_GLONERR ||
- lock->get_state() == LOCK_GLONERM)
- return;
-
assert(lock->is_stable());
+
assert((in->client_caps.size() == 1) && in->mds_caps_wanted.empty());
if (lock->get_state() == LOCK_SYNC) {
// change lock
lock->set_state(LOCK_GLONERR);
+ lock->get_parent()->auth_pin();
} else {
// only one guy with file open, who gets it all, so
lock->set_state(LOCK_LONER);
// change lock
lock->set_state(LOCK_GLONERM);
+ lock->get_parent()->auth_pin();
} else {
lock->set_state(LOCK_LONER);
issue_caps(in);
lock->get_rdlock();
lock->finish_waiters(SimpleLock::WAIT_RD|SimpleLock::WAIT_STABLE);
lock->put_rdlock();
- file_eval(lock);
+ file_eval_gather(lock);
break;
case LOCK_AC_LOCK:
assert(lock->get_state() == LOCK_SYNC ||
lock->get_state() == LOCK_MIXED);
+ lock->set_state(LOCK_GLOCKR);
+
// call back caps?
if (issued & CAP_FILE_RD) {
dout(7) << "handle_file_lock client readers, gathering caps on " << *in << endl;
issue_caps(in);
+ break;
}
- if (lock->is_rdlocked()) {
+ else if (lock->is_rdlocked()) {
dout(7) << "handle_file_lock rdlocked, waiting before ack on " << *in << endl;
- lock->set_state(LOCK_GLOCKR);
break;
}
- if (issued & CAP_FILE_RD) {
- dout(7) << "handle_file_lock RD cap issued, waiting before ack on " << *in << endl;
- lock->set_state(LOCK_GLOCKR);
- break;
- }
// nothing to wait for, lock and ack.
{
// waiters
lock->finish_waiters(SimpleLock::WAIT_WR|SimpleLock::WAIT_STABLE);
- file_eval(lock);
+ file_eval_gather(lock);
break;
} else {
dout(7) << "handle_lock_inode_file " << *in << " from " << from
<< ", last one" << endl;
- file_eval(lock);
+ file_eval_gather(lock);
}
break;
} else {
dout(7) << "handle_lock_inode_file " << *in << " from " << from
<< ", last one" << endl;
- file_eval(lock);
+ file_eval_gather(lock);
}
break;
} else {
dout(7) << "handle_lock_inode_file " << *in << " from " << from
<< ", last one" << endl;
- file_eval(lock);
+ file_eval_gather(lock);
}
break;
class SimpleLock;
class FileLock;
class ScatterLock;
+class LocalLock;
class Locker {
private:
public:
Locker(MDS *m, MDCache *c) : mds(m), mdcache(c) {}
+ SimpleLock *get_lock(int lock_type, MDSCacheObjectInfo &info);
+
void dispatch(Message *m);
void handle_lock(MLock *m);
+protected:
void send_lock_message(SimpleLock *lock, int msg);
void send_lock_message(SimpleLock *lock, int msg, const bufferlist &data);
// -- locks --
+public:
bool acquire_locks(MDRequest *mdr,
set<SimpleLock*> &rdlocks,
set<SimpleLock*> &wrlocks,
set<SimpleLock*> &xlocks);
+ void drop_locks(MDRequest *mdr);
+
+protected:
bool rdlock_start(SimpleLock *lock, MDRequest *mdr);
void rdlock_finish(SimpleLock *lock, MDRequest *mdr);
bool xlock_start(SimpleLock *lock, MDRequest *mdr);
- void xlock_finish(SimpleLock *lock, MDRequest *mdr);
+public:
+ void xlock_finish(SimpleLock *lock, MDRequest *mdr); // public for Server's slave UNXLOCK
+protected:
bool wrlock_start(SimpleLock *lock, MDRequest *mdr);
void wrlock_finish(SimpleLock *lock, MDRequest *mdr);
+public:
+ void rejoin_set_state(SimpleLock *lock, int s, list<Context*>& waiters);
+
// simple
- void handle_simple_lock(SimpleLock *lock, MLock *m);
+public:
+ void try_simple_eval(SimpleLock *lock);
+ void simple_eval_gather(SimpleLock *lock);
+ bool simple_rdlock_try(SimpleLock *lock, Context *con);
+protected:
void simple_eval(SimpleLock *lock);
+ void handle_simple_lock(SimpleLock *lock, MLock *m);
void simple_sync(SimpleLock *lock);
void simple_lock(SimpleLock *lock);
- bool simple_rdlock_try(SimpleLock *lock, Context *con);
bool simple_rdlock_start(SimpleLock *lock, MDRequest *mdr);
void simple_rdlock_finish(SimpleLock *lock, MDRequest *mdr);
bool simple_xlock_start(SimpleLock *lock, MDRequest *mdr);
void simple_xlock_finish(SimpleLock *lock, MDRequest *mdr);
- bool dentry_can_rdlock_trace(vector<CDentry*>& trace, MClientRequest *req);
+public:
+ bool dentry_can_rdlock_trace(vector<CDentry*>& trace);
void dentry_anon_rdlock_trace_start(vector<CDentry*>& trace);
void dentry_anon_rdlock_trace_finish(vector<CDentry*>& trace);
// scatter
+public:
+ void try_scatter_eval(ScatterLock *lock);
+ void scatter_eval(ScatterLock *lock); // public for MDCache::adjust_subtree_auth()
+ void scatter_eval_gather(ScatterLock *lock);
+
+protected:
void handle_scatter_lock(ScatterLock *lock, MLock *m);
- void scatter_eval(ScatterLock *lock);
void scatter_sync(ScatterLock *lock);
+ void scatter_lock(ScatterLock *lock);
void scatter_scatter(ScatterLock *lock);
+ void scatter_tempsync(ScatterLock *lock);
bool scatter_rdlock_start(ScatterLock *lock, MDRequest *mdr);
void scatter_rdlock_finish(ScatterLock *lock, MDRequest *mdr);
bool scatter_wrlock_start(ScatterLock *lock, MDRequest *mdr);
void scatter_wrlock_finish(ScatterLock *lock, MDRequest *mdr);
+ void scatter_writebehind(ScatterLock *lock);
+ class C_Locker_ScatterWB : public Context {
+ Locker *locker;
+ ScatterLock *lock;
+ public:
+ C_Locker_ScatterWB(Locker *l, ScatterLock *sl) : locker(l), lock(sl) {}
+ void finish(int r) {
+ locker->scatter_writebehind_finish(lock);
+ }
+ };
+ void scatter_writebehind_finish(ScatterLock *lock);
+
+ // local
+protected:
+ bool local_wrlock_start(LocalLock *lock, MDRequest *mdr);
+ void local_wrlock_finish(LocalLock *lock, MDRequest *mdr);
+ bool local_xlock_start(LocalLock *lock, MDRequest *mdr);
+ void local_xlock_finish(LocalLock *lock, MDRequest *mdr);
+
+
// file
- void handle_file_lock(FileLock *lock, MLock *m);
+public:
+ void file_eval_gather(FileLock *lock);
+ void try_file_eval(FileLock *lock);
+protected:
void file_eval(FileLock *lock);
+ void handle_file_lock(FileLock *lock, MLock *m);
bool file_sync(FileLock *lock);
void file_lock(FileLock *lock);
void file_mixed(FileLock *lock);
#include "events/EString.h"
#include "events/ESession.h"
-#include "events/EClientMap.h"
-#include "events/EImportMap.h"
+#include "events/ESubtreeMap.h"
#include "events/EExport.h"
#include "events/EImportStart.h"
#include "events/EImportFinish.h"
#include "events/ESlaveUpdate.h"
#include "events/EOpen.h"
-#include "events/EAlloc.h"
#include "events/EPurgeFinish.h"
#include "events/EAnchor.h"
case EVENT_STRING: le = new EString; break;
case EVENT_SESSION: le = new ESession; break;
- case EVENT_CLIENTMAP: le = new EClientMap; break;
- case EVENT_IMPORTMAP: le = new EImportMap; break;
+ case EVENT_SUBTREEMAP: le = new ESubtreeMap; break;
case EVENT_EXPORT: le = new EExport; break;
case EVENT_IMPORTSTART: le = new EImportStart; break;
case EVENT_IMPORTFINISH: le = new EImportFinish; break;
case EVENT_SLAVEUPDATE: le = new ESlaveUpdate; break;
case EVENT_OPEN: le = new EOpen; break;
- case EVENT_ALLOC: le = new EAlloc; break;
case EVENT_PURGEFINISH: le = new EPurgeFinish; break;
case EVENT_ANCHOR: le = new EAnchor; break;
#define EVENT_STRING 1
#define EVENT_SESSION 7
-#define EVENT_CLIENTMAP 8
-#define EVENT_IMPORTMAP 2
+#define EVENT_SUBTREEMAP 2
#define EVENT_EXPORT 30
#define EVENT_IMPORTSTART 31
#define EVENT_IMPORTFINISH 32
set<int> up;
- mds->get_mds_map()->get_up_mds_set(up);
+ mds->get_mds_map()->get_in_mds_set(up);
for (set<int>::iterator p = up.begin(); p != up.end(); ++p) {
if (*p == mds->get_nodeid()) continue;
MHeartbeat *hb = new MHeartbeat(load, beat_epoch);
#include "osdc/Filer.h"
-#include "events/EImportMap.h"
+#include "events/ESubtreeMap.h"
#include "events/EUpdate.h"
#include "events/ESlaveUpdate.h"
#include "events/EString.h"
#include "messages/MGenericMessage.h"
-#include "messages/MMDSImportMap.h"
+#include "messages/MMDSResolve.h"
+#include "messages/MMDSResolveAck.h"
#include "messages/MMDSCacheRejoin.h"
#include "messages/MDiscover.h"
#include "messages/MInodeFileCaps.h"
-#include "messages/MInodeLink.h"
-#include "messages/MInodeLinkAck.h"
-#include "messages/MInodeUnlink.h"
-#include "messages/MInodeUnlinkAck.h"
-
#include "messages/MLock.h"
#include "messages/MDentryUnlink.h"
#include "messages/MClientRequest.h"
#include "messages/MClientFileCaps.h"
+#include "messages/MMDSSlaveRequest.h"
+
#include "IdAllocator.h"
#include "common/Timer.h"
migrator = new Migrator(mds, this);
// renamer = new Renamer(mds, this);
root = NULL;
+ stray = NULL;
lru.lru_set_max(g_conf.mds_cache_size);
lru.lru_set_midpoint(g_conf.mds_cache_mid);
root->inode.nlink = 1;
root->inode.layout = g_OSD_MDDirLayout;
+ root->force_auth = pair<int,int>(0, CDIR_AUTH_UNKNOWN);
+
set_root( root );
add_inode( root );
// discover
filepath want;
- MDiscover *req = new MDiscover(who,
+ MDiscover *req = new MDiscover(mds->get_nodeid(),
ino,
want,
- false);
- mds->send_message_mds(req, 0, MDS_PORT_CACHE);
+ false); // there _is_ no base dir for the stray inode
+ mds->send_message_mds(req, who, MDS_PORT_CACHE);
// wait
waiting_for_stray[ino].push_back(c);
}
+CDentry *MDCache::get_or_create_stray_dentry(CInode *in)
+{
+ string straydname;
+ in->name_stray_dentry(straydname);
+ frag_t fg = stray->pick_dirfrag(straydname);
+
+ CDir *straydir = stray->get_or_open_dirfrag(this, fg);
+
+ CDentry *straydn = straydir->lookup(straydname);
+ if (!straydn)
+ straydn = straydir->add_dentry(straydname, 0);
+
+ return straydn;
+}
+
+
+
+MDSCacheObject *MDCache::get_object(MDSCacheObjectInfo &info)
+{
+ // inode?
+ if (info.ino)
+ return get_inode(info.ino);
+
+ // dir or dentry.
+ CDir *dir = get_dirfrag(info.dirfrag);
+ if (!dir) return 0;
+
+ if (info.dname.length())
+ return dir->lookup(info.dname);
+ else
+ return dir;
+}
// ====================================================================
// subtree management
+void MDCache::list_subtrees(list<CDir*>& ls)
+{
+ for (map<CDir*,set<CDir*> >::iterator p = subtrees.begin();
+ p != subtrees.end();
+ ++p)
+ ls.push_back(p->first);
+}
+
/*
* adjust the dir_auth of a subtree.
* merge with parent and/or child subtrees, if is it appropriate.
// i am now the subtree root.
root = dir;
+
+ eval_subtree_root(dir);
}
// adjust export pins
p != oldbounds.end();
++p)
try_subtree_merge_at(*p);
-
}
void MDCache::try_subtree_merge_at(CDir *dir)
// we are no longer a subtree or bound
subtrees.erase(dir);
subtrees[parent].erase(dir);
+
+ eval_subtree_root(dir);
}
show_subtrees(15);
}
+void MDCache::eval_subtree_root(CDir *dir)
+{
+ // evaluate subtree inode dirlock?
+ // (we should scatter the dirlock on subtree bounds)
+ if (dir->inode->is_auth() &&
+ dir->inode->dirlock.is_stable()) {
+ // force the issue a bit
+ if (!dir->inode->is_frozen())
+ mds->locker->scatter_eval(&dir->inode->dirlock);
+ else
+ mds->locker->try_scatter_eval(&dir->inode->dirlock); // ** may or may not be auth_pinned **
+ }
+}
+
void MDCache::adjust_bounded_subtree_auth(CDir *dir, set<CDir*>& bounds, pair<int,int> auth)
{
p = n;
}
+ // adjust export pins
+ adjust_export_state(dir);
+ for (set<CDir*>::iterator p = subtrees[dir].begin();
+ p != subtrees[dir].end();
+ ++p)
+ adjust_export_state(*p);
+
// bound should now match.
verify_subtree_bounds(dir, bounds);
{
dout(10) << "adjust_subtree_after_rename " << *diri << " from " << *olddir << endl;
+ //show_subtrees();
+
list<CDir*> dfls;
diri->get_dirfrags(dfls);
for (list<CDir*>::iterator p = dfls.begin(); p != dfls.end(); ++p) {
CDir *dir = *p;
-
+
+ dout(10) << "dirfrag " << *dir << endl;
CDir *oldparent = get_subtree_root(olddir);
+ dout(10) << " old parent " << *oldparent << endl;
CDir *newparent = get_subtree_root(diri->get_parent_dir());
+ dout(10) << " new parent " << *newparent << endl;
if (oldparent == newparent) {
dout(10) << "parent unchanged for " << *dir << " at " << *oldparent << endl;
// mid-subtree.
// see if any old bounds move to the new parent.
+ list<CDir*> tomove;
for (set<CDir*>::iterator p = subtrees[oldparent].begin();
p != subtrees[oldparent].end();
++p) {
CDir *broot = get_subtree_root(bound->get_parent_dir());
if (broot != oldparent) {
assert(broot == newparent);
- dout(10) << "moving bound " << *bound << " from " << *oldparent << " to " << *newparent << endl;
- subtrees[oldparent].erase(broot);
- subtrees[newparent].insert(broot);
+ tomove.push_back(bound);
}
+ }
+ for (list<CDir*>::iterator p = tomove.begin(); p != tomove.end(); ++p) {
+ CDir *bound = *p;
+ dout(10) << "moving bound " << *bound << " from " << *oldparent << " to " << *newparent << endl;
+ subtrees[oldparent].erase(bound);
+ subtrees[newparent].insert(bound);
}
// did auth change?
}
}
+ for (list<CDir*>::iterator p = dfls.begin(); p != dfls.end(); ++p) {
+ CDir *dir = *p;
+
+ // un-force dir to subtree root
+ if (dir->dir_auth == pair<int,int>(dir->dir_auth.first, dir->dir_auth.first))
+ adjust_subtree_auth(dir, dir->dir_auth.first);
+ }
+
+ show_subtrees();
}
* take note of where we write import_maps in the log, as we need
* to take care not to expire them until an updated map is safely flushed.
*/
-class C_MDS_WroteImportMap : public Context {
- MDLog *mdlog;
+class C_MDS_WroteSubtreeMap : public Context {
+ MDCache *mdcache;
off_t end_off;
public:
- C_MDS_WroteImportMap(MDLog *ml, off_t eo) : mdlog(ml), end_off(eo) { }
+ C_MDS_WroteSubtreeMap(MDCache *mc, off_t eo) : mdcache(mc), end_off(eo) { }
void finish(int r) {
- // cout << "WroteImportMap at " << end_off << endl;
- if (r >= 0)
- mdlog->last_import_map = end_off;
- mdlog->writing_import_map = false;
+ mdcache->_logged_subtree_map(end_off);
}
};
-void MDCache::log_import_map(Context *onsync)
+void MDCache::log_subtree_map(Context *onsync)
{
- dout(10) << "log_import_map " << num_subtrees() << " subtrees"
+ dout(10) << "log_subtree_map " << num_subtrees() << " subtrees, "
<< num_subtrees_fullauth() << " fullauth"
<< endl;
- EImportMap *le = new EImportMap;
+ ESubtreeMap *le = new ESubtreeMap;
// include all auth subtrees, and their bounds.
// and a spanning tree to tie it to the root.
CDir *dir = p->first;
if (!dir->is_auth()) continue;
- le->imports.insert(dir->dirfrag());
- le->metablob.add_dir_context(dir, true);
+ dout(15) << " subtree " << *dir << endl;
+ le->subtrees[dir->dirfrag()].clear();
+ le->metablob.add_dir_context(dir, EMetaBlob::TO_ROOT);
le->metablob.add_dir(dir, false);
// bounds
q != p->second.end();
++q) {
CDir *bound = *q;
- le->bounds[dir->dirfrag()].insert(bound->dirfrag());
- le->metablob.add_dir_context(bound);
+ dout(15) << " subtree bound " << *bound << endl;
+ le->subtrees[dir->dirfrag()].push_back(bound->dirfrag());
+ le->metablob.add_dir_context(bound, EMetaBlob::TO_ROOT);
le->metablob.add_dir(bound, false);
}
}
- mds->mdlog->writing_import_map = true;
+ //le->metablob.print(cout);
+
+ Context *fin = new C_MDS_WroteSubtreeMap(this, mds->mdlog->get_write_pos());
+ mds->mdlog->writing_subtree_map = true;
mds->mdlog->submit_entry(le);
- mds->mdlog->wait_for_sync(new C_MDS_WroteImportMap(mds->mdlog, mds->mdlog->get_write_pos()));
+ mds->mdlog->wait_for_sync(fin);
if (onsync)
mds->mdlog->wait_for_sync(onsync);
}
+void MDCache::_logged_subtree_map(off_t off)
+{
+ dout(10) << "_logged_subtree_map at " << off << endl;
+ mds->mdlog->last_subtree_map = off;
+ mds->mdlog->writing_subtree_map = false;
+
+ list<Context*> ls;
+ mds->mdlog->take_subtree_map_expire_waiters(ls);
+ mds->queue_waiters(ls);
+}
+
-void MDCache::send_import_map(int who)
+void MDCache::send_resolve(int who)
{
if (migrator->is_exporting())
- send_import_map_later(who);
+ send_resolve_later(who);
else
- send_import_map_now(who);
+ send_resolve_now(who);
}
-void MDCache::send_import_map_later(int who)
+void MDCache::send_resolve_later(int who)
{
- dout(10) << "send_import_map_later to mds" << who << endl;
- wants_import_map.insert(who);
+ dout(10) << "send_resolve_later to mds" << who << endl;
+ wants_resolve.insert(who);
}
-void MDCache::send_pending_import_maps()
+void MDCache::maybe_send_pending_resolves()
{
- if (wants_import_map.empty())
+ if (wants_resolve.empty())
return; // nothing to send.
// only if it's appropriate!
if (migrator->is_exporting() ||
migrator->is_importing()) {
- dout(7) << "send_pending_import_maps waiting, imports/exports still in progress" << endl;
+ dout(7) << "maybe_send_pending_resolves waiting, imports/exports still in progress" << endl;
return; // not now
}
// ok, send them.
- for (set<int>::iterator p = wants_import_map.begin();
- p != wants_import_map.end();
+ for (set<int>::iterator p = wants_resolve.begin();
+ p != wants_resolve.end();
p++)
- send_import_map_now(*p);
- wants_import_map.clear();
+ send_resolve_now(*p);
+ wants_resolve.clear();
}
-void MDCache::send_import_map_now(int who)
+
+class C_MDC_SendResolve : public Context {
+ MDCache *mdc;
+ int who;
+public:
+ C_MDC_SendResolve(MDCache *c, int w) : mdc(c), who(w) { }
+ void finish(int r) {
+ mdc->send_resolve_now(who);
+ }
+};
+
+void MDCache::send_resolve_now(int who)
{
- dout(10) << "send_import_map_now to mds" << who << endl;
- MMDSImportMap *m = new MMDSImportMap;
+ dout(10) << "send_resolve_now to mds" << who << endl;
+ MMDSResolve *m = new MMDSResolve;
show_subtrees();
migrator->get_import_bound_inos(dir->dirfrag()));
} else {
// not ambiguous.
- m->add_import(dir->dirfrag());
+ m->add_subtree(dir->dirfrag());
// bounds too
for (set<CDir*>::iterator q = subtrees[dir].begin();
q != subtrees[dir].end();
++q) {
CDir *bound = *q;
- m->add_import_export(dir->dirfrag(), bound->dirfrag());
+ m->add_subtree_bound(dir->dirfrag(), bound->dirfrag());
}
}
}
++p)
m->add_ambiguous_import(p->first, p->second);
+
+ // list prepare requests lacking a commit
+ // [active survivor]
+ for (hash_map<metareqid_t, MDRequest*>::iterator p = active_requests.begin();
+ p != active_requests.end();
+ ++p) {
+ if (p->second->is_slave() && p->second->slave_to_mds == who) {
+ dout(10) << " including uncommitted " << *p->second << endl;
+ m->add_slave_request(p->first);
+ }
+ }
+ // [resolving]
+ if (uncommitted_slave_updates.count(who)) {
+ for (map<metareqid_t, EMetaBlob>::iterator p = uncommitted_slave_updates[who].begin();
+ p != uncommitted_slave_updates[who].end();
+ ++p) {
+ dout(10) << " including uncommitted " << p->first << endl;
+ m->add_slave_request(p->first);
+ }
+ need_resolve_ack.insert(who);
+ }
+
+
// send
mds->send_message_mds(m, who, MDS_PORT_CACHE);
}
dout(1) << "my recovery peers will be " << recovery_set << endl;
// adjust my recovery lists
- wants_import_map.erase(who); // MDS will ask again
- got_import_map.erase(who); // i'll get another.
+ wants_resolve.erase(who); // MDS will ask again
+ got_resolve.erase(who); // i'll get another.
rejoin_ack_gather.erase(who); // i'll need/get another.
// adjust subtree auth
- for (map<CDir*,set<CDir*> >::iterator p = subtrees.begin();
- p != subtrees.end();
+ list<CDir*> subs;
+ list_subtrees(subs);
+ for (list<CDir*>::iterator p = subs.begin();
+ p != subs.end();
++p) {
- CDir *dir = p->first;
+ CDir *dir = *p;
// only if we are a _bystander_.
if (dir->dir_auth.first == who &&
dir->dir_auth.second >= 0 &&
}
// tell the migrator too.
- migrator->handle_mds_failure(who);
+ migrator->handle_mds_failure_or_stop(who);
// kick any dir discovers that are waiting
hash_map<inodeno_t,set<int> >::iterator p = dir_discovers.begin();
p = n;
}
- // clean up any slave requests from this node
- list<MDRequest*> ls;
+ // clean up any requests slave to/from this node
+ list<MDRequest*> finish;
for (hash_map<metareqid_t, MDRequest*>::iterator p = active_requests.begin();
p != active_requests.end();
- ++p)
- if (p->second->by_mds == who)
- ls.push_back(p->second);
- while (!ls.empty()) {
- dout(10) << "cleaning up slave request " << *ls.front() << endl;
- request_finish(ls.front());
- ls.pop_front();
+ ++p) {
+ // slave to the failed node?
+ if (p->second->slave_to_mds == who) {
+ if (p->second->slave_did_prepare()) {
+ dout(10) << " slave request " << *p->second << " uncommitted, will resolve shortly" << endl;
+ } else {
+ dout(10) << " slave request " << *p->second << " has no prepare, finishing up" << endl;
+ if (p->second->slave_request)
+ p->second->aborted = true;
+ else
+ finish.push_back(p->second);
+ }
+ }
+
+ // failed node is slave?
+ if (!p->second->committing) {
+ if (p->second->witnessed.count(who)) {
+ dout(10) << " master request " << *p->second << " no longer witnessed by slave mds" << who
+ << endl;
+ // discard this peer's prepare (if any)
+ p->second->witnessed.erase(who);
+ }
+
+ if (p->second->waiting_on_slave.count(who)) {
+ dout(10) << " master request " << *p->second << " waiting for slave mds" << who
+ << " to recover" << endl;
+ // retry request when peer recovers
+ p->second->waiting_on_slave.erase(who);
+ mds->wait_for_active_peer(who, new C_MDS_RetryRequest(this, p->second));
+ }
+ }
+ }
+
+ while (!finish.empty()) {
+ dout(10) << "cleaning up slave request " << *finish.front() << endl;
+ request_finish(finish.front());
+ finish.pop_front();
}
show_subtrees();
/*
- * during resolve state, we share import_maps to determine who
- * is authoritative for which trees. we expect to get an import_map
+ * during resolve state, we share resolves to determine who
+ * is authoritative for which trees. we expect to get an resolve
* from _everyone_ in the recovery_set (the mds cluster at the time of
* the first failure).
*/
-void MDCache::handle_import_map(MMDSImportMap *m)
+void MDCache::handle_resolve(MMDSResolve *m)
{
- dout(7) << "handle_import_map from " << m->get_source() << endl;
+ dout(7) << "handle_resolve from " << m->get_source() << endl;
int from = m->get_source().num();
+ // ambiguous slave requests?
+ if (!m->slave_requests.empty()) {
+ MMDSResolveAck *ack = new MMDSResolveAck;
+
+ for (list<metareqid_t>::iterator p = m->slave_requests.begin();
+ p != m->slave_requests.end();
+ ++p) {
+ if (mds->clientmap.have_completed_request(*p)) {
+ // COMMIT
+ dout(10) << " ambiguous slave request " << *p << " will COMMIT" << endl;
+ ack->add_commit(*p);
+ } else {
+ // ABORT
+ dout(10) << " ambiguous slave request " << *p << " will ABORT" << endl;
+ ack->add_abort(*p);
+ }
+ }
+
+ mds->send_message_mds(ack, from, MDS_PORT_CACHE);
+ }
+
// update my dir_auth values
- for (map<dirfrag_t, list<dirfrag_t> >::iterator pi = m->imap.begin();
- pi != m->imap.end();
+ for (map<dirfrag_t, list<dirfrag_t> >::iterator pi = m->subtrees.begin();
+ pi != m->subtrees.end();
++pi) {
CDir *im = get_dirfrag(pi->first);
if (im) {
}
// am i a surviving ambiguous importer?
+ /*
+ * note: it would be cleaner to do this check before updating our own
+ * subtree map.. then the import_finish or _reverse could operate on an
+ * un-munged subtree map. but... checking for import completion against
+ * the provided resolve isn't easy. so, we skip audit checks in these
+ * functions.
+ */
if (mds->is_active() || mds->is_stopping()) {
// check for any import success/failure (from this node)
map<dirfrag_t, list<dirfrag_t> >::iterator p = my_ambiguous_imports.begin();
CDir *dir = get_dirfrag(p->first);
assert(dir);
dout(10) << "checking ambiguous import " << *dir << endl;
- assert(migrator->is_importing(dir->dirfrag()));
- assert(migrator->get_import_state(dir->dirfrag()) == Migrator::IMPORT_ACKING);
- if (migrator->get_import_peer(dir->dirfrag()) == from) {
- if (dir->is_ambiguous_dir_auth()) {
- dout(7) << "ambiguous import succeeded on " << *dir << endl;
- migrator->import_finish(dir, true); // don't wait for log flush
- } else {
- dout(7) << "ambiguous import failed on " << *dir << endl;
- migrator->import_reverse(dir, false); // don't adjust dir_auth.
+ if (migrator->is_importing(dir->dirfrag())) {
+ assert(migrator->get_import_state(dir->dirfrag()) == Migrator::IMPORT_ACKING);
+ if (migrator->get_import_peer(dir->dirfrag()) == from) {
+ if (dir->is_ambiguous_dir_auth()) {
+ dout(7) << "ambiguous import succeeded on " << *dir << endl;
+ migrator->import_finish(dir, true); // don't wait for log flush
+ } else {
+ dout(7) << "ambiguous import failed on " << *dir << endl;
+ migrator->import_reverse(dir, false); // don't adjust dir_auth.
+ }
+ my_ambiguous_imports.erase(p);
}
- my_ambiguous_imports.erase(p);
}
p = n;
}
show_subtrees();
- // recovering?
- if (!mds->is_rejoin() && !mds->is_active() && !mds->is_stopping()) {
- // note ambiguous imports too.. unless i'm already active
- for (map<dirfrag_t, list<dirfrag_t> >::iterator pi = m->ambiguous_imap.begin();
- pi != m->ambiguous_imap.end();
+ // resolving?
+ if (mds->is_resolve()) {
+ // note ambiguous imports too
+ for (map<dirfrag_t, list<dirfrag_t> >::iterator pi = m->ambiguous_imports.begin();
+ pi != m->ambiguous_imports.end();
++pi) {
dout(10) << "noting ambiguous import on " << pi->first << " bounds " << pi->second << endl;
other_ambiguous_imports[from][pi->first].swap( pi->second );
}
// did i get them all?
- got_import_map.insert(from);
+ got_resolve.insert(from);
+
+ maybe_resolve_finish();
+ }
+
+ delete m;
+}
+
+void MDCache::maybe_resolve_finish()
+{
+ if (got_resolve != recovery_set) {
+ dout(10) << "still waiting for more resolves, got (" << got_resolve
+ << "), need (" << recovery_set << ")" << endl;
+ }
+ else if (!need_resolve_ack.empty()) {
+ dout(10) << "still waiting for resolve_ack from (" << need_resolve_ack << ")" << endl;
+ }
+ else {
+ dout(10) << "got all import maps, resolve_acks, done resolving subtrees" << endl;
+ disambiguate_imports();
+ recalc_auth_bits();
+ trim_non_auth();
+
+ mds->resolve_done();
+ }
+}
+
+void MDCache::handle_resolve_ack(MMDSResolveAck *ack)
+{
+ dout(10) << "handle_resolve_ack " << *ack << " from " << ack->get_source() << endl;
+ int from = ack->get_source().num();
+
+ for (list<metareqid_t>::iterator p = ack->commit.begin();
+ p != ack->commit.end();
+ ++p) {
+ dout(10) << " commit on slave " << *p << endl;
- if (got_import_map == recovery_set) {
- dout(10) << "got all import maps, done resolving subtrees" << endl;
- disambiguate_imports();
- recalc_auth_bits();
- trim_non_auth();
-
- // reconnect clients
- mds->set_want_state(MDSMap::STATE_RECONNECT);
+ if (mds->is_resolve()) {
+ // replay
+ assert(uncommitted_slave_updates[from].count(*p));
+ uncommitted_slave_updates[from][*p].replay(mds);
+ uncommitted_slave_updates[from].erase(*p);
+ // log commit
+ mds->mdlog->submit_entry(new ESlaveUpdate("unknown", *p, from, ESlaveUpdate::OP_COMMIT));
+ } else {
+ MDRequest *mdr = request_get(*p);
+ assert(mdr->slave_request == 0); // shouldn't be doing anything!
+ request_finish(mdr);
+ }
+ }
+
+ for (list<metareqid_t>::iterator p = ack->abort.begin();
+ p != ack->abort.end();
+ ++p) {
+ dout(10) << " abort on slave " << *p << endl;
+ if (mds->is_resolve()) {
+ assert(uncommitted_slave_updates[from].count(*p));
+ uncommitted_slave_updates[from].erase(*p);
+ mds->mdlog->submit_entry(new ESlaveUpdate("unknown", *p, from, ESlaveUpdate::OP_ROLLBACK));
} else {
- dout(10) << "still waiting for more importmaps, got " << got_import_map
- << ", need " << recovery_set << endl;
+ MDRequest *mdr = request_get(*p);
+ if (mdr->slave_commit) {
+ mdr->slave_commit->finish(-1);
+ delete mdr->slave_commit;
+ mdr->slave_commit = 0;
+ }
+ if (mdr->slave_request)
+ mdr->aborted = true;
+ else
+ request_finish(mdr);
}
}
- delete m;
+ need_resolve_ack.erase(from);
+
+ if (mds->is_resolve())
+ maybe_resolve_finish();
+
+ delete ack;
}
+
void MDCache::disambiguate_imports()
{
dout(10) << "disambiguate_imports" << endl;
show_cache();
}
+
+
+// ===========================================================================
+// REJOIN
+
+
/*
* rejoin phase!
* we start out by sending rejoins to everyone in the recovery set.
* if we are rejoin, send for all regions in our cache.
* if we are active|stopping, send only to nodes that are are rejoining.
*/
-void MDCache::send_cache_rejoins()
+void MDCache::rejoin_send_rejoins()
{
- dout(10) << "send_cache_rejoins with recovery_set " << recovery_set << endl;
+ dout(10) << "rejoin_send_rejoins with recovery_set " << recovery_set << endl;
map<int, MMDSCacheRejoin*> rejoins;
+ // encode cap list once.
+ bufferlist cap_export_bl;
+ if (mds->is_rejoin()) {
+ ::_encode(cap_exports, cap_export_bl);
+ ::_encode(cap_export_paths, cap_export_bl);
+ }
+
// if i am rejoining, send a rejoin to everyone.
// otherwise, just send to others who are rejoining.
for (set<int>::iterator p = recovery_set.begin();
p != recovery_set.end();
++p) {
if (*p == mds->get_nodeid()) continue; // nothing to myself!
- if (mds->is_rejoin() ||
- mds->mdsmap->is_rejoin(*p))
- rejoins[*p] = new MMDSCacheRejoin(MMDSCacheRejoin::OP_REJOIN);
+ if (mds->is_rejoin()) {
+ rejoin_gather.insert(*p);
+ rejoins[*p] = new MMDSCacheRejoin(MMDSCacheRejoin::OP_WEAK);
+ rejoins[*p]->copy_cap_exports(cap_export_bl);
+ } else if (mds->mdsmap->is_rejoin(*p))
+ rejoins[*p] = new MMDSCacheRejoin(MMDSCacheRejoin::OP_STRONG);
}
-
+
assert(!migrator->is_importing());
assert(!migrator->is_exporting());
if (auth == mds->get_nodeid()) continue; // skip my own regions!
if (rejoins.count(auth) == 0) continue; // don't care about this node's regions
- cache_rejoin_walk(dir, rejoins[auth]);
+ rejoin_walk(dir, rejoins[auth]);
+ }
+
+ if (!mds->is_rejoin()) {
+ // strong.
+ // note request authpins, xlocks
+ for (hash_map<metareqid_t, MDRequest*>::iterator p = active_requests.begin();
+ p != active_requests.end();
+ ++p) {
+ // auth pins
+ for (set<MDSCacheObject*>::iterator q = p->second->auth_pins.begin();
+ q != p->second->auth_pins.end();
+ ++q) {
+ if (!(*q)->is_auth()) {
+ int who = (*q)->authority().first;
+ if (rejoins.count(who) == 0) continue;
+ MMDSCacheRejoin *rejoin = rejoins[who];
+
+ dout(15) << " " << *p->second << " authpin on " << **q << endl;
+ MDSCacheObjectInfo i;
+ (*q)->set_object_info(i);
+ if (i.ino)
+ rejoin->add_inode_authpin(i.ino, p->second->reqid);
+ else
+ rejoin->add_dentry_authpin(i.dirfrag, i.dname, p->second->reqid);
+ }
+ }
+ // xlocks
+ for (set<SimpleLock*>::iterator q = p->second->xlocks.begin();
+ q != p->second->xlocks.end();
+ ++q) {
+ if (!(*q)->get_parent()->is_auth()) {
+ int who = (*q)->get_parent()->authority().first;
+ if (rejoins.count(who) == 0) continue;
+ MMDSCacheRejoin *rejoin = rejoins[who];
+
+ dout(15) << " " << *p->second << " xlock on " << **q << " " << *(*q)->get_parent() << endl;
+ MDSCacheObjectInfo i;
+ (*q)->get_parent()->set_object_info(i);
+ if (i.ino)
+ rejoin->add_inode_xlock(i.ino, (*q)->get_type(), p->second->reqid);
+ else
+ rejoin->add_dentry_xlock(i.dirfrag, i.dname, p->second->reqid);
+ }
+ }
+ }
}
// send the messages
// nothing?
if (mds->is_rejoin() && rejoins.empty()) {
- dout(10) << "nothing to rejoin, going active" << endl;
- mds->set_want_state(MDSMap::STATE_ACTIVE);
+ dout(10) << "nothing left to rejoin" << endl;
+ mds->rejoin_done();
}
}
-void MDCache::cache_rejoin_walk(CDir *dir, MMDSCacheRejoin *rejoin)
+/**
+ * rejoin_walk - build rejoin declarations for a subtree
+ *
+ * @dir subtree root
+ * @rejoin rejoin message
+ *
+ * from a rejoining node:
+ * weak dirfrag
+ * weak dentries (w/ connectivity)
+ *
+ * from a surviving node:
+ * strong dirfrag
+ * strong dentries (no connectivity!)
+ * strong inodes
+ */
+void MDCache::rejoin_walk(CDir *dir, MMDSCacheRejoin *rejoin)
{
- dout(10) << "cache_rejoin_walk " << *dir << endl;
+ dout(10) << "rejoin_walk " << *dir << endl;
- //if (mds->is_rejoin())
- rejoin->add_weak_dirfrag(dir->dirfrag());
- //else
- //rejoin->add_strong_dirfrag(dir->dirfrag(), dir->get_replica_nonce());
-
list<CDir*> nested; // finish this dir, then do nested items
- // walk dentries
- for (map<string,CDentry*>::iterator p = dir->items.begin();
- p != dir->items.end();
- ++p) {
- // dentry
- CDentry *dn = p->second;
- if (mds->is_rejoin())
- rejoin->add_weak_dentry(dir->dirfrag(), p->first);
- else {
+ if (mds->is_rejoin()) {
+ // WEAK
+ rejoin->add_weak_dirfrag(dir->dirfrag());
+
+ for (map<string,CDentry*>::iterator p = dir->items.begin();
+ p != dir->items.end();
+ ++p) {
+ CDentry *dn = p->second;
+ if (dn->is_primary()) {
+ rejoin->add_weak_primary_dentry(dir->dirfrag(), p->first, dn->get_inode()->ino());
+ dn->get_inode()->get_nested_dirfrags(nested);
+ } else if (dn->is_remote())
+ rejoin->add_weak_remote_dentry(dir->dirfrag(), p->first, dn->get_remote_ino());
+ else
+ assert(0); // i shouldn't have a non-auth null dentry after replay + trim_non_auth()
+ }
+ } else {
+ // STRONG
+ rejoin->add_strong_dirfrag(dir->dirfrag(), dir->get_replica_nonce());
+
+ for (map<string,CDentry*>::iterator p = dir->items.begin();
+ p != dir->items.end();
+ ++p) {
+ CDentry *dn = p->second;
rejoin->add_strong_dentry(dir->dirfrag(), p->first,
+ dn->is_primary() ? dn->get_inode()->ino():inodeno_t(0),
+ dn->is_remote() ? dn->get_remote_ino():inodeno_t(0),
dn->get_replica_nonce(),
dn->lock.get_state());
- if (dn->lock.is_xlocked())
- rejoin->add_dentry_xlock(dir->dirfrag(), p->first,
- dn->lock.get_xlocked_by()->reqid);
- }
-
- // inode?
- if (dn->is_primary() && dn->get_inode()) {
- CInode *in = dn->get_inode();
- if (mds->is_rejoin() && in->get_caps_wanted() == 0)
- rejoin->add_weak_inode(in->ino());
- else {
+ if (dn->is_primary()) {
+ CInode *in = dn->get_inode();
rejoin->add_strong_inode(in->ino(), in->get_replica_nonce(),
in->get_caps_wanted(),
in->authlock.get_state(),
in->dirfragtreelock.get_state(),
in->filelock.get_state(),
in->dirlock.get_state());
- if (in->authlock.is_xlocked())
- rejoin->add_inode_xlock(in->ino(), in->authlock.get_type(),
- in->authlock.get_xlocked_by()->reqid);
- if (in->linklock.is_xlocked())
- rejoin->add_inode_xlock(in->ino(), in->linklock.get_type(),
- in->linklock.get_xlocked_by()->reqid);
- if (in->dirfragtreelock.is_xlocked())
- rejoin->add_inode_xlock(in->ino(), in->dirfragtreelock.get_type(),
- in->dirfragtreelock.get_xlocked_by()->reqid);
- if (in->filelock.is_xlocked())
- rejoin->add_inode_xlock(in->ino(), in->filelock.get_type(),
- in->filelock.get_xlocked_by()->reqid);
+ in->get_nested_dirfrags(nested);
}
-
- // dirfrags in this subtree?
- list<CDir*> dfs;
- in->get_dirfrags(dfs);
- for (list<CDir*>::iterator p = dfs.begin();
- p != dfs.end();
- ++p)
- if (!(*p)->is_subtree_root())
- nested.push_back(*p);
}
}
for (list<CDir*>::iterator p = nested.begin();
p != nested.end();
++p)
- cache_rejoin_walk(*p, rejoin);
+ rejoin_walk(*p, rejoin);
}
dout(7) << "handle_cache_rejoin " << *m << " from " << m->get_source() << endl;
switch (m->op) {
- case MMDSCacheRejoin::OP_REJOIN:
- handle_cache_rejoin_rejoin(m);
+ case MMDSCacheRejoin::OP_WEAK:
+ handle_cache_rejoin_weak(m);
+ break;
+ case MMDSCacheRejoin::OP_STRONG:
+ handle_cache_rejoin_strong(m);
break;
case MMDSCacheRejoin::OP_ACK:
handle_cache_rejoin_ack(m);
break;
-
+ case MMDSCacheRejoin::OP_PURGE:
+ handle_cache_rejoin_purge(m);
+ break;
case MMDSCacheRejoin::OP_MISSING:
handle_cache_rejoin_missing(m);
break;
delete m;
}
-void MDCache::handle_cache_rejoin_rejoin(MMDSCacheRejoin *m)
-{
- int from = m->get_source().num();
- // do immediate ack?
- MMDSCacheRejoin *ack = 0;
- MMDSCacheRejoin *missing = 0;
+/*
+ * handle_cache_rejoin_weak
+ *
+ * the sender
+ * - is recovering from their journal.
+ * - may have incorrect (out of date) inode contents
+ *
+ * if the sender didn't trim_non_auth(), they
+ * - may have incorrect (out of date) dentry/inode linkage
+ * - may have deleted/purged inodes
+ * and i may have to go to disk to get accurate inode contents. yuck.
+ */
+void MDCache::handle_cache_rejoin_weak(MMDSCacheRejoin *weak)
+{
+ int from = weak->get_source().num();
+ // possible response(s)
+ MMDSCacheRejoin *ack = 0; // if survivor
+ MMDSCacheRejoin *purge = 0; // if i'm missing something, purge it from the (recovering) sender.
+ bool survivor = false; // am i a survivor?
+
if (mds->is_active() || mds->is_stopping()) {
- dout(10) << "i am active. removing stale cache replicas" << endl;
-
- // first, scour cache of unmentioned replica references
- for (hash_map<inodeno_t,CInode*>::iterator p = inode_map.begin();
- p != inode_map.end();
+ survivor = true;
+ dout(10) << "i am a surivivor, and will ack immediately" << endl;
+ ack = new MMDSCacheRejoin(MMDSCacheRejoin::OP_ACK);
+
+ // check cap exports
+ for (map<inodeno_t,map<int,inode_caps_reconnect_t> >::iterator p = weak->cap_exports.begin();
+ p != weak->cap_exports.end();
++p) {
- // inode
- CInode *in = p->second;
- if (in->is_replica(from) && m->weak_inodes.count(p->first) == 0) {
- inode_remove_replica(in, from);
- dout(10) << " rem " << *in << endl;
+ CInode *in = get_inode(p->first);
+ if (!in || !in->is_auth()) continue;
+ for (map<int,inode_caps_reconnect_t>::iterator q = p->second.begin();
+ q != p->second.end();
+ ++q) {
+ dout(10) << " claiming cap import " << p->first << " client" << q->first << " on " << *in << endl;
+ rejoin_import_cap(in, q->first, q->second, from);
}
+ }
+ } else {
+ assert(mds->is_rejoin());
- // dentry
- if (in->parent) {
- CDentry *dn = in->parent;
- if (dn->is_replica(from) &&
- (m->weak_dentries.count(dn->get_dir()->dirfrag()) == 0 ||
- m->weak_dentries[dn->get_dir()->dirfrag()].count(dn->get_name()) == 0)) {
- dn->remove_replica(from);
- dout(10) << " rem " << *dn << endl;
- }
+ // check cap exports.
+ for (map<inodeno_t,map<int,inode_caps_reconnect_t> >::iterator p = weak->cap_exports.begin();
+ p != weak->cap_exports.end();
+ ++p) {
+ CInode *in = get_inode(p->first);
+ if (in && !in->is_auth()) continue;
+ if (!in) {
+ if (!path_is_mine(weak->cap_export_paths[p->first]))
+ continue;
+ cap_import_paths[p->first] = weak->cap_export_paths[p->first];
+ dout(10) << " noting cap import " << p->first << " path " << weak->cap_export_paths[p->first] << endl;
}
-
- // dir
- list<CDir*> dfs;
- in->get_dirfrags(dfs);
- for (list<CDir*>::iterator p = dfs.begin();
- p != dfs.end();
- ++p) {
- CDir *dir = *p;
- if (dir->is_replica(from) && m->weak_dirfrags.count(dir->dirfrag()) == 0) {
- dir->remove_replica(from);
- dout(10) << " rem " << *dir << endl;
- }
+
+ // note
+ for (map<int,inode_caps_reconnect_t>::iterator q = p->second.begin();
+ q != p->second.end();
+ ++q) {
+ dout(10) << " claiming cap import " << p->first << " client" << q->first << endl;
+ cap_imports[p->first][q->first][from] = q->second;
}
}
-
- // do immediate ack.
- ack = new MMDSCacheRejoin(MMDSCacheRejoin::OP_ACK);
}
-
- // dirs
- for (set<dirfrag_t>::iterator p = m->weak_dirfrags.begin();
- p != m->weak_dirfrags.end();
+
+ // walk weak map
+ for (map<dirfrag_t, map<string, MMDSCacheRejoin::dn_weak> >::iterator p = weak->weak.begin();
+ p != weak->weak.end();
++p) {
- CDir *dir = get_dirfrag(*p);
- if (dir) {
- int nonce = dir->add_replica(from);
- dout(10) << " have " << *dir << endl;
- if (ack)
- ack->add_strong_dirfrag(*p, nonce);
+ CDir *dir = get_dirfrag(p->first);
+ if (!dir) {
+ dout(10) << " purge " << p->first << endl;
+ if (!purge) purge = new MMDSCacheRejoin(MMDSCacheRejoin::OP_PURGE);
+ purge->add_weak_dirfrag(p->first, p->second);
+ continue;
+ }
+
+ int nonce = dir->add_replica(from);
+ dout(10) << " have " << *dir << endl;
+ if (ack)
+ ack->add_strong_dirfrag(p->first, nonce);
- // dentries
- for (set<string>::iterator q = m->weak_dentries[*p].begin();
- q != m->weak_dentries[*p].end();
- ++q) {
- CDentry *dn = dir->lookup(*q);
- if (dn) {
- int nonce = dn->add_replica(from);
- dout(10) << " have " << *dn << endl;
- if (ack)
- ack->add_strong_dentry(*p, *q, dn->lock.get_state(), nonce);
- } else {
- dout(10) << " missing " << *p << " " << *q << endl;
- if (!missing) missing = new MMDSCacheRejoin(MMDSCacheRejoin::OP_MISSING);
- missing->add_weak_dentry(*p, *q);
- }
- if (ack)
- ack->add_strong_dentry(*p, *q, nonce, dn->lock.get_state());
+ // weak dentries
+ for (map<string,MMDSCacheRejoin::dn_weak>::iterator q = p->second.begin();
+ q != p->second.end();
+ ++q) {
+ CDentry *dn = dir->lookup(q->first);
+ if (!dn ||
+ (dn->is_primary() && !q->second.is_primary())) { // make sure dn type matches, or purge
+ dout(10) << " purge " << p->first << " " << q->first << endl;
+ if (!purge) purge = new MMDSCacheRejoin(MMDSCacheRejoin::OP_PURGE);
+ purge->add_weak_null_dentry(p->first, q->first);
+ continue;
}
- } else {
- dout(10) << " missing " << *p << endl;
- if (!missing) missing = new MMDSCacheRejoin(MMDSCacheRejoin::OP_MISSING);
- missing->add_weak_dirfrag(*p);
- // dentries
- for (set<string>::iterator q = m->weak_dentries[*p].begin();
- q != m->weak_dentries[*p].end();
- ++q)
- missing->add_weak_dentry(*p, *q);
- }
- }
+ if (survivor) dentry_remove_replica(dn, from);
+ int nonce = dn->add_replica(from);
+ dout(10) << " have " << *dn << endl;
+ if (ack)
+ ack->add_strong_dentry(p->first, q->first,
+ dn->is_primary() ? dn->get_inode()->ino():inodeno_t(0),
+ dn->is_remote() ? dn->get_remote_ino():inodeno_t(0),
+ nonce, dn->lock.get_replica_state());
+
+ // inode?
+ if (dn->is_primary()) {
+ assert(q->second.is_primary()); // or we would have purged, above
+ CInode *in = dn->get_inode();
+ assert(in);
- // inodes
- for (set<inodeno_t>::iterator p = m->weak_inodes.begin();
- p != m->weak_inodes.end();
- ++p) {
- CInode *in = get_inode(*p);
- if (in) {
- int nonce = in->add_replica(from);
- in->mds_caps_wanted.erase(from);
- dout(10) << " have (weak) " << *in << endl;
- if (ack) {
- in->authlock.remove_gather(from);
- in->linklock.remove_gather(from);
- in->dirfragtreelock.remove_gather(from);
- in->filelock.remove_gather(from);
- in->dirlock.remove_gather(from);
- ack->add_strong_inode(in->ino(),
- nonce,
- 0,
- in->authlock.get_replica_state(),
- in->linklock.get_replica_state(),
- in->dirfragtreelock.get_replica_state(),
- in->filelock.get_replica_state(),
- in->dirlock.get_replica_state());
- }
- } else {
- dout(10) << " missing " << *p << endl;
- if (!missing) missing = new MMDSCacheRejoin(MMDSCacheRejoin::OP_MISSING);
- missing->add_weak_inode(*p);
- }
- }
+ if (survivor) inode_remove_replica(in, from);
+ int nonce = in->add_replica(from);
+ dout(10) << " have " << *in << endl;
- // strong inodes too?
- for (map<inodeno_t, MMDSCacheRejoin::inode_strong>::iterator p = m->strong_inodes.begin();
- p != m->strong_inodes.end();
- ++p) {
- CInode *in = get_inode(p->first);
- if (in) {
- int nonce = in->add_replica(from);
- if (p->second.caps_wanted)
- in->mds_caps_wanted[from] = p->second.caps_wanted;
- else
- in->mds_caps_wanted.erase(from);
- dout(10) << " have (strong) " << *in << endl;
- if (ack) {
- // i had inode, just tell replica the correct state
- in->authlock.remove_gather(from);
- in->linklock.remove_gather(from);
- in->dirfragtreelock.remove_gather(from);
- in->filelock.remove_gather(from);
- in->dirlock.remove_gather(from);
- ack->add_strong_inode(in->ino(),
- nonce,
- 0,
- in->authlock.get_replica_state(),
- in->linklock.get_replica_state(),
- in->dirfragtreelock.get_replica_state(),
- in->filelock.get_replica_state(),
- in->dirlock.get_replica_state());
- } else {
- // take note of replica state values.
- // SimpleLock --
- // we can ignore; locked replicas can be safely changed to sync.
- // FileLock --
- // we can also ignore.
- // replicas will at most issue RDCACHE|RD, which is covered by the default SYNC,
- // so only _locally_ opened files are significant.
- // ScatterLock -- adjust accordingly
- if (p->second.dirlock == LOCK_SCATTER ||
- p->second.dirlock == LOCK_GSCATTERS) // replica still has rdlocks
+ // scatter the dirlock, just in case?
+ if (!survivor && in->is_dir())
in->dirlock.set_state(LOCK_SCATTER);
+
+ if (ack) {
+ ack->add_full_inode(in->inode, in->symlink, in->dirfragtree);
+ ack->add_strong_inode(in->ino(),
+ nonce,
+ 0,
+ in->authlock.get_replica_state(),
+ in->linklock.get_replica_state(),
+ in->dirfragtreelock.get_replica_state(),
+ in->filelock.get_replica_state(),
+ in->dirlock.get_replica_state());
+ }
}
- } else {
- dout(10) << " missing " << p->first << endl;
- if (!missing) missing = new MMDSCacheRejoin(MMDSCacheRejoin::OP_MISSING);
- missing->add_weak_inode(p->first);
}
}
- // xlocks
- for (map<inodeno_t, map<int, metareqid_t> >::iterator p = m->xlocked_inodes.begin();
- p != m->xlocked_inodes.end();
- ++p) {
- for (map<int, metareqid_t>::iterator q = p->second.begin();
- q != p->second.end();
- q++) {
- CInode *in = get_inode(p->first);
- if (!in) continue; // already missing, from strong_inodes list above.
-
- dout(10) << " inode xlock by " << q->second << " on " << *in << endl;
-
- // create slave mdrequest
- MDRequest *mdr = request_start(q->second);
+ if (survivor)
+ rejoin_scour_survivor_replicas(from, ack);
+
+ // send purge?
+ // (before ack)
+ if (purge) {
+ assert(0); // not if sender did trim_non_auth().
+ mds->send_message_mds(purge, from, MDS_PORT_CACHE);
+ }
+
+ if (survivor) {
+ // send ack
+ mds->send_message_mds(ack, from, MDS_PORT_CACHE);
+ } else {
+ // done?
+ rejoin_gather.erase(from);
+ if (rejoin_gather.empty()) {
+ rejoin_gather_finish();
+ } else {
+ dout(7) << "still need rejoin from (" << rejoin_gather << ")" << endl;
+ }
+ }
+}
+
+
+/**
+ * parallel_fetch -- make a pass at fetching a bunch of paths in parallel
+ *
+ * @pathmap - map of inodeno to full pathnames. we remove items from this map
+ * as we discover we have them.
+ * @retry - non-completion callback context. called when a pass of fetches
+ * completes. deleted if we are done (i.e. pathmap is empty).
+ */
+bool MDCache::parallel_fetch(map<inodeno_t,string>& pathmap,
+ Context *retry)
+{
+ dout(10) << "parallel_fetch on " << pathmap.size() << " paths" << endl;
+
+ // scan list
+ set<CDir*> fetch_queue;
+ map<inodeno_t,string>::iterator p = pathmap.begin();
+ while (p != pathmap.end()) {
+ CInode *in = get_inode(p->first);
+ if (in) {
+ dout(15) << " have " << *in << endl;
+ pathmap.erase(p++);
+ continue;
+ }
+
+ // traverse
+ dout(17) << " missing " << p->first << " at " << p->second << endl;
+ filepath path(p->second);
+ CDir *dir = path_traverse_to_dir(path);
+ assert(dir);
+ fetch_queue.insert(dir);
+ p++;
+ }
+
+ if (pathmap.empty()) {
+ dout(10) << "parallel_fetch done" << endl;
+ assert(fetch_queue.empty());
+ delete retry;
+ return true;
+ }
+
+ // do a parallel fetch
+ C_Gather *gather = new C_Gather(retry);
+ for (set<CDir*>::iterator p = fetch_queue.begin();
+ p != fetch_queue.end();
+ ++p) {
+ dout(10) << "parallel_fetch fetching " << **p << endl;
+ (*p)->fetch(gather->new_sub());
+ }
+
+ return false;
+}
+
+
+
+/*
+ * rejoin_scour_survivor_replica - remove source from replica list on unmentioned objects
+ *
+ * all validated replicas are acked with a strong nonce, etc. if that isn't in the
+ * ack, the replica dne, and we can remove it from our replica maps.
+ */
+void MDCache::rejoin_scour_survivor_replicas(int from, MMDSCacheRejoin *ack)
+{
+ dout(10) << "rejoin_scour_survivor_replicas from mds" << from << endl;
+
+ // FIXME: what about root and stray inodes.
+
+ for (hash_map<inodeno_t,CInode*>::iterator p = inode_map.begin();
+ p != inode_map.end();
+ ++p) {
+ CInode *in = p->second;
+
+ // inode?
+ if (in->is_auth() &&
+ in->is_replica(from) &&
+ ack->strong_inodes.count(p->second->ino()) == 0) {
+ inode_remove_replica(in, from);
+ dout(10) << " rem " << *in << endl;
+ }
+
+ if (!in->is_dir()) continue;
+
+ list<CDir*> dfs;
+ in->get_dirfrags(dfs);
+ for (list<CDir*>::iterator p = dfs.begin();
+ p != dfs.end();
+ ++p) {
+ CDir *dir = *p;
- // auth_pin
- mdr->auth_pin(in);
+ if (dir->is_auth() &&
+ dir->is_replica(from) &&
+ ack->strong_dirfrags.count(dir->dirfrag())) {
+ dir->remove_replica(from);
+ dout(10) << " rem " << *dir << endl;
+ }
- // xlock
- SimpleLock *lock = in->get_lock(q->first);
- lock->set_state(LOCK_LOCK);
- lock->get_xlock(mdr);
- mdr->xlocks.insert(lock);
- mdr->locks.insert(lock);
+ // dentries
+ for (map<string,CDentry*>::iterator p = dir->items.begin();
+ p != dir->items.end();
+ ++p) {
+ CDentry *dn = p->second;
+
+ if (dn->is_replica(from) &&
+ (ack->strong_dentries.count(dir->dirfrag()) == 0 ||
+ ack->strong_dentries[dir->dirfrag()].count(dn->get_name()) == 0)) {
+ dentry_remove_replica(dn, from);
+ dout(10) << " rem " << *dn << endl;
+ }
+ }
}
}
- for (map<dirfrag_t, map<string, metareqid_t> >::iterator p = m->xlocked_dentries.begin();
- p != m->xlocked_dentries.end();
+}
+
+
+CInode *MDCache::rejoin_invent_inode(inodeno_t ino)
+{
+ CInode *in = new CInode(this);
+ memset(&in->inode, 0, sizeof(inode_t));
+ in->inode.ino = ino;
+ in->state_set(CInode::STATE_REJOINUNDEF);
+ add_inode(in);
+ rejoin_undef_inodes.insert(in);
+ dout(10) << " invented " << *in << endl;
+ return in;
+}
+
+
+void MDCache::handle_cache_rejoin_strong(MMDSCacheRejoin *strong)
+{
+ int from = strong->get_source().num();
+
+ // only a recovering node will get a strong rejoin.
+ assert(mds->is_rejoin());
+
+ MMDSCacheRejoin *missing = 0; // if i'm missing something..
+
+ // strong dirfrags/dentries.
+ // also process auth_pins, xlocks.
+ for (map<dirfrag_t, MMDSCacheRejoin::dirfrag_strong>::iterator p = strong->strong_dirfrags.begin();
+ p != strong->strong_dirfrags.end();
++p) {
CDir *dir = get_dirfrag(p->first);
- if (!dir) continue; // already missing, from above.
- for (map<string, metareqid_t>::iterator q = p->second.begin();
- q != p->second.end();
+ if (!dir) {
+ CInode *in = get_inode(p->first.ino);
+ if (!in) in = rejoin_invent_inode(p->first.ino);
+ if (!in->is_dir()) {
+ assert(in->state_test(CInode::STATE_REJOINUNDEF));
+ in->inode.mode = INODE_MODE_DIR;
+ }
+ dir = in->get_or_open_dirfrag(this, p->first.frag);
+ } else {
+ dir->add_replica(from);
+ dout(10) << " have " << *dir << endl;
+ }
+
+ for (map<string,MMDSCacheRejoin::dn_strong>::iterator q = strong->strong_dentries[p->first].begin();
+ q != strong->strong_dentries[p->first].end();
++q) {
CDentry *dn = dir->lookup(q->first);
- if (!dn) continue; // already missing, from above.
- dout(10) << " dn xlock by " << q->second << " on " << *dn << endl;
-
- // create slave mdrequest
- MDRequest *mdr = request_start(q->second);
+ if (!dn) {
+ if (q->second.is_remote()) {
+ dn = dir->add_dentry(q->first, q->second.remote_ino);
+ } else if (q->second.is_null()) {
+ dn = dir->add_dentry(q->first);
+ } else {
+ CInode *in = get_inode(q->second.ino);
+ if (!in) in = rejoin_invent_inode(q->second.ino);
+ dn = dir->add_dentry(q->first, in);
- // auth_pin
- mdr->auth_pin(dn->dir);
+ dout(10) << " missing " << q->second.ino << endl;
+ if (!missing) missing = new MMDSCacheRejoin(MMDSCacheRejoin::OP_MISSING);
+ missing->add_weak_inode(q->second.ino); // we want it back!
+ }
+ dout(10) << " invented " << *dn << endl;
+ }
- // xlock
- dn->lock.set_state(LOCK_LOCK);
- dn->lock.get_xlock(mdr);
- mdr->xlocks.insert(&dn->lock);
- mdr->locks.insert(&dn->lock);
+ // dn auth_pin?
+ if (strong->authpinned_dentries.count(p->first) &&
+ strong->authpinned_dentries[p->first].count(q->first)) {
+ metareqid_t ri = strong->authpinned_dentries[p->first][q->first];
+ dout(10) << " dn authpin by " << ri << " on " << *dn << endl;
+
+ // get/create slave mdrequest
+ MDRequest *mdr;
+ if (have_request(ri))
+ mdr = request_get(ri);
+ else
+ mdr = request_start_slave(ri, from);
+ mdr->auth_pin(dn);
+ }
+
+ // dn xlock?
+ if (strong->xlocked_dentries.count(p->first) &&
+ strong->xlocked_dentries[p->first].count(q->first)) {
+ metareqid_t ri = strong->xlocked_dentries[p->first][q->first];
+ dout(10) << " dn xlock by " << ri << " on " << *dn << endl;
+ MDRequest *mdr = request_get(ri); // should have this from auth_pin above.
+ assert(mdr->is_auth_pinned(dn));
+ dn->lock.set_state(LOCK_LOCK);
+ dn->lock.get_xlock(mdr);
+ mdr->xlocks.insert(&dn->lock);
+ mdr->locks.insert(&dn->lock);
+ }
+
+ dn->add_replica(from);
+ dout(10) << " have " << *dn << endl;
+
+ // inode?
+ if (dn->is_primary()) {
+ CInode *in = dn->get_inode();
+ assert(in);
+ assert(strong->strong_inodes.count(in->ino()));
+ MMDSCacheRejoin::inode_strong &is = strong->strong_inodes[in->ino()];
+
+ // caps_wanted
+ if (is.caps_wanted) {
+ in->mds_caps_wanted[from] = is.caps_wanted;
+ dout(15) << " inode caps_wanted " << cap_string(is.caps_wanted)
+ << " on " << *in << endl;
+ }
+
+ // scatterlock?
+ if (is.dirlock == LOCK_SCATTER ||
+ is.dirlock == LOCK_GLOCKC) // replica still has wrlocks
+ in->dirlock.set_state(LOCK_SCATTER);
+
+ // auth pin?
+ if (strong->authpinned_inodes.count(in->ino())) {
+ metareqid_t ri = strong->authpinned_inodes[in->ino()];
+ dout(10) << " inode authpin by " << ri << " on " << *in << endl;
+
+ // get/create slave mdrequest
+ MDRequest *mdr;
+ if (have_request(ri))
+ mdr = request_get(ri);
+ else
+ mdr = request_start_slave(ri, from);
+ mdr->auth_pin(in);
+ }
+
+ // xlock(s)?
+ if (strong->xlocked_inodes.count(in->ino())) {
+ for (map<int,metareqid_t>::iterator r = strong->xlocked_inodes[in->ino()].begin();
+ r != strong->xlocked_inodes[in->ino()].end();
+ ++r) {
+ SimpleLock *lock = in->get_lock(r->first);
+ dout(10) << " inode xlock by " << r->second << " on " << *lock << " on " << *in << endl;
+ MDRequest *mdr = request_get(r->second); // should have this from auth_pin above.
+ assert(mdr->is_auth_pinned(in));
+ lock->set_state(LOCK_LOCK);
+ lock->get_xlock(mdr);
+ mdr->xlocks.insert(lock);
+ mdr->locks.insert(lock);
+ }
+ }
+
+ in->add_replica(from);
+ dout(10) << " have " << *in << endl;
+ }
}
}
-
- // send ack?
- if (ack)
- mds->send_message_mds(ack, from, MDS_PORT_CACHE);
- else
- want_rejoin_ack.insert(from);
// send missing?
- if (missing)
+ if (missing) {
mds->send_message_mds(missing, from, MDS_PORT_CACHE);
+ } else {
+ // done?
+ rejoin_gather.erase(from);
+ if (rejoin_gather.empty()) {
+ rejoin_gather_finish();
+ } else {
+ dout(7) << "still need rejoin from (" << rejoin_gather << ")" << endl;
+ }
+ }
}
-void MDCache::handle_cache_rejoin_ack(MMDSCacheRejoin *m)
+void MDCache::handle_cache_rejoin_ack(MMDSCacheRejoin *ack)
{
- dout(7) << "handle_cache_rejoin_ack from " << m->get_source() << endl;
- int from = m->get_source().num();
+ dout(7) << "handle_cache_rejoin_ack from " << ack->get_source() << endl;
+ int from = ack->get_source().num();
+
+ bool rejoin = mds->is_rejoin();
+
+ list<Context*> waiters;
// dirs
- for (map<dirfrag_t, MMDSCacheRejoin::dirfrag_strong>::iterator p = m->strong_dirfrags.begin();
- p != m->strong_dirfrags.end();
+ for (map<dirfrag_t, MMDSCacheRejoin::dirfrag_strong>::iterator p = ack->strong_dirfrags.begin();
+ p != ack->strong_dirfrags.end();
++p) {
CDir *dir = get_dirfrag(p->first);
- assert(dir);
+ if (!dir) continue;
dir->set_replica_nonce(p->second.nonce);
dir->state_clear(CDir::STATE_REJOINING);
dout(10) << " got " << *dir << endl;
// dentries
- for (map<string,MMDSCacheRejoin::dn_strong>::iterator q = m->strong_dentries[p->first].begin();
- q != m->strong_dentries[p->first].end();
+ for (map<string,MMDSCacheRejoin::dn_strong>::iterator q = ack->strong_dentries[p->first].begin();
+ q != ack->strong_dentries[p->first].end();
++q) {
CDentry *dn = dir->lookup(q->first);
- assert(dn);
+ if (!dn) continue;
+
dn->set_replica_nonce(q->second.nonce);
- dn->lock.set_state(q->second.lock);
+ mds->locker->rejoin_set_state(&dn->lock, q->second.lock, waiters);
dn->state_clear(CDentry::STATE_REJOINING);
dout(10) << " got " << *dn << endl;
}
}
+ // full inodes
+ if (rejoin) {
+ for (list<MMDSCacheRejoin::inode_full>::iterator p = ack->full_inodes.begin();
+ p != ack->full_inodes.end();
+ ++p) {
+ CInode *in = get_inode(p->inode.ino);
+ if (!in) continue;
+ in->inode = p->inode;
+ in->symlink = p->symlink;
+ in->dirfragtree = p->dirfragtree;
+ dout(10) << " got inode content " << *in << endl;
+ }
+ }
+
// inodes
- for (map<inodeno_t, MMDSCacheRejoin::inode_strong>::iterator p = m->strong_inodes.begin();
- p != m->strong_inodes.end();
+ for (map<inodeno_t, MMDSCacheRejoin::inode_strong>::iterator p = ack->strong_inodes.begin();
+ p != ack->strong_inodes.end();
++p) {
CInode *in = get_inode(p->first);
- assert(in);
+ if (!in) continue;
in->set_replica_nonce(p->second.nonce);
- in->authlock.set_state(p->second.authlock);
- in->linklock.set_state(p->second.linklock);
- in->dirfragtreelock.set_state(p->second.dirfragtreelock);
- in->filelock.set_state(p->second.filelock);
- in->dirlock.set_state(p->second.dirlock);
+ mds->locker->rejoin_set_state(&in->authlock, p->second.authlock, waiters);
+ mds->locker->rejoin_set_state(&in->linklock, p->second.linklock, waiters);
+ mds->locker->rejoin_set_state(&in->dirfragtreelock, p->second.dirfragtreelock, waiters);
+ mds->locker->rejoin_set_state(&in->filelock, p->second.filelock, waiters);
+ mds->locker->rejoin_set_state(&in->dirlock, p->second.dirlock, waiters);
in->state_clear(CInode::STATE_REJOINING);
dout(10) << " got " << *in << endl;
}
// done?
rejoin_ack_gather.erase(from);
if (mds->is_rejoin() &&
+ rejoin_gather.empty() && // make sure we've gotten our FULL inodes, too.
rejoin_ack_gather.empty()) {
- dout(7) << "all done, going active!" << endl;
- send_cache_rejoin_acks();
-
- show_subtrees();
- show_cache();
- mds->set_want_state(MDSMap::STATE_ACTIVE);
+ mds->rejoin_done();
} else {
- dout(7) << "still need rejoin_ack from " << rejoin_ack_gather << endl;
+ dout(7) << "still need rejoin from (" << rejoin_gather << ")"
+ << ", rejoin_ack from (" << rejoin_ack_gather << ")" << endl;
}
-
}
-void MDCache::handle_cache_rejoin_missing(MMDSCacheRejoin *m)
+void MDCache::handle_cache_rejoin_purge(MMDSCacheRejoin *purge)
{
- dout(7) << "handle_cache_rejoin_missing from " << m->get_source() << endl;
-
- MMDSCacheRejoin *full = new MMDSCacheRejoin(MMDSCacheRejoin::OP_FULL);
+ dout(7) << "handle_cache_rejoin_purge from " << purge->get_source() << endl;
+ assert(mds->is_rejoin());
- // dirs
- for (set<dirfrag_t>::iterator p = m->weak_dirfrags.begin();
- p != m->weak_dirfrags.end();
+ /*
+ * this is tricky, because we have to trim our cache
+ * in a particular order, and our input (purge->weak) is sorted
+ * by dirfrag_t.
+ *
+ * so, we carelessly trim, and assuming disconnected inodes will be
+ * clean in the end...
+ */
+ set<CInode*> disconnected;
+
+ for (map<dirfrag_t, map<string, MMDSCacheRejoin::dn_weak> >::iterator p = purge->weak.begin();
+ p != purge->weak.end();
++p) {
- CDir *dir = get_dirfrag(*p);
- if (!dir) {
- dout(10) << " don't have dirfrag " << *p << endl;
- continue; // we must have trimmed it after the original rejoin
- }
+ CDir *dir = get_dirfrag(p->first);
+ assert(dir);
- dout(10) << " sending " << *dir << endl;
-
// dentries
- for (set<string>::iterator q = m->weak_dentries[*p].begin();
- q != m->weak_dentries[*p].end();
+ for (map<string,MMDSCacheRejoin::dn_weak>::iterator q = p->second.begin();
+ q != p->second.end();
++q) {
- CDentry *dn = dir->lookup(*q);
- if (!dn) {
- dout(10) << " don't have dentry " << *q << " in " << *dir << endl;
- continue; // we must have trimmed it after our original rejoin
+ CDentry *dn = dir->lookup(q->first);
+ assert(dn);
+
+ if (dn->is_primary()) {
+ CInode *in = dn->get_inode();
+ dir->unlink_inode(dn);
+
+ if (in->has_dirfrags()) {
+ dout(10) << " disconnecting inode with dirfrags " << *in << endl;
+ disconnected.insert(in);
+ } else {
+ dout(10) << " removing " << *in << endl;
+ remove_inode(in);
+ }
+ }
+
+ dout(10) << " removing " << *dn << endl;
+ dir->remove_dentry(dn);
+ }
+
+ if (dir->items.empty()) {
+ // purge the dir, too.
+ CInode *diri = dir->get_inode();
+
+ dout(10) << " closing dirfrag " << *dir << endl;
+ diri->close_dirfrag(dir->dirfrag().frag);
+
+ // FIXME: what about root, stray.
+
+ if (!diri->get_parent_dn() &&
+ !diri->has_dirfrags()) {
+ dout(10) << " removing " << *diri << endl;
+ remove_inode(diri);
+ disconnected.erase(diri);
}
- dout(10) << " sending " << *dn << endl;
- if (mds->is_rejoin())
- full->add_weak_dentry(*p, *q);
- else
- full->add_strong_dentry(*p, *q, dn->get_replica_nonce(), dn->lock.get_state());
}
}
-
+
+ for (set<CInode*>::iterator p = disconnected.begin();
+ p != disconnected.end();
+ ++p)
+ dout(0) << " PROBLEM: still have disconnected dir inode " << **p << endl;
+ assert(disconnected.empty());
+}
+
+
+void MDCache::handle_cache_rejoin_missing(MMDSCacheRejoin *missing)
+{
+ dout(7) << "handle_cache_rejoin_missing from " << missing->get_source() << endl;
+
+ MMDSCacheRejoin *full = new MMDSCacheRejoin(MMDSCacheRejoin::OP_FULL);
+
// inodes
- for (set<inodeno_t>::iterator p = m->weak_inodes.begin();
- p != m->weak_inodes.end();
+ for (set<inodeno_t>::iterator p = missing->weak_inodes.begin();
+ p != missing->weak_inodes.end();
++p) {
CInode *in = get_inode(*p);
if (!in) {
dout(10) << " sending " << *in << endl;
full->add_full_inode(in->inode, in->symlink, in->dirfragtree);
- if (mds->is_rejoin())
- full->add_weak_inode(in->ino());
- else
- full->add_strong_inode(in->ino(),
- in->get_replica_nonce(),
- in->get_caps_wanted(),
- in->authlock.get_replica_state(),
- in->linklock.get_replica_state(),
- in->dirfragtreelock.get_replica_state(),
- in->filelock.get_replica_state(),
- in->dirlock.get_replica_state());
}
- mds->send_message_mds(full, m->get_source().num(), MDS_PORT_CACHE);
+ mds->send_message_mds(full, missing->get_source().num(), MDS_PORT_CACHE);
}
-void MDCache::handle_cache_rejoin_full(MMDSCacheRejoin *m)
+void MDCache::handle_cache_rejoin_full(MMDSCacheRejoin *full)
{
- dout(7) << "handle_cache_rejoin_full from " << m->get_source() << endl;
-
+ dout(7) << "handle_cache_rejoin_full from " << full->get_source() << endl;
+ int from = full->get_source().num();
- assert(0); // write me
+ // integrate full inodes
+ for (list<MMDSCacheRejoin::inode_full>::iterator p = full->full_inodes.begin();
+ p != full->full_inodes.end();
+ ++p) {
+ CInode *in = get_inode(p->inode.ino);
+ assert(in);
+ set<CInode*>::iterator q = rejoin_undef_inodes.find(in);
+ if (q != rejoin_undef_inodes.end()) {
+ CInode *in = *q;
+ in->inode = p->inode;
+ in->symlink = p->symlink;
+ in->dirfragtree = p->dirfragtree;
+ in->state_clear(CInode::STATE_REJOINUNDEF);
+ dout(10) << " got full " << *in << endl;
+ rejoin_undef_inodes.erase(q);
+ } else {
+ dout(10) << " had full " << *in << endl;
+ }
+ }
- delete m;
+ // done?
+ rejoin_gather.erase(from);
+ if (rejoin_gather.empty()) {
+ rejoin_gather_finish();
+ } else {
+ dout(7) << "still need rejoin from (" << rejoin_gather << ")" << endl;
+ }
}
-void MDCache::send_cache_rejoin_acks()
+
+
+/**
+ * rejoin_trim_undef_inodes() -- remove REJOINUNDEF flagged inodes
+ *
+ * FIXME: wait, can this actually happen? a survivor should generate cache trim
+ * messages that clean these guys up...
+ */
+void MDCache::rejoin_trim_undef_inodes()
{
- dout(7) << "send_cache_rejoin_acks to " << want_rejoin_ack << endl;
-
+ dout(10) << "rejoin_trim_undef_inodes" << endl;
+
+ set<CInode*>::iterator p = rejoin_undef_inodes.begin();
+ while (p != rejoin_undef_inodes.end()) {
+ CInode *in = *p;
+ in->clear_replicas();
+
+ // close out dirfrags
+ if (in->is_dir()) {
+ list<CDir*> dfls;
+ in->get_dirfrags(dfls);
+ for (list<CDir*>::iterator p = dfls.begin();
+ p != dfls.end();
+ ++p) {
+ CDir *dir = *p;
+ dir->clear_replicas();
+
+ for (map<string,CDentry*>::iterator p = dir->items.begin();
+ p != dir->items.end();
+ ++p) {
+ CDentry *dn = p->second;
+ dn->clear_replicas();
+
+ dout(10) << " trimming " << *dn << endl;
+ dir->remove_dentry(dn);
+ }
+
+ dout(10) << " trimming " << *dir << endl;
+ in->close_dirfrag(dir->dirfrag().frag);
+ }
+ }
+
+ CDentry *dn = in->get_parent_dn();
+ if (dn) {
+ dn->clear_replicas();
+ dout(10) << " trimming " << *dn << endl;
+ dn->dir->remove_dentry(dn);
+ } else {
+ dout(10) << " trimming " << *in << endl;
+ remove_inode(in);
+ }
+ }
+
+ assert(rejoin_undef_inodes.empty()); // hmm: this shouldn't ever happen, actually!
+ rejoin_undef_inodes.clear();
+}
+
+class C_MDC_RejoinGatherFinish : public Context {
+ MDCache *cache;
+public:
+ C_MDC_RejoinGatherFinish(MDCache *c) : cache(c) {}
+ void finish(int r) {
+ cache->rejoin_gather_finish();
+ }
+};
+
+void MDCache::rejoin_gather_finish()
+{
+ dout(10) << "rejoin_gather_finish" << endl;
assert(mds->is_rejoin());
- /* nope, not necessary, we adjust lock state gradually.
- after we've processed all rejoins, lockstate is legal.
- we just have to do a final _eval-ish thing at the end...
+ rejoin_trim_undef_inodes();
- // calculate proper filelock states
- for (set<CInode*>::iterator p = filelock_replica_readers.begin();
- p != filelock_replica_readers.end();
+ // fetch paths?
+ if (!cap_import_paths.empty() &&
+ !parallel_fetch(cap_import_paths, new C_MDC_RejoinGatherFinish(this)))
+ return;
+
+ // process cap imports
+ // ino -> client -> frommds -> capex
+ for (map<inodeno_t,map<int, map<int,inode_caps_reconnect_t> > >::iterator p = cap_imports.begin();
+ p != cap_imports.end();
++p) {
- dout(10) << "replica(s) have RD caps on " << *p->first << endl;
-
- for (set<int>::iterator q = p->second.begin();
+ CInode *in = get_inode(p->first);
+ assert(in);
+ mds->server->add_reconnected_cap_inode(in);
+ for (map<int, map<int,inode_caps_reconnect_t> >::iterator q = p->second.begin();
q != p->second.end();
- ++q) {
- if (*q == LOCK_
- }
+ ++q)
+ for (map<int,inode_caps_reconnect_t>::iterator r = q->second.begin();
+ r != q->second.end();
+ ++r)
+ if (r->first >= 0)
+ rejoin_import_cap(in, q->first, r->second, r->first);
}
- */
+
+ mds->server->process_reconnected_caps();
+
+ rejoin_send_acks();
- // send acks
+ // did we already get our acks too?
+ // this happens when the rejoin_gather has to wait on a MISSING/FULL exchange.
+ if (rejoin_ack_gather.empty())
+ mds->rejoin_done();
+}
+
+void MDCache::rejoin_import_cap(CInode *in, int client, inode_caps_reconnect_t& icr, int frommds)
+{
+ dout(10) << "rejoin_import_cap for client" << client << " from mds" << frommds
+ << " on " << *in << endl;
+
+ // add cap
+ in->reconnect_cap(client, icr);
+
+ // send REAP
+ // FIXME client session weirdness.
+ MClientFileCaps *reap = new MClientFileCaps(MClientFileCaps::OP_REAP,
+ in->inode,
+ in->client_caps[client].get_last_seq(),
+ in->client_caps[client].pending(),
+ in->client_caps[client].wanted());
+
+ reap->set_mds( frommds ); // reap from whom?
+ mds->messenger->send_message(reap,
+ mds->clientmap.get_inst(client),
+ 0, MDS_PORT_CACHE);
+}
+
+void MDCache::rejoin_send_acks()
+{
+ dout(7) << "rejoin_send_acks" << endl;
+
+ // send acks to everyone in the recovery set
map<int,MMDSCacheRejoin*> ack;
+ set<int> weak;
+ for (set<int>::iterator p = recovery_set.begin();
+ p != recovery_set.end();
+ ++p) {
+ ack[*p] = new MMDSCacheRejoin(MMDSCacheRejoin::OP_ACK);
+ if (mds->mdsmap->is_rejoin(*p)) weak.insert(*p);
+ }
+ // walk subtrees
for (map<CDir*,set<CDir*> >::iterator p = subtrees.begin();
p != subtrees.end();
p++) {
// dir
for (map<int,int>::iterator r = dir->replicas_begin();
r != dir->replicas_end();
- ++r) {
- if (!ack[r->first]) ack[r->first] = new MMDSCacheRejoin(MMDSCacheRejoin::OP_ACK);
+ ++r)
ack[r->first]->add_strong_dirfrag(dir->dirfrag(), r->second);
- }
for (map<string,CDentry*>::iterator q = dir->items.begin();
q != dir->items.end();
// dentry
for (map<int,int>::iterator r = dn->replicas_begin();
r != dn->replicas_end();
- ++r) {
- //if (!ack[r->first]) ack[r->first] = new MMDSCacheRejoin(MMDSCacheRejoin::OP_ACK);
- ack[r->first]->add_strong_dentry(dir->dirfrag(), dn->name, r->second,
+ ++r)
+ ack[r->first]->add_strong_dentry(dir->dirfrag(), dn->name,
+ dn->is_primary() ? dn->get_inode()->ino():inodeno_t(0),
+ dn->is_remote() ? dn->get_remote_ino():inodeno_t(0),
+ r->second,
dn->lock.get_replica_state());
- }
if (!dn->is_primary()) continue;
// inode
CInode *in = dn->inode;
-
- // twiddle filelock at all?
- // hmm.
for (map<int,int>::iterator r = in->replicas_begin();
r != in->replicas_end();
++r) {
- //if (!ack[r->first]) ack[r->first] = new MMDSCacheRejoin(MMDSCacheRejoin::OP_ACK);
+ if (weak.count(r->first))
+ ack[r->first]->add_full_inode(in->inode, in->symlink, in->dirfragtree);
ack[r->first]->add_strong_inode(in->ino(), r->second, 0,
in->authlock.get_replica_state(),
in->linklock.get_replica_state(),
purging[inode->ino][newsize] = *inode;
assert(inode->size > newsize);
+ _do_purge_inode(inode, newsize);
+}
+void MDCache::_do_purge_inode(inode_t *inode, off_t newsize)
+{
// remove
- mds->filer->remove(*inode, newsize, inode->size,
- 0, new C_MDC_PurgeFinish(this, inode->ino, newsize));
-
- /*} else {
+ if (inode->size > 0) {
+ mds->filer->remove(*inode, newsize, inode->size,
+ 0, new C_MDC_PurgeFinish(this, inode->ino, newsize));
+ } else {
// no need, empty file, just log it
purge_inode_finish(inode->ino, newsize);
}
- */
}
void MDCache::purge_inode_finish(inodeno_t ino, off_t newsize)
dout(10) << "start_recovered_purges " << p->first
<< " size " << q->second.size
<< " to " << q->first << endl;
- mds->filer->remove(q->second, q->first, q->second.size,
- 0, new C_MDC_PurgeFinish(this, p->first, q->first));
+ _do_purge_inode(&q->second, q->first);
}
}
}
while (lru.lru_get_size() > (unsigned)max) {
CDentry *dn = (CDentry*)lru.lru_expire();
if (!dn) break;
-
- CDir *dir = dn->get_dir();
- assert(dir);
-
- CDir *con = get_subtree_root(dir);
- assert(con);
-
- dout(12) << "trim removing " << *dn << endl;
- dout(12) << " in container " << *con << endl;
-
- // notify dentry authority?
- if (!dn->is_auth()) {
- pair<int,int> auth = dn->authority();
-
- for (int p=0; p<2; p++) {
- int a = auth.first;
- if (p) a = auth.second;
- if (a < 0 || (p == 1 && auth.second == auth.first)) break;
- if (mds->get_nodeid() == auth.second &&
- con->is_importing()) break; // don't send any expire while importing.
- if (a == mds->get_nodeid()) continue; // on export, ignore myself.
-
- dout(12) << " sending expire to mds" << a << " on " << *dn << endl;
- assert(a != mds->get_nodeid());
- if (expiremap.count(a) == 0)
- expiremap[a] = new MCacheExpire(mds->get_nodeid());
- expiremap[a]->add_dentry(con->dirfrag(), dir->dirfrag(), dn->get_name(), dn->get_replica_nonce());
- }
- }
-
- // unlink the dentry
- if (dn->is_remote()) {
- // just unlink.
- dir->unlink_inode(dn);
- }
- else if (dn->is_primary()) {
- // expire the inode, too.
- CInode *in = dn->get_inode();
- assert(in);
- trim_inode(dn, in, con, expiremap);
- }
- else {
- assert(dn->is_null());
- }
-
- // adjust the dir state
- // NOTE: we can safely remove a clean, null dentry without effecting
- // directory completeness.
- if (!(dn->is_null() && dn->is_clean()))
- dir->state_clear(CDir::STATE_COMPLETE);
-
- // remove dentry
- dir->remove_dentry(dn);
-
- // reexport?
- if (dir->get_size() == 0 && dir->is_subtree_root())
- migrator->export_empty_import(dir);
-
- if (mds->logger) mds->logger->inc("cex");
+ trim_dentry(dn, expiremap);
}
// trim root inode+dir?
}
}
+ // send!
+ send_expire_messages(expiremap);
+
+ return true;
+}
+
+void MDCache::send_expire_messages(map<int, MCacheExpire*>& expiremap)
+{
// send expires
for (map<int, MCacheExpire*>::iterator it = expiremap.begin();
it != expiremap.end();
dout(7) << "sending cache_expire to " << it->first << endl;
mds->send_message_mds(it->second, it->first, MDS_PORT_CACHE);
}
+}
- return true;
+
+void MDCache::trim_dentry(CDentry *dn, map<int, MCacheExpire*>& expiremap)
+{
+ dout(12) << "trim_dentry " << *dn << endl;
+
+ CDir *dir = dn->get_dir();
+ assert(dir);
+
+ CDir *con = get_subtree_root(dir);
+ assert(con);
+
+ dout(12) << " in container " << *con << endl;
+
+ // notify dentry authority?
+ if (!dn->is_auth()) {
+ pair<int,int> auth = dn->authority();
+
+ for (int p=0; p<2; p++) {
+ int a = auth.first;
+ if (p) a = auth.second;
+ if (a < 0 || (p == 1 && auth.second == auth.first)) break;
+ if (mds->get_nodeid() == auth.second &&
+ con->is_importing()) break; // don't send any expire while importing.
+ if (a == mds->get_nodeid()) continue; // on export, ignore myself.
+
+ dout(12) << " sending expire to mds" << a << " on " << *dn << endl;
+ assert(a != mds->get_nodeid());
+ if (expiremap.count(a) == 0)
+ expiremap[a] = new MCacheExpire(mds->get_nodeid());
+ expiremap[a]->add_dentry(con->dirfrag(), dir->dirfrag(), dn->get_name(), dn->get_replica_nonce());
+ }
+ }
+
+ // unlink the dentry
+ if (dn->is_remote()) {
+ // just unlink.
+ dir->unlink_inode(dn);
+ }
+ else if (dn->is_primary()) {
+ // expire the inode, too.
+ CInode *in = dn->get_inode();
+ assert(in);
+ trim_inode(dn, in, con, expiremap);
+ }
+ else {
+ assert(dn->is_null());
+ }
+
+ // adjust the dir state
+ // NOTE: we can safely remove a clean, null dentry without effecting
+ // directory completeness.
+ if (!(dn->is_null() && dn->is_clean()))
+ dir->state_clear(CDir::STATE_COMPLETE);
+
+ // remove dentry
+ dir->remove_dentry(dn);
+
+ // reexport?
+ if (dir->get_size() == 0 && dir->is_subtree_root())
+ migrator->export_empty_import(dir);
+
+ if (mds->logger) mds->logger->inc("cex");
}
+
void MDCache::trim_dirfrag(CDir *dir, CDir *con, map<int, MCacheExpire*>& expiremap)
{
assert(dir->get_num_ref() == 0);
if (con)
df = con->dirfrag();
else
- df = dirfrag_t(1,frag_t());
+ df = dirfrag_t(0,frag_t()); // must be a root or stray inode.
for (int p=0; p<2; p++) {
int a = auth.first;
}
+/**
+ * trim_non_auth - remove any non-auth items from our cache
+ *
+ * this reduces the amount of non-auth metadata in our cache, reducing the
+ * load incurred by the rejoin phase.
+ *
+ * the only non-auth items that remain are those that are needed to
+ * attach our own subtrees to the root.
+ *
+ * why we have to do this:
+ * we may not have accurate linkage for non-auth items. which means we will
+ * know which subtree it falls into, and can not be sure to declare it to the
+ * correct authority.
+ */
void MDCache::trim_non_auth()
{
dout(7) << "trim_non_auth" << endl;
for (map<dirfrag_t,MCacheExpire::realm>::iterator p = m->realms.begin();
p != m->realms.end();
++p) {
- // get container
- CDir *con = get_dirfrag(p->first);
- assert(con); // we had better have this.
-
- if (!con->is_auth() ||
- (con->is_auth() && con->is_exporting() &&
- migrator->get_export_state(con) == Migrator::EXPORT_WARNING &&
- migrator->export_has_warned(con,from))) {
- // not auth.
- dout(7) << "delaying nonauth|warned expires for " << *con << endl;
- assert(con->is_frozen_tree_root());
-
- // make a message container
- if (delayed_expire[con].count(from) == 0)
- delayed_expire[con][from] = new MCacheExpire(from);
-
- // merge these expires into it
- delayed_expire[con][from]->add_realm(p->first, p->second);
- continue;
+ // check container?
+ if (p->first.ino > 0) {
+ CDir *con = get_dirfrag(p->first);
+ assert(con); // we had better have this.
+
+ if (!con->is_auth() ||
+ (con->is_auth() && con->is_exporting() &&
+ migrator->get_export_state(con) == Migrator::EXPORT_WARNING &&
+ migrator->export_has_warned(con,from))) {
+ // not auth.
+ dout(7) << "delaying nonauth|warned expires for " << *con << endl;
+ assert(con->is_frozen_tree_root());
+
+ // make a message container
+ if (delayed_expire[con].count(from) == 0)
+ delayed_expire[con][from] = new MCacheExpire(from);
+
+ // merge these expires into it
+ delayed_expire[con][from]->add_realm(p->first, p->second);
+ continue;
+ }
+ dout(7) << "expires for " << *con << endl;
+ } else {
+ dout(7) << "containerless expires (root, stray inodes)" << endl;
}
- dout(7) << "expires for " << *con << endl;
// INODES
for (map<inodeno_t,int>::iterator it = p->second.inodes.begin();
if (nonce == dn->get_replica_nonce(from)) {
dout(7) << " dentry_expire on " << *dn << " from mds" << from << endl;
- dn->remove_replica(from);
+ dentry_remove_replica(dn, from);
}
else {
dout(7) << " dentry_expire on " << *dn << " from mds" << from
// note: this code calls _eval more often than it needs to!
// fix lock
- if (in->authlock.remove_replica(from))
- mds->locker->simple_eval(&in->authlock);
- if (in->linklock.remove_replica(from))
- mds->locker->simple_eval(&in->linklock);
- if (in->dirfragtreelock.remove_replica(from))
- mds->locker->simple_eval(&in->dirfragtreelock);
- if (in->filelock.remove_replica(from))
- mds->locker->simple_eval(&in->filelock);
+ if (in->authlock.remove_replica(from)) mds->locker->simple_eval_gather(&in->authlock);
+ if (in->linklock.remove_replica(from)) mds->locker->simple_eval_gather(&in->linklock);
+ if (in->dirfragtreelock.remove_replica(from)) mds->locker->simple_eval_gather(&in->dirfragtreelock);
+ if (in->filelock.remove_replica(from)) mds->locker->file_eval_gather(&in->filelock);
+ if (in->dirlock.remove_replica(from)) mds->locker->scatter_eval_gather(&in->dirlock);
// alone now?
+ /*
if (!in->is_replicated()) {
- mds->locker->simple_eval(&in->authlock);
- mds->locker->simple_eval(&in->linklock);
- mds->locker->simple_eval(&in->dirfragtreelock);
- mds->locker->file_eval(&in->filelock);
+ mds->locker->simple_eval_gather(&in->authlock);
+ mds->locker->simple_eval_gather(&in->linklock);
+ mds->locker->simple_eval_gather(&in->dirfragtreelock);
+ mds->locker->file_eval_gather(&in->filelock);
+ mds->locker->scatter_eval_gather(&in->dirlock);
}
+ */
+}
+
+void MDCache::dentry_remove_replica(CDentry *dn, int from)
+{
+ dn->remove_replica(from);
+
+ // fix lock
+ if (dn->lock.remove_replica(from) ||
+ !dn->is_replicated())
+ mds->locker->simple_eval_gather(&dn->lock);
}
{
dout(7) << "shutdown_pass" << endl;
- if (mds->is_out()) {
+ if (mds->is_stopped()) {
dout(7) << " already shut down" << endl;
show_cache();
show_subtrees();
// send all imports back to 0.
if (!subtrees.empty() &&
mds->get_nodeid() != 0 &&
- !migrator->is_exporting() &&
- !migrator->is_importing()) {
+ !migrator->is_exporting() //&&
+ //!migrator->is_importing()
+ ) {
// export to root
dout(7) << "looking for subtrees to export to mds0" << endl;
list<CDir*> ls;
if (!subtrees.empty()) {
dout(7) << "still have " << num_subtrees() << " subtrees" << endl;
show_subtrees();
- show_cache();
+ migrator->show_importing();
+ migrator->show_exporting();
+ //show_cache();
return false;
}
assert(subtrees.empty());
// (wait for) flush log?
if (g_conf.mds_log_flush_on_shutdown) {
- if (mds->mdlog->get_non_importmap_events()) {
+ if (mds->mdlog->get_non_subtreemap_events()) {
dout(7) << "waiting for log to flush .. " << mds->mdlog->get_num_events()
- << " (" << mds->mdlog->get_non_importmap_events() << ")" << endl;
+ << " (" << mds->mdlog->get_non_subtreemap_events() << ")" << endl;
return false;
}
}
}
if (mds->mdlog->get_num_events()) {
- dout(7) << "waiting for log to flush (including import_map, now) .. " << mds->mdlog->get_num_events()
- << " (" << mds->mdlog->get_non_importmap_events() << ")" << endl;
+ dout(7) << "waiting for log to flush (including subtree_map, now) .. " << mds->mdlog->get_num_events()
+ << " (" << mds->mdlog->get_non_subtreemap_events() << ")" << endl;
return false;
}
{
switch (m->get_type()) {
- case MSG_MDS_IMPORTMAP:
- handle_import_map((MMDSImportMap*)m);
+ // RESOLVE
+ case MSG_MDS_RESOLVE:
+ handle_resolve((MMDSResolve*)m);
+ break;
+ case MSG_MDS_RESOLVEACK:
+ handle_resolve_ack((MMDSResolveAck*)m);
break;
+ // REJOIN
case MSG_MDS_CACHEREJOIN:
handle_cache_rejoin((MMDSCacheRejoin*)m);
break;
- /*
- case MSG_MDS_CACHEREJOINACK:
- handle_cache_rejoin_ack((MMDSCacheRejoinAck*)m);
- break;
- */
-
case MSG_MDS_DISCOVER:
handle_discover((MDiscover*)m);
break;
*/
- case MSG_MDS_INODELINK:
- handle_inode_link((MInodeLink*)m);
- break;
-
case MSG_MDS_DIRUPDATE:
handle_dir_update((MDirUpdate*)m);
break;
/* path_traverse
*
* return values:
- * <0 : traverse error (ENOTDIR, ENOENT)
+ * <0 : traverse error (ENOTDIR, ENOENT, etc.)
* 0 : success
* >0 : delayed or forwarded
*
- * Notes:
- * onfinish context is only needed if you specify MDS_TRAVERSE_DISCOVER _and_
- * you aren't absolutely certain that the path actually exists. If it doesn't,
- * the context is needed to pass a (failure) result code.
+ * onfail values:
+ *
+ * MDS_TRAVERSE_FORWARD - forward to auth (or best guess)
+ * MDS_TRAVERSE_DISCOVER - discover missing items. skip permission checks.
+ * MDS_TRAVERSE_DISCOVERXLOCK - discover XLOCKED items too (be careful!).
+ * MDS_TRAVERSE_FAIL - return an error
*/
-/*
-class C_MDC_TraverseDiscover : public Context {
- Context *onfinish, *ondelay;
- public:
- C_MDC_TraverseDiscover(Context *onfinish, Context *ondelay) {
- this->ondelay = ondelay;
- this->onfinish = onfinish;
- }
- void finish(int r) {
- //dout(10) << "TraverseDiscover r = " << r << endl;
- if (r < 0 && onfinish) { // ENOENT on discover, pass back to caller.
- onfinish->finish(r);
- } else {
- ondelay->finish(r); // retry as usual
- }
- delete onfinish;
- delete ondelay;
- }
-};
-*/
-
-int MDCache::path_traverse(MDRequest *mdr,
- CInode *base, // traverse starting from here.
- filepath& origpath,
- vector<CDentry*>& trace,
- bool follow_trailing_symlink,
- Message *req,
- Context *ondelay,
- int onfail,
- bool is_client_req,
- bool null_okay) // true if req is MClientRequest .. gross, FIXME
+Context *MDCache::_get_waiter(MDRequest *mdr, Message *req)
{
- set< pair<CInode*, string> > symlinks_resolved; // keep a list of symlinks we touch to avoid loops
+ if (mdr)
+ return new C_MDS_RetryRequest(this, mdr);
+ else
+ return new C_MDS_RetryMessage(mds, req);
+}
+int MDCache::path_traverse(MDRequest *mdr, Message *req, // who
+ CInode *base, filepath& origpath, // what
+ vector<CDentry*>& trace, // result
+ bool follow_trailing_symlink, // how
+ int onfail)
+{
+ assert(mdr || req);
+ bool null_okay = onfail == MDS_TRAVERSE_DISCOVERXLOCK;
bool noperm = false;
if (onfail == MDS_TRAVERSE_DISCOVER ||
- onfail == MDS_TRAVERSE_DISCOVERXLOCK) noperm = true;
+ onfail == MDS_TRAVERSE_DISCOVERXLOCK)
+ noperm = true;
+
+ // keep a list of symlinks we touch to avoid loops
+ set< pair<CInode*, string> > symlinks_resolved;
// root
CInode *cur = base;
if (!cur) cur = get_root();
if (cur == NULL) {
dout(7) << "traverse: i don't have root" << endl;
- open_root(ondelay);
+ open_root(_get_waiter(mdr, req));
return 1;
}
// ENOTDIR?
if (!cur->is_dir()) {
dout(7) << "traverse: " << *cur << " not a dir " << endl;
- delete ondelay;
return -ENOTDIR;
}
// parent dir frozen_dir?
if (cur->is_frozen_dir()) {
dout(7) << "traverse: " << *cur->get_parent_dir() << " is frozen_dir, waiting" << endl;
- cur->get_parent_dir()->add_waiter(CDir::WAIT_UNFREEZE, ondelay);
+ cur->get_parent_dir()->add_waiter(CDir::WAIT_UNFREEZE, _get_waiter(mdr, req));
return 1;
}
} else {
// discover?
assert(!cur->is_auth());
- if (cur->is_waiter_for(CInode::WAIT_DIR)) {
- dout(10) << "traverse: need dir, already doing discover for " << *cur << endl;
- }
- else if (cur->is_ambiguous_auth()) {
+ if (cur->is_ambiguous_auth()) {
dout(10) << "traverse: need dir, waiting for single auth on " << *cur << endl;
- cur->add_waiter(CInode::WAIT_SINGLEAUTH, ondelay);
+ cur->add_waiter(CInode::WAIT_SINGLEAUTH, _get_waiter(mdr, req));
return 1;
+ } else if (dir_discovers.count(cur->ino())) {
+ dout(10) << "traverse: need dir, already doing discover for " << *cur << endl;
+ assert(cur->is_waiter_for(CInode::WAIT_DIR));
} else {
filepath want = path.postfixpath(depth);
dout(10) << "traverse: need dir, doing discover, want " << want.get_path()
mds->send_message_mds(new MDiscover(mds->get_nodeid(),
cur->ino(),
want,
- true), // need this dir too
+ true, // need this dir!
+ onfail == MDS_TRAVERSE_DISCOVERXLOCK),
cur->authority().first, MDS_PORT_CACHE);
dir_discovers[cur->ino()].insert(cur->authority().first);
}
- cur->add_waiter(CInode::WAIT_DIR, ondelay);
+ cur->add_waiter(CInode::WAIT_DIR, _get_waiter(mdr, req));
return 1;
}
}
// frozen?
/*
if (curdir->is_frozen()) {
- // doh!
+ // doh!
// FIXME: traverse is allowed?
dout(7) << "traverse: " << *curdir << " is frozen, waiting" << endl;
- curdir->add_waiter(CDir::WAIT_UNFREEZE, ondelay);
+ curdir->add_waiter(CDir::WAIT_UNFREEZE, _get_waiter(mdr, req));
if (onfinish) delete onfinish;
return 1;
}
*/
// must read directory hard data (permissions, x bit) to traverse
- if (!noperm && !mds->locker->simple_rdlock_try(&cur->authlock, ondelay)) {
+ if (!noperm &&
+ !mds->locker->simple_rdlock_try(&cur->authlock, _get_waiter(mdr, req)))
return 1;
- }
// check permissions?
// XXX
// dentry exists. xlocked?
if (!noperm && dn->lock.is_xlocked() && dn->lock.get_xlocked_by() != mdr) {
dout(10) << "traverse: xlocked dentry at " << *dn << endl;
- dn->lock.add_waiter(SimpleLock::WAIT_RD, ondelay);
+ dn->lock.add_waiter(SimpleLock::WAIT_RD, _get_waiter(mdr, req));
return 1;
}
dn->link_remote(in);
} else {
dout(7) << "remote link to " << dn->get_remote_ino() << ", which i don't have" << endl;
- assert(0); // REWRITE ME
- //open_remote_ino(dn->get_remote_ino(), req, ondelay);
+ assert(mdr); // we shouldn't hit non-primary dentries doing a non-mdr traversal!
+ open_remote_ino(dn->get_remote_ino(), mdr, _get_waiter(mdr, req));
return 1;
}
}
}
// forwarder wants replicas?
- if (is_client_req && ((MClientRequest*)req)->get_mds_wants_replica_in_dirino()) {
- dout(30) << "traverse: REP is here, " << ((MClientRequest*)req)->get_mds_wants_replica_in_dirino() << " vs " << curdir->dirfrag() << endl;
+ if (mdr && mdr->client_request &&
+ mdr->client_request->get_mds_wants_replica_in_dirino()) {
+ dout(30) << "traverse: REP is here, "
+ << mdr->client_request->get_mds_wants_replica_in_dirino()
+ << " vs " << curdir->dirfrag() << endl;
- if (((MClientRequest*)req)->get_mds_wants_replica_in_dirino() == curdir->ino() &&
+ if (mdr->client_request->get_mds_wants_replica_in_dirino() == curdir->ino() &&
curdir->is_auth() &&
curdir->is_rep() &&
curdir->is_replica(req->get_source().num()) &&
// dentry is mine.
if (curdir->is_complete()) {
// file not found
- delete ondelay;
return -ENOENT;
} else {
-
- //wrong?
- //if (onfail == MDS_TRAVERSE_DISCOVER)
- // return -1;
-
- // directory isn't complete; reload
+ // directory isn't complete; reload
dout(7) << "traverse: incomplete dir contents for " << *cur << ", fetching" << endl;
touch_inode(cur);
- curdir->fetch(ondelay);
-
- if (mds->logger) mds->logger->inc("cmiss");
-
+ curdir->fetch(_get_waiter(mdr, req));
+ if (mds->logger) mds->logger->inc("cmiss");
return 1;
}
} else {
}
else if (curdir->is_ambiguous_auth()) {
dout(7) << "traverse: waiting for single auth on " << *curdir << endl;
- curdir->add_waiter(CDir::WAIT_SINGLEAUTH, ondelay);
+ curdir->add_waiter(CDir::WAIT_SINGLEAUTH, _get_waiter(mdr, req));
return 1;
- } else {
+ }
+ else {
dout(7) << "traverse: discover " << want << " from " << *curdir << endl;
touch_inode(cur);
mds->send_message_mds(new MDiscover(mds->get_nodeid(),
cur->ino(),
want,
- false),
+ false,
+ onfail == MDS_TRAVERSE_DISCOVERXLOCK),
dauth.first, MDS_PORT_CACHE);
if (mds->logger) mds->logger->inc("dis");
}
// delay processing of current request.
- curdir->add_dentry_waiter(path[depth], ondelay);
+ curdir->add_dentry_waiter(path[depth], _get_waiter(mdr, req));
if (mds->logger) mds->logger->inc("cmiss");
return 1;
}
if (curdir->is_ambiguous_auth()) {
// wait
dout(7) << "traverse: waiting for single auth in " << *curdir << endl;
- curdir->add_waiter(CDir::WAIT_SINGLEAUTH, ondelay);
+ curdir->add_waiter(CDir::WAIT_SINGLEAUTH, _get_waiter(mdr, req));
return 1;
} else {
dout(7) << "traverse: forwarding, not auth for " << *curdir << endl;
// request replication?
- if (is_client_req && curdir->is_rep()) {
+ if (mdr && mdr->client_request && curdir->is_rep()) {
dout(15) << "traverse: REP fw to mds" << dauth << ", requesting rep under "
<< *curdir << " req " << *(MClientRequest*)req << endl;
- ((MClientRequest*)req)->set_mds_wants_replica_in_dirino(curdir->ino());
+ mdr->client_request->set_mds_wants_replica_in_dirino(curdir->ino());
req->clear_payload(); // reencode!
}
- mds->forward_message_mds(req, dauth.first, req->get_dest_port());
+ if (mdr)
+ request_forward(mdr, dauth.first, req->get_dest_port());
+ else
+ mds->forward_message_mds(req, dauth.first, req->get_dest_port());
if (mds->logger) mds->logger->inc("cfw");
- delete ondelay;
return 2;
}
}
if (onfail == MDS_TRAVERSE_FAIL) {
- delete ondelay;
return -ENOENT; // not necessarily exactly true....
}
}
}
// success.
- delete ondelay;
return 0;
}
+bool MDCache::path_is_mine(filepath& path)
+{
+ dout(15) << "path_is_mine " << path << endl;
+
+ // start at root. FIXME.
+ CInode *cur = root;
+ assert(cur);
+
+ for (unsigned i=0; i<path.depth(); i++) {
+ dout(15) << "path_is_mine seg " << i << ": " << path[i] << " under " << *cur << endl;
+ frag_t fg = cur->pick_dirfrag(path[i]);
+ CDir *dir = cur->get_dirfrag(fg);
+ if (!dir) return cur->is_auth();
+ CDentry *dn = dir->lookup(path[i]);
+ if (!dn) return dir->is_auth();
+ assert(dn->is_primary());
+ cur = dn->get_inode();
+ }
+
+ return cur->is_auth();
+}
+
+/**
+ * path_traverse_to_dir -- traverse to deepest dir we have
+ *
+ * @path - path to traverse (as far as we can)
+ *
+ * assumes we _don't_ have the full path. (if we do, we return NULL.)
+ */
+CDir *MDCache::path_traverse_to_dir(filepath& path)
+{
+ CInode *cur = root;
+ assert(cur);
+ for (unsigned i=0; i<path.depth(); i++) {
+ dout(20) << "path_traverse_to_dir seg " << i << ": " << path[i] << " under " << *cur << endl;
+ frag_t fg = cur->pick_dirfrag(path[i]);
+ CDir *dir = cur->get_or_open_dirfrag(this, fg);
+ CDentry *dn = dir->lookup(path[i]);
+ if (!dn) return dir;
+ assert(dn->is_primary());
+ cur = dn->get_inode();
+ }
+
+ return NULL; // oh, we have the full path.
+}
+
void MDCache::open_remote_dir(CInode *diri, frag_t fg, Context *fin)
}
-/** get_dentry_inode
+/**
+ * get_dentry_inode - get or open inode
+ *
+ * @dn the dentry
+ * @mdr current request
+ *
* will return inode for primary, or link up/open up remote link's inode as necessary.
*/
CInode *MDCache::get_dentry_inode(CDentry *dn, MDRequest *mdr)
dn->link_remote(in);
return in;
} else {
- dout(10) << "get_dentry_ninode on remote dn, opening inode for " << *dn << endl;
+ dout(10) << "get_dentry_inode on remote dn, opening inode for " << *dn << endl;
open_remote_ino(dn->get_remote_ino(), mdr, new C_MDS_RetryRequest(this, mdr));
return 0;
}
CInode *in = 0;
while (1) {
// inode?
- CInode *in = get_inode(anchortrace[i-1].ino);
+ dout(10) << " " << i << ": " << anchortrace[i-1] << endl;
+ in = get_inode(anchortrace[i-1].ino);
if (in) break;
i--;
if (!i) {
- in = root;
+ in = get_inode(anchortrace[i].dirfrag.ino);
+ assert(in); // actually, we may need to open the root or a foreign stray inode, here.
break;
}
}
}
-MDRequest *MDCache::request_start(metareqid_t ri)
-{
- MDRequest *mdr = new MDRequest(ri);
- active_requests[mdr->reqid] = mdr;
- dout(7) << "request_start " << *mdr << endl;
- return mdr;
-}
-
MDRequest *MDCache::request_start(MClientRequest *req)
{
+ // did we win a forward race against a slave?
+ if (active_requests.count(req->get_reqid())) {
+ MDRequest *mdr = active_requests[req->get_reqid()];
+ dout(10) << "request_start already had " << *mdr << ", cleaning up" << endl;
+ assert(mdr->is_slave());
+ request_cleanup(mdr);
+ delete mdr;
+ }
+
+ // register new client request
MDRequest *mdr = new MDRequest(req->get_reqid(), req);
- active_requests[mdr->reqid] = mdr;
+ active_requests[req->get_reqid()] = mdr;
dout(7) << "request_start " << *mdr << endl;
return mdr;
}
-MDRequest *MDCache::request_start(MLock *req)
+MDRequest *MDCache::request_start_slave(metareqid_t ri, int by)
{
- MDRequest *mdr = new MDRequest(req->get_reqid(), req);
- mdr->by_mds = req->get_source().num();
+ MDRequest *mdr = new MDRequest(ri, by);
+ assert(active_requests.count(mdr->reqid) == 0);
active_requests[mdr->reqid] = mdr;
- dout(7) << "request_start " << *mdr << endl;
+ dout(7) << "request_start_slave " << *mdr << " by mds" << by << endl;
return mdr;
}
+
MDRequest *MDCache::request_get(metareqid_t rid)
{
assert(active_requests.count(rid));
{
dout(7) << "request_finish " << *mdr << endl;
- delete mdr->request;
+ // slave finisher?
+ if (mdr->slave_commit) {
+ mdr->slave_commit->finish(0);
+ delete mdr->slave_commit;
+ mdr->slave_commit = 0;
+ }
+
+ delete mdr->client_request;
+ delete mdr->slave_request;
request_cleanup(mdr);
if (mds->logger) mds->logger->inc("reply");
void MDCache::request_forward(MDRequest *mdr, int who, int port)
{
if (!port) port = MDS_PORT_SERVER;
-
- dout(7) << "request_forward to " << who << " req " << *mdr << endl;
-
- mds->forward_message_mds(mdr->request, who, port);
+ dout(7) << "request_forward " << *mdr << " to mds" << who << " req " << *mdr << endl;
+
+ mds->forward_message_mds(mdr->client_request, who, port);
request_cleanup(mdr);
if (mds->logger) mds->logger->inc("fw");
void MDCache::dispatch_request(MDRequest *mdr)
{
- assert(mdr->request);
-
- switch (mdr->request->get_type()) {
- case MSG_CLIENT_REQUEST:
- mds->server->dispatch_request(mdr);
- break;
-
- case MSG_MDS_LOCK:
- mds->locker->handle_lock((MLock*)mdr->request);
- break;
-
- default:
- assert(0); // shouldn't get here
- }
+ if (mdr->client_request) {
+ mds->server->dispatch_client_request(mdr);
+ } else if (mdr->slave_request) {
+ mds->server->dispatch_slave_request(mdr);
+ } else
+ assert(0);
}
-
-void MDCache::request_drop_locks(MDRequest *mdr)
+void MDCache::request_forget_foreign_locks(MDRequest *mdr)
{
- // leftover locks
- while (!mdr->xlocks.empty())
- mds->locker->xlock_finish(*mdr->xlocks.begin(), mdr);
- while (!mdr->rdlocks.empty())
- mds->locker->rdlock_finish(*mdr->rdlocks.begin(), mdr);
- while (!mdr->wrlocks.empty())
- mds->locker->wrlock_finish(*mdr->wrlocks.begin(), mdr);
-
- // make sure ref and trace are empty
- // if we are doing our own locking, we can't use them!
- assert(mdr->ref == 0);
- assert(mdr->trace.empty());
+ // xlocks
+ set<SimpleLock*>::iterator p = mdr->xlocks.begin();
+ while (p != mdr->xlocks.end()) {
+ if ((*p)->get_parent()->is_auth())
+ p++;
+ else {
+ dout(10) << "request_forget_foreign_locks " << **p
+ << " on " << *(*p)->get_parent() << endl;
+ (*p)->put_xlock();
+ mdr->locks.erase(*p);
+ mdr->xlocks.erase(p++);
+ }
+ }
}
-
void MDCache::request_cleanup(MDRequest *mdr)
{
+ dout(15) << "request_cleanup " << *mdr << endl;
metareqid_t ri = mdr->reqid;
// clear ref, trace
mdr->ref = 0;
mdr->trace.clear();
+ // clean up slaves
+ // (will implicitly drop remote dn pins)
+ for (set<int>::iterator p = mdr->slaves.begin();
+ p != mdr->slaves.end();
+ ++p) {
+ MMDSSlaveRequest *r = new MMDSSlaveRequest(mdr->reqid, MMDSSlaveRequest::OP_FINISH);
+ mds->send_message_mds(r, *p, MDS_PORT_SERVER);
+ }
+ // strip foreign xlocks out of lock lists, since the OP_FINISH drops them implicitly.
+ request_forget_foreign_locks(mdr);
+
+
// drop locks
- request_drop_locks(mdr);
+ mds->locker->drop_locks(mdr);
- // drop auth pins
- mdr->drop_auth_pins();
+ // drop (local) auth pins
+ mdr->drop_local_auth_pins();
// drop cache pins
for (set<MDSCacheObject*>::iterator it = mdr->pins.begin();
active_requests.erase(mdr->reqid);
delete mdr;
+
+
+
// log some stats *****
if (mds->logger) {
mds->logger->set("c", lru.lru_get_size());
}
};
-void MDCache::anchor_create(CInode *in, Context *onfinish)
+void MDCache::anchor_create(MDRequest *mdr, CInode *in, Context *onfinish)
{
assert(in->is_auth());
// auth pin
- if (!in->can_auth_pin()) {
+ if (!in->can_auth_pin() &&
+ !mdr->is_auth_pinned(in)) {
dout(7) << "anchor_create not authpinnable, waiting on " << *in << endl;
in->add_waiter(CInode::WAIT_AUTHPINNABLE, onfinish);
return;
assert(in->is_auth());
// auth pin
- if (!in->can_auth_pin()) {
+ if (!in->can_auth_pin()/* &&
+ !mdr->is_auth_pinned(in)*/) {
dout(7) << "anchor_destroy not authpinnable, waiting on " << *in << endl;
in->add_waiter(CInode::WAIT_AUTHPINNABLE, onfinish);
return;
// log removal
version_t pdv = dn->pre_dirty();
- EUpdate *le = new EUpdate;
+ EUpdate *le = new EUpdate("purge_stray");
le->metablob.add_dir_context(dn->dir);
le->metablob.add_null_dentry(dn, true);
le->metablob.add_inode_truncate(dn->inode->inode, 0);
dn->dir->remove_dentry(dn);
// purge+remove inode
- if (in->inode.size > 0)
- purge_inode(&in->inode, 0);
+ purge_inode(&in->inode, 0);
remove_inode(in);
}
}
-// -------------------------------------------------------------------------------
-// HARD LINKS
-
-
-class C_MDC_InodeLinkAgree : public Context {
- MDS *mds;
- MInodeLink *m;
-public:
- C_MDC_InodeLinkAgree(MDS *_mds, MInodeLink *_m) : mds(_mds), m(_m) {}
- void finish(int r) {
- mds->send_message_mds(new MInodeLink(MInodeLink::OP_AGREE,
- m->get_ino(),
- m->get_inc(),
- m->get_reqid()),
- m->get_source().num(),
- m->get_source_port());
- delete m;
- }
-};
-
-void MDCache::handle_inode_link(MInodeLink *m)
-{
- CInode *in = get_inode(m->get_ino());
- assert(in);
- dout(7) << "handle_inode_link " << *m << " on " << *in << endl;
-
- // get request.
- // we should have this bc the inode is xlocked.
- MDRequest *mdr = request_get(m->get_reqid());
-
- switch (m->get_op()) {
- // auth
- case MInodeLink::OP_PREPARE:
- assert(in->is_auth());
- {
- version_t pv = in->pre_dirty();
- ESlaveUpdate *le = new ESlaveUpdate("link_prepare", m->get_reqid(), 0);
- le->metablob.add_dir_context(in->get_parent_dir());
- inode_t *pi = le->metablob.add_primary_dentry(in->parent, true, in);
- if (m->get_inc())
- pi->nlink++;
- else
- pi->nlink--;
- pi->ctime = m->get_ctime();
- pi->version = pv;
- mdr->projected_inode[in->ino()] = *pi;
- mds->mdlog->submit_entry(le);
- mds->mdlog->wait_for_sync(new C_MDC_InodeLinkAgree(mds, m));
- }
- return;
-
- case MInodeLink::OP_COMMIT:
- assert(in->is_auth());
- {
- // make the update to our cache
- in->inode = mdr->projected_inode[in->ino()];
- in->mark_dirty(in->inode.version);
-
- // journal the commit
- ESlaveUpdate *le = new ESlaveUpdate("link_commit", m->get_reqid(), 1);
- mds->mdlog->submit_entry(le);
- }
- delete m;
- return;
-
-
- case MInodeLink::OP_AGREE:
- assert(!in->is_auth());
- in->finish_waiting(CInode::WAIT_SLAVEAGREE);
- delete m;
- return;
-
- default:
- assert(0);
- }
-}
-
-
-
// REPLICAS
// get started.
if (dis->get_base_ino() == MDS_INO_ROOT) {
// wants root
- dout(7) << "handle_discover from mds" << dis->get_asker() << " wants root + " << dis->get_want().get_path() << endl;
+ dout(7) << "handle_discover from mds" << dis->get_asker()
+ << " wants root + " << dis->get_want().get_path() << endl;
assert(mds->get_nodeid() == 0);
assert(root->is_auth());
dout(10) << "added root " << *root << endl;
cur = root;
+ }
+ else if (dis->get_base_ino() == MDS_INO_STRAY(whoami)) {
+ // wants root
+ dout(7) << "handle_discover from mds" << dis->get_asker()
+ << " wants stray + " << dis->get_want().get_path() << endl;
- } else {
+ reply->add_inode( stray->replicate_to( dis->get_asker() ) );
+ dout(10) << "added stray " << *stray << endl;
+
+ cur = stray;
+ }
+ else {
// there's a base inode
cur = get_inode(dis->get_base_ino());
for (unsigned i = 0;
i < dis->get_want().depth() || dis->get_want().depth() == 0;
i++) {
-
+
// -- figure out the dir
// is *cur even a dir at all?
} else {
// requester explicity specified the frag
fg = dis->get_base_dir_frag();
- assert(dis->wants_base_dir() || dis->get_base_ino() == MDS_INO_ROOT);
+ assert(dis->wants_base_dir() || dis->get_base_ino() < MDS_INO_BASE);
}
CDir *curdir = cur->get_dirfrag(fg);
dout(7) << *cur << " dirfrag not open, not inode auth, setting dir_auth_hint" << endl;
reply->set_dir_auth_hint(cur->authority().first);
}
+ reply->set_wanted_xlocks_hint(dis->wants_xlocked());
// set hint (+ dentry, if there is one)
if (dis->get_want().depth() > i)
// add dir
if (reply->is_empty() && !dis->wants_base_dir()) {
- dout(7) << "not adding unwanted base dir " << *curdir << endl;
+ dout(7) << "handle_discover not adding unwanted base dir " << *curdir << endl;
} else {
assert(!curdir->is_ambiguous_auth()); // would be frozen.
reply->add_dir( curdir->replicate_to(dis->get_asker()) );
- dout(7) << "added dir " << *curdir << endl;
+ dout(7) << "handle_discover added dir " << *curdir << endl;
}
if (dis->get_want().depth() == 0) break;
}
assert(dn);
+ // xlocked dentry?
+ // ...always block on non-tail items (they are unrelated)
+ // ...allow xlocked tail disocvery _only_ if explicitly requested
+ if (dn->lock.is_xlocked()) {
+ // is this the last (tail) item in the discover traversal?
+ bool tailitem = (dis->get_want().depth() == 0) || (i == dis->get_want().depth() - 1);
+ if (tailitem && dis->wants_xlocked()) {
+ dout(7) << "handle_discover allowing discovery of xlocked tail " << *dn << endl;
+ } else {
+ dout(7) << "handle_discover blocking on xlocked " << *dn << endl;
+ dn->lock.add_waiter(SimpleLock::WAIT_RD, new C_MDS_RetryMessage(mds, dis));
+ delete reply;
+ return;
+ }
+ }
+
// add dentry
reply->add_dentry( dn->replicate_to( dis->get_asker() ) );
- dout(7) << "added dentry " << *dn << endl;
+ dout(7) << "handle_discover added dentry " << *dn << endl;
if (!dn->is_primary()) break; // stop on null or remote link.
assert(next->is_auth());
reply->add_inode( next->replicate_to( dis->get_asker() ) );
- dout(7) << "added inode " << *next << endl;
+ dout(7) << "handle_discover added inode " << *next << endl;
// descend, keep going.
cur = next;
// how did we do?
if (reply->is_empty()) {
- dout(7) << "dropping this empty reply)." << endl;
+ dout(7) << "handle_discover dropping this empty reply)." << endl;
delete reply;
} else {
- dout(7) << "sending result back to asker mds" << dis->get_asker() << endl;
+ dout(7) << "handle_discover sending result back to asker mds" << dis->get_asker() << endl;
mds->send_message_mds(reply, dis->get_asker(), MDS_PORT_CACHE);
}
if (cur) {
dout(7) << "discover_reply " << *cur << " + " << m->get_path() << ", have " << m->get_num_inodes() << " inodes" << endl;
- } else {
- if (!m->has_root()) {
- dout(7) << "discover_reply don't have base ino " << m->get_base_ino() << ", dropping" << endl;
- delete m;
- return;
- }
-
+ }
+ else if (m->get_base_ino() == MDS_INO_ROOT) {
// it's the root inode.
assert(!root);
- assert(m->get_base_ino() == MDS_INO_ROOT);
assert(!m->has_base_dentry());
assert(!m->has_base_dir());
dout(7) << "discover_reply root + " << m->get_path() << " " << m->get_num_inodes() << " inodes" << endl;
// add in root
- cur = new CInode(this, false);
- m->get_inode(0).update_inode(cur); // that thar 0 is an array index (the 0th inode in the reply).
-
- // root
- set_root( cur );
- add_inode( cur );
+ cur = add_replica_inode(m->get_inode(0), NULL);
+ cur->force_auth = pair<int,int>(m->get_source().num(), CDIR_AUTH_UNKNOWN);
+ set_root(cur);
dout(7) << "discover_reply got root " << *cur << endl;
// take root waiters
finished.swap(waiting_for_root);
}
+ else if (MDS_INO_IS_STRAY(m->get_base_ino())) {
+ dout(7) << "discover_reply stray + " << m->get_path() << " " << m->get_num_inodes() << " inodes" << endl;
+
+ // add
+ cur = add_replica_inode(m->get_inode(0), NULL);
+ cur->force_auth = pair<int,int>(m->get_source().num(), CDIR_AUTH_UNKNOWN);
+
+ dout(7) << "discover_reply got stray " << *cur << endl;
+
+ // take waiters
+ finished.swap(waiting_for_stray[cur->ino()]);
+ waiting_for_stray.erase(cur->ino());
+ }
// fyi
if (m->is_flag_error_dir()) dout(7) << " flag error, dir" << endl;
if (m->is_flag_error_dn()) dout(7) << " flag error, dentry = " << m->get_error_dentry() << endl;
dout(10) << "depth = " << m->get_depth()
<< ", has base_dir/base_dn/root = "
- << m->has_base_dir() << " / " << m->has_base_dentry() << " / " << m->has_root()
+ << m->has_base_dir() << " / " << m->has_base_dentry() << " / " << m->has_base_inode()
<< ", num dirs/dentries/inodes = "
<< m->get_num_dirs() << " / " << m->get_num_dentries() << " / " << m->get_num_inodes()
<< endl;
// indexese follow each ([[dir] dentry] inode)
// can start, end with any type.
- for (int i=m->has_root(); i<m->get_depth(); i++) {
+ for (int i=m->has_base_inode(); i<m->get_depth(); i++) {
dout(10) << "discover_reply i=" << i << " cur " << *cur << endl;
+ // dir
frag_t fg;
CDir *curdir = 0;
-
- // dir
- if ((i > 0) ||
- (i == 0 && m->has_base_dir())) {
+ if (i > 0 || m->has_base_dir()) {
assert(m->get_dir(i).get_dirfrag().ino == cur->ino());
fg = m->get_dir(i).get_dirfrag().frag;
-
+
// add/update the dir replica
curdir = add_replica_dir(cur, fg, m->get_dir(i),
m->get_source().num(),
finished);
- }
+ }
+ if (!curdir) {
+ fg = cur->pick_dirfrag(m->get_dentry(i).get_dname());
+ curdir = cur->get_dirfrag(fg);
+ }
// dentry error?
if (i == m->get_depth()-1 &&
break;
}
- if (i >= m->get_last_dentry()) break;
-
- // dentry
- dout(7) << "i = " << i << " dentry is " << m->get_dentry(i).get_dname() << endl;
-
- if (!curdir) {
- fg = cur->pick_dirfrag(m->get_dentry(i).get_dname());
- curdir = cur->get_dirfrag(fg);
- }
assert(curdir);
+ // dentry
CDentry *dn = 0;
- if (i > 0 ||
- m->has_base_dentry()) {
- dn = curdir->lookup( m->get_dentry(i).get_dname() );
-
- if (dn) {
- dout(7) << "had " << *dn << endl;
- m->get_dentry(i).update_dentry(dn);
- } else {
- dn = curdir->add_dentry( m->get_dentry(i).get_dname(), 0, false );
- m->get_dentry(i).update_new_dentry(dn);
- dout(7) << "added " << *dn << endl;
- }
-
- curdir->take_dentry_waiting(m->get_dentry(i).get_dname(), finished);
+ if (i >= m->get_last_dentry()) break;
+ if (i > 0 || m->has_base_dentry()) {
+ dn = add_replica_dentry(curdir, m->get_dentry(i), finished);
}
-
- if (i >= m->get_last_inode()) break;
// inode
- dout(7) << "i = " << i << " ino is " << m->get_ino(i) << endl;
- CInode *in = get_inode( m->get_inode(i).get_ino() );
- assert(dn);
-
- if (in) {
- dout(7) << "had " << *in << ", new nonce " << m->get_inode(i).get_replica_nonce() << endl;
- in->replica_nonce = m->get_inode(i).get_replica_nonce();
-
- assert(in == dn->inode); // if we have it, it should be already linked to *dn.
- }
- else {
- // didn't have it.
- in = new CInode(this, false);
- m->get_inode(i).update_inode(in);
- add_inode( in );
-
- // link in
- assert(dn->inode == 0); // better not be something else linked to this dentry.
- dn->dir->link_inode(dn, in);
-
- dout(7) << "added " << *in << " nonce " << in->replica_nonce << endl;
- }
-
- // onward!
- cur = in;
+ if (i >= m->get_last_inode()) break;
+ cur = add_replica_inode(m->get_inode(i), dn);
}
// dir_auth hint?
if (m->get_dir_auth_hint() != CDIR_AUTH_UNKNOWN &&
m->get_dir_auth_hint() != mds->get_nodeid()) {
dout(7) << " dir_auth_hint is " << m->get_dir_auth_hint() << endl;
- // let's try again.
- int hint = m->get_dir_auth_hint();
- // include any path fragment we were looking for at the time
+ // try again. include dentry _and_ dirfrag, just in case.
+ int hint = m->get_dir_auth_hint();
filepath want;
- if (m->get_error_dentry().length() > 0)
- want.push_dentry(m->get_error_dentry());
-
- mds->send_message_mds(new MDiscover(mds->get_nodeid(),
- cur->ino(),
- want,
- true), // being conservative here.
- hint, MDS_PORT_CACHE);
+ want.push_dentry(m->get_error_dentry());
+ MDiscover *dis = new MDiscover(mds->get_nodeid(),
+ cur->ino(),
+ want,
+ true,
+ m->get_wanted_xlocks_hint());
+ frag_t fg = cur->pick_dirfrag(m->get_error_dentry());
+ dis->set_base_dir_frag(fg);
+ mds->send_message_mds(dis, hint, MDS_PORT_CACHE);
- // note the dangling discover
- dir_discovers[cur->ino()].insert(hint);
+ // note the dangling discover... but only if it's already noted in dir_discovers (i.e. someone is waiting)
+ if (dir_discovers.count(cur->ino())) {
+ dir_discovers[cur->ino()].insert(hint);
+ assert(cur->is_waiter_for(CInode::WAIT_DIR));
+ }
}
else if (m->is_flag_error_dir()) {
// dir error at the end there?
cur->take_waiting(CInode::WAIT_DIR, error);
dir_discovers.erase(cur->ino());
}
-
-
+
// finish errors directly
finish_contexts(error, -ENOENT);
-
mds->queue_waiters(finished);
// done
return dir;
}
-
+
+CDentry *MDCache::add_replica_dentry(CDir *dir, CDentryDiscover &dis, list<Context*>& finished)
+{
+ CDentry *dn = dir->lookup( dis.get_dname() );
+
+ // have it?
+ if (dn) {
+ dis.update_dentry(dn);
+ dout(7) << "add_replica_dentry had " << *dn << endl;
+ } else {
+ dn = dir->add_dentry( dis.get_dname(), 0 );
+ dis.update_dentry(dn);
+ dis.init_dentry_lock(dn);
+ dout(7) << "add_replica_dentry added " << *dn << endl;
+ }
+
+ // remote_ino linkage?
+ if (dis.get_remote_ino()) {
+ if (dn->is_null())
+ dir->link_inode(dn, dis.get_remote_ino());
+
+ // hrm. yeah.
+ assert(dn->is_remote() && dn->get_remote_ino() == dis.get_remote_ino());
+ }
+
+ dir->take_dentry_waiting(dis.get_dname(), finished);
+
+ return dn;
+}
+CInode *MDCache::add_replica_inode(CInodeDiscover& dis, CDentry *dn)
+{
+ CInode *in = get_inode(dis.get_ino());
+ if (!in) {
+ in = new CInode(this, false);
+ dis.update_inode(in);
+ dis.init_inode_locks(in);
+ add_inode(in);
+ dout(10) << "add_replica_inode had " << *in << endl;
+ if (dn && dn->is_null())
+ dn->dir->link_inode(dn, in);
+ } else {
+ dis.update_inode(in);
+ dout(10) << "add_replica_inode added " << *in << endl;
+ }
+
+ if (dn) {
+ assert(dn->is_primary());
+ assert(dn->inode == in);
+ }
+
+ return in;
+}
+
+
+CDentry *MDCache::add_replica_stray(bufferlist &bl, CInode *in, int from)
+{
+ list<Context*> finished;
+ int off = 0;
+
+ // inode
+ CInodeDiscover indis;
+ indis._decode(bl, off);
+ CInode *strayin = add_replica_inode(indis, NULL);
+ strayin->force_auth = pair<int,int>(from, CDIR_AUTH_UNKNOWN);
+ dout(15) << "strayin " << *strayin << endl;
+
+ // dir
+ CDirDiscover dirdis;
+ dirdis._decode(bl, off);
+ CDir *straydir = add_replica_dir(strayin, dirdis.get_dirfrag().frag, dirdis,
+ from, finished);
+ dout(15) << "straydir " << *straydir << endl;
+
+ // dentry
+ CDentryDiscover dndis;
+ dndis._decode(bl, off);
+
+ string straydname;
+ in->name_stray_dentry(straydname);
+ CDentry *straydn = add_replica_dentry(straydir, dndis, finished);
+
+ mds->queue_waiters(finished);
+
+ return straydn;
+}
dout(5) << "trying discover on dir_update for " << path << endl;
- int r = path_traverse(0, 0,
- path, trace, true,
- m, new C_MDS_RetryMessage(mds, m),
+ int r = path_traverse(0, m,
+ 0, path, trace, true,
MDS_TRAVERSE_DISCOVER);
if (r > 0)
return;
// move to stray?
CDentry *straydn = 0;
if (m->strayin) {
- // inode
- CInode *in = get_inode(MDS_INO_STRAY(m->get_source().num()));
- if (!in) {
- in = new CInode(this, false);
- m->strayin->update_inode(in);
- add_inode(in);
- } else {
- m->strayin->update_inode(in);
- }
-
- // dirfrag
list<Context*> finished;
+ CInode *in = add_replica_inode(*m->strayin, NULL);
CDir *dir = add_replica_dir(in, m->straydir->get_dirfrag().frag, *m->straydir,
m->get_source().num(), finished);
+ straydn = add_replica_dentry(dir, *m->straydn, finished);
if (!finished.empty()) mds->queue_waiters(finished);
-
- // dentry
- straydn = dir->add_dentry( m->straydn->get_dname(), 0, false );
- m->straydn->update_new_dentry(straydn);
}
// open inode?
list<CDir*> rootfrags;
if (root) root->get_dirfrags(rootfrags);
if (stray) stray->get_dirfrags(rootfrags);
+ dout(15) << "rootfrags " << rootfrags << endl;
// queue stuff
list<pair<CDir*,int> > q;
string indent;
set<CDir*> seen;
- // calc depth
+ // calc max depth
for (list<CDir*>::iterator p = rootfrags.begin(); p != rootfrags.end(); ++p)
q.push_back(pair<CDir*,int>(*p, 0));
int d = q.front().second;
q.pop_front();
+ if (subtrees.count(dir) == 0) continue;
+
if (d > depth) depth = d;
// sanity check
+ //dout(25) << "saw depth " << d << " " << *dir << endl;
if (seen.count(dir)) dout(0) << "aah, already seen " << *dir << endl;
assert(seen.count(dir) == 0);
seen.insert(dir);
if (!subtrees[dir].empty()) {
for (set<CDir*>::iterator p = subtrees[dir].begin();
p != subtrees[dir].end();
- ++p)
+ ++p) {
+ //dout(25) << " saw sub " << **p << endl;
q.push_front(pair<CDir*,int>(*p, d+1));
+ }
}
}
int d = q.front().second;
q.pop_front();
+ if (subtrees.count(dir) == 0) continue;
+
// adjust indenter
while ((unsigned)d < indent.size())
indent.resize(d);
#include "CDentry.h"
#include "CDir.h"
#include "include/Context.h"
+#include "events/EMetaBlob.h"
class MDS;
class Migrator;
class Message;
-class MMDSImportMap;
+class MMDSResolve;
+class MMDSResolveAck;
class MMDSCacheRejoin;
class MMDSCacheRejoinAck;
class MDiscover;
class Message;
class MClientRequest;
-
+class MMDSSlaveRequest;
// MDCache
//typedef const char* pchar;
+struct PVList {
+ map<MDSCacheObject*,version_t> ls;
+
+ version_t add(MDSCacheObject* o, version_t v) {
+ return ls[o] = v;
+ }
+};
/** active_request_t
* state we track for requests we are currently processing.
*/
struct MDRequest {
metareqid_t reqid;
- Message *request; // MClientRequest, or MLock
- int by_mds; // if MLock, and remote xlock attempt
-
+
+ // -- i am a client (master) request
+ MClientRequest *client_request; // client request (if any)
+ set<int> slaves; // mds nodes that have slave requests to me (implies client_request)
+ set<int> waiting_on_slave; // peers i'm waiting for slavereq replies from.
+
vector<CDentry*> trace; // original path traversal.
CInode *ref; // reference inode. if there is only one, and its path is pinned.
-
+
+ // -- i am a slave request
+ MMDSSlaveRequest *slave_request; // slave request (if one is pending; implies slave == true)
+ int slave_to_mds; // this is a slave request if >= 0.
+
+ // -- my pins and locks --
// cache pins (so things don't expire)
- set< MDSCacheObject* > pins;
-
+ set< MDSCacheObject* > pins;
+
// auth pins
- set< CDir* > dir_auth_pins;
- set< CInode* > inode_auth_pins;
+ set< MDSCacheObject* > auth_pins;
// held locks
set< SimpleLock* > rdlocks; // always local.
set< SimpleLock* > xlocks; // local or remote.
set< SimpleLock*, SimpleLock::ptr_lt > locks; // full ordering
- // projected updates
- map< inodeno_t, inode_t > projected_inode;
+ // if this flag is set, do not attempt to acquire further locks.
+ // (useful for wrlock, which may be a moving auth target)
+ bool done_locking;
+ bool committing;
+ bool aborted;
+
+ // for rename/link/unlink
+ utime_t now;
+ set<int> witnessed; // nodes who have journaled a RenamePrepare
+ map<MDSCacheObject*,version_t> pvmap;
+
+ // for rename
+ set<int> extra_witnesses; // replica list from srcdn auth (rename)
+ version_t src_reanchor_atid; // src->dst
+ version_t dst_reanchor_atid; // dst->stray
+ bufferlist inode_import;
+ version_t inode_import_v;
+ CDentry *srcdn; // srcdn, if auth, on slave
+
+ // called when slave commits
+ Context *slave_commit;
// ---------------------------------------------------
- MDRequest() : request(0), by_mds(-1), ref(0) {}
- MDRequest(metareqid_t ri, Message *req=0) : reqid(ri), request(req), by_mds(-1), ref(0) {}
+ MDRequest() :
+ client_request(0), ref(0),
+ slave_request(0), slave_to_mds(-1),
+ done_locking(false), committing(false), aborted(false),
+ src_reanchor_atid(0), dst_reanchor_atid(0), inode_import_v(0),
+ slave_commit(0) { }
+ MDRequest(metareqid_t ri, MClientRequest *req) :
+ reqid(ri), client_request(req), ref(0),
+ slave_request(0), slave_to_mds(-1),
+ done_locking(false), committing(false), aborted(false),
+ src_reanchor_atid(0), dst_reanchor_atid(0), inode_import_v(0),
+ slave_commit(0) { }
+ MDRequest(metareqid_t ri, int by) :
+ reqid(ri), client_request(0), ref(0),
+ slave_request(0), slave_to_mds(by),
+ done_locking(false), committing(false), aborted(false),
+ src_reanchor_atid(0), dst_reanchor_atid(0), inode_import_v(0),
+ slave_commit(0) { }
- // request
- MClientRequest *client_request() {
- return (MClientRequest*)request;
- }
+ bool is_master() { return slave_to_mds < 0; }
+ bool is_slave() { return slave_to_mds >= 0; }
+ bool slave_did_prepare() { return slave_commit; }
+
// pin items in cache
void pin(MDSCacheObject *o) {
if (pins.count(o) == 0) {
}
// auth pins
- bool is_auth_pinned(CInode *in) { return inode_auth_pins.count(in); }
- bool is_auth_pinned(CDir *dir) { return dir_auth_pins.count(dir); }
- void auth_pin(CInode *in) {
- if (!is_auth_pinned(in)) {
- in->auth_pin();
- inode_auth_pins.insert(in);
- }
+ bool is_auth_pinned(MDSCacheObject *object) {
+ return auth_pins.count(object);
}
- void auth_pin(CDir *dir) {
- if (!is_auth_pinned(dir)) {
- dir->auth_pin();
- dir_auth_pins.insert(dir);
+ void auth_pin(MDSCacheObject *object) {
+ if (!is_auth_pinned(object)) {
+ object->auth_pin();
+ auth_pins.insert(object);
}
}
- void drop_auth_pins() {
- for (set<CInode*>::iterator it = inode_auth_pins.begin();
- it != inode_auth_pins.end();
- it++)
- (*it)->auth_unpin();
- inode_auth_pins.clear();
- for (set<CDir*>::iterator it = dir_auth_pins.begin();
- it != dir_auth_pins.end();
- it++)
- (*it)->auth_unpin();
- dir_auth_pins.clear();
+ void drop_local_auth_pins() {
+ set<MDSCacheObject*>::iterator it = auth_pins.begin();
+ while (it != auth_pins.end()) {
+ if ((*it)->is_auth()) {
+ (*it)->auth_unpin();
+ auth_pins.erase(it++);
+ } else {
+ it++;
+ }
+ }
+ auth_pins.clear();
}
};
{
out << "request(" << mdr.reqid;
//if (mdr.request) out << " " << *mdr.request;
+ if (mdr.is_slave()) out << " slave_to mds" << mdr.slave_to_mds;
+ if (mdr.client_request) out << " cr=" << mdr.client_request;
+ if (mdr.slave_request) out << " sr=" << mdr.slave_request;
out << ")";
return out;
}
// join/split subtrees as appropriate
public:
bool is_subtrees() { return !subtrees.empty(); }
+ void list_subtrees(list<CDir*>& ls);
void adjust_subtree_auth(CDir *root, pair<int,int> auth);
void adjust_subtree_auth(CDir *root, int a, int b=CDIR_AUTH_UNKNOWN) {
adjust_subtree_auth(root, pair<int,int>(a,b));
void adjust_export_state(CDir *dir);
void try_subtree_merge(CDir *root);
void try_subtree_merge_at(CDir *root);
+ void eval_subtree_root(CDir *dir);
CDir *get_subtree_root(CDir *dir);
void remove_subtree(CDir *dir);
void get_subtree_bounds(CDir *root, set<CDir*>& bounds);
protected:
// delayed cache expire
- map<CDir*, map<int, MCacheExpire*> > delayed_expire; // import|export dir -> expire msg
+ map<CDir*, map<int, MCacheExpire*> > delayed_expire; // subtree root -> expire msg
// -- discover --
hash_map<inodeno_t, set<int> > dir_discovers; // dirino -> mds set i'm trying to discover.
hash_map<metareqid_t, MDRequest*> active_requests;
public:
- MDRequest* request_start(metareqid_t rid);
MDRequest* request_start(MClientRequest *req);
- MDRequest* request_start(MLock *req);
+ MDRequest* request_start_slave(metareqid_t rid, int by);
+ bool have_request(metareqid_t rid) {
+ return active_requests.count(rid);
+ }
MDRequest* request_get(metareqid_t rid);
void request_pin_ref(MDRequest *r, CInode *ref, vector<CDentry*>& trace);
void request_finish(MDRequest *mdr);
void request_forward(MDRequest *mdr, int mds, int port=0);
void dispatch_request(MDRequest *mdr);
- void request_drop_locks(MDRequest *mdr);
+ void request_forget_foreign_locks(MDRequest *mdr);
void request_cleanup(MDRequest *r);
protected:
set<int> recovery_set;
+public:
+ void set_recovery_set(set<int>& s);
+ void handle_mds_failure(int who);
+ void handle_mds_recovery(int who);
+
+protected:
+ // [resolve]
// from EImportStart w/o EImportFinish during journal replay
map<dirfrag_t, list<dirfrag_t> > my_ambiguous_imports;
- // from MMDSImportMaps
+ // from MMDSResolves
map<int, map<dirfrag_t, list<dirfrag_t> > > other_ambiguous_imports;
- set<int> wants_import_map; // nodes i need to send my import map to
- set<int> got_import_map; // nodes i got import_maps from
+ map<int, map<metareqid_t, EMetaBlob> > uncommitted_slave_updates; // for replay.
+ map<metareqid_t, bool> ambiguous_slave_updates; // for log trimming.
+ map<metareqid_t, Context*> waiting_for_slave_update_commit;
+ friend class ESlaveUpdate;
+
+ set<int> wants_resolve; // nodes i need to send my resolve to
+ set<int> got_resolve; // nodes i got resolves from
+ set<int> need_resolve_ack; // nodes i need a resolve_ack from
- void handle_import_map(MMDSImportMap *m);
+ void handle_resolve(MMDSResolve *m);
+ void handle_resolve_ack(MMDSResolveAck *m);
+ void maybe_resolve_finish();
void disambiguate_imports();
+ void recalc_auth_bits();
+public:
+ // ambiguous imports
+ void add_ambiguous_import(dirfrag_t base, list<dirfrag_t>& bounds);
+ void add_ambiguous_import(CDir *base, const set<CDir*>& bounds);
+ bool have_ambiguous_import(dirfrag_t base) {
+ return my_ambiguous_imports.count(base);
+ }
+ void cancel_ambiguous_import(dirfrag_t dirino);
+ void finish_ambiguous_import(dirfrag_t dirino);
+ void send_resolve(int who);
+ void send_resolve_now(int who);
+ void send_resolve_later(int who);
+ void maybe_send_pending_resolves();
+ void log_subtree_map(Context *onsync=0);
+ void _logged_subtree_map(off_t off);
+protected:
+ // [rejoin]
set<int> rejoin_gather; // nodes from whom i need a rejoin
set<int> rejoin_ack_gather; // nodes from whom i need a rejoin ack
- set<int> want_rejoin_ack; // nodes to whom i need to send a rejoin ack
- void cache_rejoin_walk(CDir *dir, MMDSCacheRejoin *rejoin);
+ map<inodeno_t,map<int,inode_caps_reconnect_t> > cap_exports; // ino -> client -> capex
+ map<inodeno_t,string> cap_export_paths;
+
+ map<inodeno_t,map<int, map<int,inode_caps_reconnect_t> > > cap_imports; // ino -> client -> frommds -> capex
+ map<inodeno_t,string> cap_import_paths;
+
+ set<CInode*> rejoin_undef_inodes;
+
+ void rejoin_walk(CDir *dir, MMDSCacheRejoin *rejoin);
void handle_cache_rejoin(MMDSCacheRejoin *m);
- void handle_cache_rejoin_rejoin(MMDSCacheRejoin *m);
+ void handle_cache_rejoin_weak(MMDSCacheRejoin *m);
+ CInode* rejoin_invent_inode(inodeno_t ino);
+ void handle_cache_rejoin_strong(MMDSCacheRejoin *m);
+ void rejoin_scour_survivor_replicas(int from, MMDSCacheRejoin *ack);
void handle_cache_rejoin_ack(MMDSCacheRejoin *m);
+ void handle_cache_rejoin_purge(MMDSCacheRejoin *m);
void handle_cache_rejoin_missing(MMDSCacheRejoin *m);
void handle_cache_rejoin_full(MMDSCacheRejoin *m);
- void send_cache_rejoin_acks();
- void recalc_auth_bits();
-
+ void rejoin_send_acks();
+ void rejoin_trim_undef_inodes();
public:
- void set_recovery_set(set<int>& s);
- void handle_mds_failure(int who);
- void handle_mds_recovery(int who);
- void send_import_map(int who);
- void send_import_map_now(int who);
- void send_import_map_later(int who);
- void send_pending_import_maps(); // maybe.
- void send_cache_rejoins();
- void log_import_map(Context *onsync=0);
-
-
- // ambiguous imports
- void add_ambiguous_import(dirfrag_t base, list<dirfrag_t>& bounds);
- void add_ambiguous_import(CDir *base, const set<CDir*>& bounds);
- void cancel_ambiguous_import(dirfrag_t dirino);
- void finish_ambiguous_import(dirfrag_t dirino);
-
+ void rejoin_gather_finish();
+ void rejoin_send_rejoins();
+ void rejoin_export_caps(inodeno_t ino, string& path, int client, inode_caps_reconnect_t& icr) {
+ cap_exports[ino][client] = icr;
+ cap_export_paths[ino] = path;
+ }
+ void rejoin_recovered_caps(inodeno_t ino, string& path, int client, inode_caps_reconnect_t& icr,
+ int frommds=-1) {
+ cap_imports[ino][client][frommds] = icr;
+ cap_import_paths[ino] = path;
+ }
+ void rejoin_import_cap(CInode *in, int client, inode_caps_reconnect_t& icr, int frommds);
friend class Locker;
// cache
void set_cache_size(size_t max) { lru.lru_set_max(max); }
size_t get_cache_size() { return lru.lru_get_size(); }
+
+ // trimming
bool trim(int max = -1); // trim cache
+ void trim_dentry(CDentry *dn, map<int, MCacheExpire*>& expiremap);
void trim_dirfrag(CDir *dir, CDir *con,
map<int, MCacheExpire*>& expiremap);
void trim_inode(CDentry *dn, CInode *in, CDir *con,
map<int,class MCacheExpire*>& expiremap);
+ void send_expire_messages(map<int, MCacheExpire*>& expiremap);
void trim_non_auth(); // trim out trimmable non-auth items
// shutdown
return inode_map[df.ino]->get_dirfrag(df.frag);
}
- int hash_dentry(inodeno_t ino, const string& s) {
- return 0; // fixme
- }
+ MDSCacheObject *get_object(MDSCacheObjectInfo &info);
+
public:
}
void inode_remove_replica(CInode *in, int rep);
+ void dentry_remove_replica(CDentry *dn, int rep);
void rename_file(CDentry *srcdn, CDentry *destdn);
public:
// inode purging
void purge_inode(inode_t *inode, off_t newsize);
+ void _do_purge_inode(inode_t *inode, off_t newsize);
void purge_inode_finish(inodeno_t ino, off_t newsize);
void purge_inode_finish_2(inodeno_t ino, off_t newsize);
bool is_purging(inodeno_t ino, off_t newsize) {
CInode *create_stray_inode(int whose=-1);
void open_local_stray();
void open_foreign_stray(int who, Context *c);
- int path_traverse(MDRequest *mdr,
- CInode *base,
- filepath& path, vector<CDentry*>& trace, bool follow_trailing_sym,
- Message *req, Context *ondelay,
- int onfail,
- bool is_client_req = false,
- bool null_okay = false);
+ CDentry *get_or_create_stray_dentry(CInode *in);
+
+ Context *_get_waiter(MDRequest *mdr, Message *req);
+ int path_traverse(MDRequest *mdr, Message *req,
+ CInode *base, filepath& path,
+ vector<CDentry*>& trace, bool follow_trailing_sym,
+ int onfail);
+ bool path_is_mine(filepath& path);
+ bool path_is_mine(string& p) {
+ filepath path(p);
+ return path_is_mine(path);
+ }
+ CDir *path_traverse_to_dir(filepath& path);
+
void open_remote_dir(CInode *diri, frag_t fg, Context *fin);
CInode *get_dentry_inode(CDentry *dn, MDRequest *mdr);
void open_remote_ino(inodeno_t ino, MDRequest *mdr, Context *fin);
vector<Anchor>& anchortrace,
Context *onfinish);
+ bool parallel_fetch(map<inodeno_t,string>& pathmap,
+ Context *c);
+
void make_trace(vector<CDentry*>& trace, CInode *in);
// -- anchors --
public:
- void anchor_create(CInode *in, Context *onfinish);
+ void anchor_create(MDRequest *mdr, CInode *in, Context *onfinish);
void anchor_destroy(CInode *in, Context *onfinish);
protected:
void _anchor_create_prepared(CInode *in, version_t atid);
int from,
list<Context*>& finished);
CDir* forge_replica_dir(CInode *diri, frag_t fg, int from);
-
- // -- hard links --
- void handle_inode_link(class MInodeLink *m);
+ CDentry *add_replica_dentry(CDir *dir, CDentryDiscover &dis, list<Context*>& finished);
+ CInode *add_replica_inode(CInodeDiscover& dis, CDentry *dn);
+
+public:
+ CDentry *add_replica_stray(bufferlist &bl, CInode *strayin, int from);
+protected:
+
+
// -- namespace --
void handle_dentry_unlink(MDentryUnlink *m);
// should we log a new import_map?
// FIXME: should this go elsewhere?
- if (last_import_map && !writing_import_map &&
- journaler->get_write_pos() - last_import_map >= g_conf.mds_log_import_map_interval) {
+ if (last_subtree_map && !writing_subtree_map &&
+ journaler->get_write_pos() - last_subtree_map >= g_conf.mds_log_subtree_map_interval) {
// log import map
- mds->mdcache->log_import_map();
+ mds->mdcache->log_subtree_map();
}
} else {
// we trimmed off the front!
// we can expire the log a bit.
journaler->set_expire_pos(le->_end_off);
+ journaler->trim();
}
trimming.erase(le->_end_off);
// trim!
dout(10) << "trim " << num_events << " events / " << max_events << " max" << endl;
- while (num_events > max_events) {
+ // hack: only trim for a few seconds at a time
+ utime_t stop = g_clock.now();
+ stop += 2.0;
+
+ while (num_events > max_events &&
+ stop > g_clock.now()) {
off_t gap = journaler->get_write_pos() - journaler->get_read_pos();
dout(5) << "trim num_events " << num_events << " > max " << max_events
num_events++;
// have we seen an import map yet?
- if (!seen_import_map &&
- le->get_type() != EVENT_IMPORTMAP) {
+ if (!seen_subtree_map &&
+ le->get_type() != EVENT_SUBTREEMAP) {
dout(10) << "_replay " << pos << " / " << journaler->get_write_pos()
- << " -- waiting for import_map. (skipping " << *le << ")" << endl;
+ << " -- waiting for subtree_map. (skipping " << *le << ")" << endl;
} else {
dout(10) << "_replay " << pos << " / " << journaler->get_write_pos()
<< " : " << *le << endl;
le->replay(mds);
- if (le->get_type() == EVENT_IMPORTMAP)
- seen_import_map = true;
+ if (le->get_type() == EVENT_SUBTREEMAP)
+ seen_subtree_map = true;
}
delete le;
-void MDLog::_replay()
-{
- mds->mds_lock.Lock();
-
- // read what's buffered
- while (journaler->is_readable() &&
- journaler->get_read_pos() < journaler->get_write_pos()) {
- // read it
- off_t pos = journaler->get_read_pos();
- bufferlist bl;
- bool r = journaler->try_read_entry(bl);
- assert(r);
-
- // unpack event
- LogEvent *le = LogEvent::decode(bl);
- num_events++;
-
- // have we seen an import map yet?
- if (!seen_import_map &&
- le->get_type() != EVENT_IMPORTMAP) {
- dout(10) << "_replay " << pos << " / " << journaler->get_write_pos()
- << " -- waiting for import_map. (skipping " << *le << ")" << endl;
- } else {
- dout(10) << "_replay " << pos << " / " << journaler->get_write_pos()
- << " : " << *le << endl;
- le->replay(mds);
-
- if (le->get_type() == EVENT_IMPORTMAP)
- seen_import_map = true;
- }
- delete le;
-
- // drop lock for a second, so other events (e.g. beacon timer!) can go off
- mds->mds_lock.Unlock();
- mds->mds_lock.Lock();
- }
-
- // wait for read?
- if (journaler->get_read_pos() < journaler->get_write_pos()) {
- journaler->wait_for_readable(new C_MDL_Replay(this));
- return;
- }
-
- // done!
- assert(journaler->get_read_pos() == journaler->get_write_pos());
- dout(10) << "_replay - complete" << endl;
-
- // move read pointer _back_ to expire pos, for eventual trimming
- journaler->set_read_pos(journaler->get_expire_pos());
-
- // kick waiter(s)
- list<Context*> ls;
- ls.swap(waitfor_replay);
- finish_contexts(ls,0);
-}
-
-
- // -- importmaps --
- off_t last_import_map; // offsets of last committed importmap. constrains trimming.
- list<Context*> import_map_expire_waiters;
- bool writing_import_map; // one is being written now
- bool seen_import_map; // for recovery
+ // -- subtreemaps --
+ off_t last_subtree_map; // offsets of last committed subtreemap. constrains trimming.
+ list<Context*> subtree_map_expire_waiters;
+ bool writing_subtree_map; // one is being written now
+ bool seen_subtree_map; // for recovery
- friend class EImportMap;
friend class C_MDS_WroteImportMap;
friend class MDCache;
void init_journaler();
public:
- void add_import_map_expire_waiter(Context *c) {
- import_map_expire_waiters.push_back(c);
+ off_t get_last_subtree_map_offset() { return last_subtree_map; }
+ void add_subtree_map_expire_waiter(Context *c) {
+ subtree_map_expire_waiters.push_back(c);
+ }
+ void take_subtree_map_expire_waiters(list<Context*>& ls) {
+ ls.splice(ls.end(), subtree_map_expire_waiters);
}
logger(0),
trim_reading(false), waiting_for_read(false),
replay_thread(this),
- last_import_map(0),
- writing_import_map(false), seen_import_map(false) {
+ last_subtree_map(0),
+ writing_subtree_map(false), seen_subtree_map(false) {
}
~MDLog();
void set_max_events(size_t max) { max_events = max; }
size_t get_max_events() { return max_events; }
size_t get_num_events() { return num_events + trimming.size(); }
- size_t get_non_importmap_events() { return num_events + trimming.size() - import_map_expire_waiters.size(); }
+ size_t get_non_subtreemap_events() { return num_events + trimming.size() - subtree_map_expire_waiters.size(); }
off_t get_read_pos();
off_t get_write_pos();
void cap() {
capped = true;
list<Context*> ls;
- ls.swap(import_map_expire_waiters);
+ ls.swap(subtree_map_expire_waiters);
finish_contexts(ls);
}
#include "common/Timer.h"
-#include "events/EClientMap.h"
+#include "events/ESession.h"
#include "messages/MMDSMap.h"
#include "messages/MMDSBeacon.h"
// cons/des
-MDS::MDS(int whoami, Messenger *m, MonMap *mm) : timer(mds_lock) {
+MDS::MDS(int whoami, Messenger *m, MonMap *mm) :
+ timer(mds_lock),
+ clientmap(this) {
this->whoami = whoami;
monmap = mm;
if (!creq->is_idempotent()) {
delete req;
- return; // don't actually forward if non-idempotent
+ return; // don't actually forward if non-idempotent! client has to do it.
}
}
}
-class C_MDS_Tick : public Context {
+
+void MDS::send_message_client(Message *m, int client)
+{
+ version_t seq = clientmap.inc_push_seq(client);
+ dout(10) << "send_message_client client" << client << " seq " << seq << " " << *m << endl;
+ messenger->send_message(m, clientmap.get_inst(client));
+}
+
+void MDS::send_message_client(Message *m, entity_inst_t clientinst)
+{
+ version_t seq = clientmap.inc_push_seq(clientinst.name.num());
+ dout(10) << "send_message_client client" << clientinst.name.num() << " seq " << seq << " " << *m << endl;
+ messenger->send_message(m, clientinst);
+}
+
+
+class C_MDS_SendMessageClientSession : public Context {
MDS *mds;
+ Message *msg;
+ entity_inst_t clientinst;
public:
- C_MDS_Tick(MDS *m) : mds(m) {}
+ C_MDS_SendMessageClientSession(MDS *md, Message *ms, entity_inst_t& ci) :
+ mds(md), msg(ms), clientinst(ci) {}
void finish(int r) {
- mds->tick();
+ mds->clientmap.open_session(clientinst);
+ mds->send_message_client(msg, clientinst.name.num());
}
};
+void MDS::send_message_client_maybe_open(Message *m, entity_inst_t clientinst)
+{
+ int client = clientinst.name.num();
+ if (!clientmap.have_session(client)) {
+ // no session!
+ dout(10) << "send_message_client opening session with " << clientinst << endl;
+ clientmap.add_opening(client);
+ mdlog->submit_entry(new ESession(clientinst, true, clientmap.inc_projected()),
+ new C_MDS_SendMessageClientSession(this, m, clientinst));
+ } else {
+ // we have a session.
+ send_message_client(m, clientinst);
+ }
+}
+
int MDS::init(bool standby)
void MDS::tick()
{
+ tick_event = 0;
+
// reschedule
reset_tick();
}
-class C_MDS_BeaconSender : public Context {
- MDS *mds;
-public:
- C_MDS_BeaconSender(MDS *m) : mds(m) {}
- void finish(int r) {
- mds->beacon_send();
- }
-};
void MDS::beacon_send()
{
delete m;
}
-class C_MDS_BeaconKiller : public Context {
- MDS *mds;
- utime_t lab;
-public:
- C_MDS_BeaconKiller(MDS *m, utime_t l) : mds(m), lab(l) {}
- void finish(int r) {
- mds->beacon_kill(lab);
- }
-};
-
void MDS::reset_beacon_killer()
{
utime_t when = beacon_last_acked_stamp;
dout(0) << "beacon_kill last_acked_stamp " << lab
<< ", killing myself."
<< endl;
- exit(0);
+ messenger->suicide();
+ //exit(0);
} else {
dout(20) << "beacon_kill last_acked_stamp " << beacon_last_acked_stamp
<< " != my " << lab
mdsmap->get_mds_set(oldactive, MDSMap::STATE_ACTIVE);
set<int> oldcreating;
mdsmap->get_mds_set(oldcreating, MDSMap::STATE_CREATING);
- set<int> oldout;
- mdsmap->get_mds_set(oldout, MDSMap::STATE_OUT);
+ set<int> oldstopped;
+ mdsmap->get_mds_set(oldstopped, MDSMap::STATE_STOPPED);
// decode and process
mdsmap->decode(m->get_encoded());
// contemplate suicide
if (mdsmap->get_inst(whoami) != messenger->get_myinst()) {
dout(1) << "apparently i've been replaced by " << mdsmap->get_inst(whoami) << ", committing suicide." << endl;
- exit(-1);
+ messenger->suicide();
+ return;
}
if (mdsmap->is_down(whoami)) {
- dout(1) << "apparently i'm down. committing suicide." << endl;
- exit(-1);
+ dout(1) << "apparently i'm down, committing suicide." << endl;
+ messenger->suicide();
+ return;
}
// now active?
if (is_active()) {
// did i just recover?
- if (oldstate == MDSMap::STATE_REJOIN) {
- dout(1) << "successful recovery!" << endl;
-
- // kick anchortable (resent AGREEs)
- if (mdsmap->get_anchortable() == whoami)
- anchortable->finish_recovery();
-
- // kick anchorclient (resent COMMITs)
- anchorclient->finish_recovery();
-
- mdcache->start_recovered_purges();
-
- // tell connected clients
- bcast_mds_map();
- }
+ if (oldstate == MDSMap::STATE_REJOIN ||
+ oldstate == MDSMap::STATE_RECONNECT)
+ recovery_done();
dout(1) << "now active" << endl;
- finish_contexts(waitfor_active); // kick waiters
- }
-
- else if (is_reconnect()) {
- server->reconnect_clients();
- }
-
- else if (is_replay()) {
- // initialize gather sets
- set<int> rs;
- mdsmap->get_recovery_mds_set(rs);
- rs.erase(whoami);
- dout(1) << "now replay. my recovery peers are " << rs << endl;
- mdcache->set_recovery_set(rs);
- }
-
- // now stopping?
- else if (is_stopping()) {
+ finish_contexts(waiting_for_active); // kick waiters
+ } else if (is_replay()) {
+ replay_start();
+ } else if (is_resolve()) {
+ resolve_start();
+ } else if (is_reconnect()) {
+ reconnect_start();
+ } else if (is_stopping()) {
assert(oldstate == MDSMap::STATE_ACTIVE);
- dout(1) << "now stopping" << endl;
-
- // start cache shutdown
- mdcache->shutdown_start();
-
- // terminate client sessions
- server->terminate_sessions();
-
- // flush log
- mdlog->set_max_events(0);
- mdlog->trim(NULL);
- }
-
- // now standby?
- else if (is_stopped()) {
+ stopping_start();
+ } else if (is_stopped()) {
assert(oldstate == MDSMap::STATE_STOPPING);
dout(1) << "now stopped, sending down:out and exiting" << endl;
shutdown_final();
// RESOLVE
- // am i newly resolving?
- if (is_resolve() && oldstate == MDSMap::STATE_REPLAY) {
- // send to all resolve, active, stopping
- dout(10) << "i am newly resolving, sharing import map" << endl;
- set<int> who;
- mdsmap->get_mds_set(who, MDSMap::STATE_RESOLVE);
- mdsmap->get_mds_set(who, MDSMap::STATE_ACTIVE);
- mdsmap->get_mds_set(who, MDSMap::STATE_STOPPING);
- mdsmap->get_mds_set(who, MDSMap::STATE_REJOIN); // hrm. FIXME.
- for (set<int>::iterator p = who.begin(); p != who.end(); ++p) {
- if (*p == whoami) continue;
- mdcache->send_import_map(*p); // now.
- }
- }
// is someone else newly resolving?
- else if (is_resolve() || is_rejoin() || is_active() || is_stopping()) {
+ if (is_resolve() || is_rejoin() || is_active() || is_stopping()) {
set<int> resolve;
mdsmap->get_mds_set(resolve, MDSMap::STATE_RESOLVE);
if (oldresolve != resolve) {
for (set<int>::iterator p = resolve.begin(); p != resolve.end(); ++p) {
if (*p == whoami) continue;
if (oldresolve.count(*p)) continue;
- mdcache->send_import_map(*p); // now or later.
+ mdcache->send_resolve(*p); // now or later.
}
}
}
// is everybody finally rejoining?
if (is_rejoin() || is_active() || is_stopping()) {
// did we start?
- if (!wasrejoining && mdsmap->is_rejoining()) {
- mdcache->send_cache_rejoins();
- }
+ if (!wasrejoining && mdsmap->is_rejoining())
+ rejoin_joint_start();
+
// did we finish?
- if (wasrejoining && !mdsmap->is_rejoining()) {
- mdcache->dump_cache();
- }
+ if (g_conf.mds_dump_cache_after_rejoin &&
+ wasrejoining && !mdsmap->is_rejoining())
+ mdcache->dump_cache(); // for DEBUG only
}
// did someone go active?
for (set<int>::iterator p = active.begin(); p != active.end(); ++p) {
if (*p == whoami) continue; // not me
if (oldactive.count(*p)) continue; // newly so?
- mdcache->handle_mds_recovery(*p);
- if (anchortable)
- anchortable->handle_mds_recovery(*p);
- anchorclient->handle_mds_recovery(*p);
+ handle_mds_recovery(*p);
}
}
- // did anyone go down?
if (is_active() || is_stopping()) {
+ // did anyone go down?
set<int> failed;
mdsmap->get_mds_set(failed, MDSMap::STATE_FAILED);
for (set<int>::iterator p = failed.begin(); p != failed.end(); ++p) {
- // newly so?
- if (oldfailed.count(*p)) continue;
-
+ if (oldfailed.count(*p)) continue; // newly so?
mdcache->handle_mds_failure(*p);
}
+
+ // did anyone stop?
+ set<int> stopped;
+ mdsmap->get_mds_set(stopped, MDSMap::STATE_STOPPED);
+ for (set<int>::iterator p = stopped.begin(); p != stopped.end(); ++p) {
+ if (oldstopped.count(*p)) continue; // newly so?
+ mdcache->migrator->handle_mds_failure_or_stop(*p);
+ }
}
+
// inst set changed?
/*
if (state >= MDSMap::STATE_ACTIVE && // only if i'm active+. otherwise they'll get map during reconnect.
}
+void MDS::set_want_state(int s)
+{
+ dout(3) << "set_want_state " << MDSMap::get_state_name(s) << endl;
+ want_state = s;
+ beacon_send();
+}
+
void MDS::boot()
{
if (is_creating())
else if (is_replay())
boot_replay(); // replay, join
else
- assert(0);
+ assert(is_standby());
}
mdlog->reset();
mdlog->write_head(fin->new_sub());
- // write our first importmap
- mdcache->log_import_map(fin->new_sub());
+ // write our first subtreemap
+ mdcache->log_subtree_map(fin->new_sub());
// fixme: fake out idalloc (reset, pretend loaded)
dout(10) << "boot_create creating fresh idalloc table" << endl;
idalloc->reset();
idalloc->save(fin->new_sub());
+
+ // write empty clientmap
+ clientmap.save(fin->new_sub());
// fixme: fake out anchortable
if (mdsmap->get_anchortable() == whoami) {
dout(2) << "boot_start opening idalloc" << endl;
idalloc->load(fin->new_sub());
+
+ dout(2) << "boot_start opening clientmap" << endl;
+ clientmap.load(fin->new_sub());
if (mdsmap->get_anchortable() == whoami) {
dout(2) << "boot_start opening anchor table" << endl;
step = 1; // fall-thru.
case 1:
- dout(2) << "boot_replay " << step << ": opening idalloc" << endl;
- idalloc->load(new C_MDS_BootRecover(this, 2));
- break;
+ {
+ C_Gather *gather = new C_Gather(new C_MDS_BootRecover(this, 2));
+ dout(2) << "boot_replay " << step << ": opening idalloc" << endl;
+ idalloc->load(gather->new_sub());
- case 2:
- if (mdsmap->get_anchortable() == whoami) {
- dout(2) << "boot_replay " << step << ": opening anchor table" << endl;
- anchortable->load(new C_MDS_BootRecover(this, 3));
- break;
+ dout(2) << "boot_replay " << step << ": opening clientmap" << endl;
+ clientmap.load(gather->new_sub());
+
+ if (mdsmap->get_anchortable() == whoami) {
+ dout(2) << "boot_replay " << step << ": opening anchor table" << endl;
+ anchortable->load(gather->new_sub());
+ }
}
- dout(2) << "boot_replay " << step << ": i have no anchor table" << endl;
- step++; // fall-thru
+ break;
- case 3:
+ case 2:
dout(2) << "boot_replay " << step << ": opening mds log" << endl;
- mdlog->open(new C_MDS_BootRecover(this, 4));
+ mdlog->open(new C_MDS_BootRecover(this, 3));
break;
- case 4:
+ case 3:
dout(2) << "boot_replay " << step << ": replaying mds log" << endl;
- mdlog->replay(new C_MDS_BootRecover(this, 5));
+ mdlog->replay(new C_MDS_BootRecover(this, 4));
break;
- case 5:
- // done with replay!
- if (mdsmap->get_num_in_mds() == 1) { // me
- dout(2) << "boot_replay " << step << ": i am alone, moving to state reconnect" << endl;
- set_want_state(MDSMap::STATE_RECONNECT);
- } else {
- dout(2) << "boot_replay " << step << ": i am not alone, moving to state resolve" << endl;
- set_want_state(MDSMap::STATE_RESOLVE);
- }
+ case 4:
+ replay_done();
break;
}
}
-void MDS::set_want_state(int s)
+void MDS::replay_start()
{
- dout(3) << "set_want_state " << MDSMap::get_state_name(s) << endl;
- want_state = s;
- beacon_send();
+ dout(1) << "replay_start" << endl;
+
+ // initialize gather sets
+ set<int> rs;
+ mdsmap->get_recovery_mds_set(rs);
+ rs.erase(whoami);
+ dout(1) << "now replay. my recovery peers are " << rs << endl;
+ mdcache->set_recovery_set(rs);
+
+ // note: don't actually start yet. boot() will get called once we have
+ // an mdsmap AND osdmap.
+}
+
+void MDS::replay_done()
+{
+ dout(1) << "replay_done" << endl;
+
+ if (mdsmap->get_num_in_mds() == 1 &&
+ mdsmap->get_num_mds(MDSMap::STATE_FAILED) == 0) { // just me!
+ dout(2) << "i am alone, moving to state reconnect" << endl;
+ set_want_state(MDSMap::STATE_RECONNECT);
+ } else {
+ dout(2) << "i am not alone, moving to state resolve" << endl;
+ set_want_state(MDSMap::STATE_RESOLVE);
+ }
}
+void MDS::resolve_start()
+{
+ dout(1) << "resolve_start" << endl;
+
+ set<int> who;
+ mdsmap->get_mds_set(who, MDSMap::STATE_RESOLVE);
+ mdsmap->get_mds_set(who, MDSMap::STATE_REJOIN);
+ mdsmap->get_mds_set(who, MDSMap::STATE_ACTIVE);
+ mdsmap->get_mds_set(who, MDSMap::STATE_STOPPING);
+ for (set<int>::iterator p = who.begin(); p != who.end(); ++p) {
+ if (*p == whoami) continue;
+ mdcache->send_resolve(*p); // now.
+ }
+}
+void MDS::resolve_done()
+{
+ dout(1) << "resolve_done" << endl;
+ set_want_state(MDSMap::STATE_RECONNECT);
+}
+
+void MDS::reconnect_start()
+{
+ dout(1) << "reconnect_start" << endl;
+ server->reconnect_clients();
+}
+void MDS::reconnect_done()
+{
+ dout(1) << "reconnect_done" << endl;
+ set_want_state(MDSMap::STATE_REJOIN); // move to rejoin state
+
+ /*
+ if (mdsmap->get_num_in_mds() == 1 &&
+ mdsmap->get_num_mds(MDSMap::STATE_FAILED) == 0) { // just me!
+
+ // finish processing caps (normally, this happens during rejoin, but we're skipping that...)
+ mdcache->rejoin_gather_finish();
+ set_want_state(MDSMap::STATE_ACTIVE); // go active
+ } else {
+ set_want_state(MDSMap::STATE_REJOIN); // move to rejoin state
+ }
+ */
+}
+
+void MDS::rejoin_joint_start()
+{
+ dout(1) << "rejoin_joint_start" << endl;
+ mdcache->rejoin_send_rejoins();
+}
+void MDS::rejoin_done()
+{
+ dout(1) << "rejoin_done" << endl;
+ mdcache->show_subtrees();
+ mdcache->show_cache();
+ set_want_state(MDSMap::STATE_ACTIVE);
+}
+
+
+void MDS::recovery_done()
+{
+ dout(1) << "recovery_done -- successful recovery!" << endl;
+ assert(is_active());
+
+ // kick anchortable (resent AGREEs)
+ if (mdsmap->get_anchortable() == whoami)
+ anchortable->finish_recovery();
+
+ // kick anchorclient (resent COMMITs)
+ anchorclient->finish_recovery();
+
+ mdcache->start_recovered_purges();
+
+ // tell connected clients
+ bcast_mds_map();
+}
-int MDS::shutdown_start()
+void MDS::handle_mds_recovery(int who)
+{
+ dout(5) << "handle_mds_recovery mds" << who << endl;
+
+ mdcache->handle_mds_recovery(who);
+
+ if (anchortable)
+ anchortable->handle_mds_recovery(who);
+ anchorclient->handle_mds_recovery(who);
+
+ queue_waiters(waiting_for_active_peer[who]);
+ waiting_for_active_peer.erase(who);
+}
+
+
+void MDS::shutdown_start()
{
dout(1) << "shutdown_start" << endl;
derr(0) << "mds shutdown start" << endl;
// tell everyone to stop.
set<int> active;
- mdsmap->get_active_mds_set(active);
+ mdsmap->get_in_mds_set(active);
for (set<int>::iterator p = active.begin();
p != active.end();
p++) {
// go
set_want_state(MDSMap::STATE_STOPPING);
- return 0;
}
-
void MDS::handle_shutdown_start(Message *m)
{
dout(1) << " handle_shutdown_start" << endl;
- // set flag
set_want_state(MDSMap::STATE_STOPPING);
-
delete m;
}
+void MDS::stopping_start()
+{
+ dout(1) << "stopping_start" << endl;
+
+ // start cache shutdown
+ mdcache->shutdown_start();
+
+ // terminate client sessions
+ server->terminate_sessions();
+
+ // flush log
+ mdlog->set_max_events(0);
+ mdlog->trim(NULL);
+}
+void MDS::stopping_done()
+{
+ dout(1) << "stopping_done" << endl;
+
+ // tell monitor we shut down cleanly.
+ set_want_state(MDSMap::STATE_STOPPED);
+}
+
+
+
int MDS::shutdown_final()
{
dout(1) << "shutdown_final" << endl;
if (logger2) logger2->flush(true);
mdlog->flush_logger();
- // send final down:out beacon (it doesn't matter if this arrives)
- set_want_state(MDSMap::STATE_OUT);
-
// stop timers
if (beacon_killer) {
timer.cancel_event(beacon_killer);
// shut down?
if (is_stopping()) {
if (mdcache->shutdown_pass()) {
- dout(7) << "shutdown_pass=true, finished w/ shutdown, moving to up:stopped" << endl;
-
- // tell monitor we shut down cleanly.
- set_want_state(MDSMap::STATE_STOPPED);
+ dout(7) << "shutdown_pass=true, finished w/ shutdown, moving to down:stopped" << endl;
+ stopping_done();
}
}
-
-class C_LogClientmap : public Context {
- ClientMap *clientmap;
- version_t cmapv;
-public:
- C_LogClientmap(ClientMap *cm, version_t v) :
- clientmap(cm), cmapv(v) {}
- void finish(int r) {
- clientmap->set_committed(cmapv);
- list<Context*> ls;
- clientmap->take_commit_waiters(cmapv, ls);
- finish_contexts(ls);
- }
-};
-
-void MDS::log_clientmap(Context *c)
-{
- dout(10) << "log_clientmap " << clientmap.get_version() << endl;
-
- bufferlist bl;
- clientmap.encode(bl);
-
- clientmap.set_committing(clientmap.get_version());
- clientmap.add_commit_waiter(c);
-
- mdlog->submit_entry(new EClientMap(bl, clientmap.get_version()),
- new C_LogClientmap(&clientmap, clientmap.get_version()));
-}
// -- MDS state --
int state; // my confirmed state
int want_state; // the state i want
- list<Context*> waitfor_active;
+
+ list<Context*> waiting_for_active;
+ map<int, list<Context*> > waiting_for_active_peer;
map<int,version_t> peer_mdsmap_epoch;
public:
- void queue_waitfor_active(Context *c) { waitfor_active.push_back(c); }
+ void wait_for_active(Context *c) {
+ waiting_for_active.push_back(c);
+ }
+ void wait_for_active_peer(int who, Context *c) {
+ waiting_for_active_peer[who].push_back(c);
+ }
bool is_dne() { return state == MDSMap::STATE_DNE; }
- bool is_out() { return state == MDSMap::STATE_OUT; }
bool is_failed() { return state == MDSMap::STATE_FAILED; }
bool is_creating() { return state == MDSMap::STATE_CREATING; }
bool is_starting() { return state == MDSMap::STATE_STARTING; }
version_t beacon_last_seq; // last seq sent to monitor
map<version_t,utime_t> beacon_seq_stamp; // seq # -> time sent
utime_t beacon_last_acked_stamp; // last time we sent a beacon that got acked
- Context *beacon_sender;
- Context *beacon_killer; // next scheduled time of death
+
+ class C_MDS_BeaconSender : public Context {
+ MDS *mds;
+ public:
+ C_MDS_BeaconSender(MDS *m) : mds(m) {}
+ void finish(int r) {
+ mds->beacon_sender = 0;
+ mds->beacon_send();
+ }
+ } *beacon_sender;
+ class C_MDS_BeaconKiller : public Context {
+ MDS *mds;
+ utime_t lab;
+ public:
+ C_MDS_BeaconKiller(MDS *m, utime_t l) : mds(m), lab(l) {}
+ void finish(int r) {
+ mds->beacon_killer = 0;
+ mds->beacon_kill(lab);
+ }
+ } *beacon_killer;
// tick and other timer fun
- Context *tick_event;
+ class C_MDS_Tick : public Context {
+ MDS *mds;
+ public:
+ C_MDS_Tick(MDS *m) : mds(m) {}
+ void finish(int r) {
+ mds->tick_event = 0;
+ mds->tick();
+ }
+ } *tick_event;
void reset_tick();
// -- client map --
ClientMap clientmap;
epoch_t last_client_mdsmap_bcast;
- void log_clientmap(Context *c);
+ //void log_clientmap(Context *c);
// shutdown crap
void send_message_mds(Message *m, int mds, int port=0, int fromport=0);
void forward_message_mds(Message *req, int mds, int port=0);
+ void send_message_client(Message *m, int client);
+ void send_message_client(Message *m, entity_inst_t clientinst);
+ void send_message_client_maybe_open(Message *m, entity_inst_t clientinst);
+
// start up, shutdown
int init(bool standby=false);
void reopen_logger();
+ void bcast_mds_map(); // to mounted clients
+
void boot();
void boot_create(); // i am new mds.
void boot_start(); // i am old but empty (was down:out) mds.
void boot_replay(int step=0); // i am recovering existing (down:failed) mds.
void boot_finish();
- void bcast_mds_map(); // to mounted clients
-
- int shutdown_start();
+ void replay_start();
+ void replay_done();
+ void resolve_start();
+ void resolve_done();
+ void reconnect_start();
+ void reconnect_done();
+ void rejoin_joint_start();
+ void rejoin_done();
+ void recovery_done();
+ void handle_mds_recovery(int who);
+
+ void shutdown_start();
+ void stopping_start();
+ void stopping_done();
int shutdown_final();
void tick();
#include <string>
using namespace std;
+
+/*
+
+ beautiful state diagram:
+
+ STOPPED DNE FAILED
+ / | \ / | |
+ / | \________ _______/ | |
+| v v v v v
+| STARTING <--> STANDBY <--> CREATING REPLAY -> RECONNECT -> REJOIN
+| \ / /
+| \____ ____________/ /
+ \ v v /
+ \ ACTIVE <----------------------------------------/
+ \ |
+ \ |
+ \ v
+ \-- STOPPING
+
+
+
+
+*/
+
+
class MDSMap {
public:
// mds states
- static const int STATE_DNE = 0; // down, never existed.
- static const int STATE_OUT = 1; // down, once existed, but no subtrees, empty log.
- static const int STATE_FAILED = 2; // down, active subtrees; needs to be recovered.
-
- static const int STATE_BOOT = 3; // up, started, joining cluster.
- static const int STATE_STANDBY = 4; // up, but inactive. waiting for assignment by monitor.
- static const int STATE_CREATING = 5; // up, creating MDS instance (new journal, idalloc..)
- static const int STATE_STARTING = 6; // up, starting prior out MDS instance.
- static const int STATE_REPLAY = 7; // up, scanning journal, recoverying any shared state
- static const int STATE_RESOLVE = 8; // up, disambiguating partial distributed operations (import/export, ...rename?)
- static const int STATE_RECONNECT = 9; // up, reconnect to clients
- static const int STATE_REJOIN = 10; // up, replayed journal, rejoining distributed cache
- static const int STATE_ACTIVE = 11; // up, active
- static const int STATE_STOPPING = 12; // up, exporting metadata (-> standby or out)
- static const int STATE_STOPPED = 13; // up, finished stopping. like standby, but not avail to takeover.
+ static const int STATE_DNE = 0; // down, never existed.
+ static const int STATE_STOPPED = -1; // down, once existed, but no subtrees. empty log.
+ static const int STATE_FAILED = 2; // down, active subtrees; needs to be recovered.
+
+ static const int STATE_BOOT = -3; // up, boot announcement. destiny unknown.
+ static const int STATE_STANDBY = -4; // up, idle. waiting for assignment by monitor.
+ static const int STATE_CREATING = -5; // up, creating MDS instance (new journal, idalloc..).
+ static const int STATE_STARTING = -6; // up, starting prior stopped MDS instance.
+
+ static const int STATE_REPLAY = 7; // up, starting prior failed instance. scanning journal.
+ static const int STATE_RESOLVE = 8; // up, disambiguating distributed operations (import, rename, etc.)
+ static const int STATE_RECONNECT = 9; // up, reconnect to clients
+ static const int STATE_REJOIN = 10; // up, replayed journal, rejoining distributed cache
+ static const int STATE_ACTIVE = 11; // up, active
+ static const int STATE_STOPPING = 12; // up, exporting metadata (-> standby or out)
static const char *get_state_name(int s) {
switch (s) {
- // down
+ // down and out
case STATE_DNE: return "down:dne";
- case STATE_OUT: return "down:out";
+ case STATE_STOPPED: return "down:stopped";
+ // down and in
case STATE_FAILED: return "down:failed";
- // up
+ // up and out
case STATE_BOOT: return "up:boot";
- case STATE_STANDBY: return "up:standby";
case STATE_CREATING: return "up:creating";
case STATE_STARTING: return "up:starting";
+ case STATE_STANDBY: return "up:standby";
+ // up and in
case STATE_REPLAY: return "up:replay";
case STATE_RESOLVE: return "up:resolve";
case STATE_RECONNECT: return "up:reconnect";
case STATE_REJOIN: return "up:rejoin";
case STATE_ACTIVE: return "up:active";
case STATE_STOPPING: return "up:stopping";
- case STATE_STOPPED: return "up:stopped";
default: assert(0);
}
return 0;
utime_t created;
epoch_t same_inst_since;
+ int target_num;
int anchortable; // which MDS has anchortable (fixme someday)
int root; // which MDS has root directory
int get_root() const { return root; }
// counts
- int get_num_mds() const { return mds_state.size(); }
+ int get_num_mds() {
+ return get_num_in_mds();
+ }
int get_num_mds(int state) {
int n = 0;
for (map<int,int>::const_iterator p = mds_state.begin();
if (p->second == state) ++n;
return n;
}
- int get_num_in_mds() {
- return get_num_up_mds() - get_num_mds(STATE_STANDBY) - get_num_mds(STATE_STOPPED);
- }
- int get_num_up_mds() {
- int n = 0;
- for (map<int,int>::const_iterator p = mds_state.begin();
- p != mds_state.end();
- p++)
- if (is_up(p->first)) ++n;
- return n;
- }
- int get_num_up_or_failed_mds() {
+
+ int get_num_in_mds() {
int n = 0;
for (map<int,int>::const_iterator p = mds_state.begin();
p != mds_state.end();
p++)
- if (is_up(p->first) || is_failed(p->first))
- ++n;
+ if (p->second > 0) ++n;
return n;
}
p++)
s.insert(p->first);
}
- void get_up_mds_set(set<int>& s) {
+ void get_mds_set(set<int>& s, int state) {
for (map<int,int>::const_iterator p = mds_state.begin();
p != mds_state.end();
p++)
- if (is_up(p->first))
+ if (p->second == state)
s.insert(p->first);
+ }
+ void get_up_mds_set(set<int>& s) {
+ for (map<int,int>::const_iterator p = mds_state.begin();
+ p != mds_state.end();
+ p++)
+ if (is_up(p->first)) s.insert(p->first);
}
- void get_mds_set(set<int>& s, int state) {
+ void get_in_mds_set(set<int>& s) {
for (map<int,int>::const_iterator p = mds_state.begin();
p != mds_state.end();
p++)
- if (p->second == state)
- s.insert(p->first);
+ if (is_in(p->first)) s.insert(p->first);
}
void get_active_mds_set(set<int>& s) {
get_mds_set(s, MDSMap::STATE_ACTIVE);
s.insert(p->first);
}
+ int get_random_in_mds() {
+ vector<int> v;
+ for (map<int,int>::const_iterator p = mds_state.begin();
+ p != mds_state.end();
+ p++)
+ if (p->second > 0) v.push_back(p->first);
+ if (v.empty())
+ return -1;
+ else
+ return v[rand() % v.size()];
+ }
+
// mds states
- bool is_down(int m) { return is_dne(m) || is_out(m) || is_failed(m); }
+ bool is_down(int m) { return is_dne(m) || is_stopped(m) || is_failed(m); }
bool is_up(int m) { return !is_down(m); }
+ bool is_in(int m) { return mds_state.count(m) && mds_state[m] > 0; }
+ bool is_out(int m) { return !mds_state.count(m) || mds_state[m] <= 0; }
bool is_dne(int m) { return mds_state.count(m) == 0 || mds_state[m] == STATE_DNE; }
- bool is_out(int m) { return mds_state.count(m) && mds_state[m] == STATE_OUT; }
bool is_failed(int m) { return mds_state.count(m) && mds_state[m] == STATE_FAILED; }
bool is_boot(int m) { return mds_state.count(m) && mds_state[m] == STATE_BOOT; }
bool has_created(int m) { return mds_created.count(m); }
// cluster states
+ bool is_full() {
+ return get_num_in_mds() >= target_num;
+ }
bool is_degraded() { // degraded = some recovery in process. fixes active membership and recovery_set.
- return get_num_mds(STATE_REPLAY) +
+ return
+ get_num_mds(STATE_REPLAY) +
get_num_mds(STATE_RESOLVE) +
get_num_mds(STATE_RECONNECT) +
get_num_mds(STATE_REJOIN) +
get_num_mds(STATE_FAILED);
}
- /*bool is_resolving() { // nodes are resolving distributed ops
- return get_num_mds(STATE_RESOLVE);
- }*/
bool is_rejoining() {
// nodes are rejoining cache state
- return get_num_mds(STATE_REJOIN) > 0 &&
+ return
+ get_num_mds(STATE_REJOIN) > 0 &&
+ get_num_mds(STATE_REPLAY) == 0 &&
get_num_mds(STATE_RECONNECT) == 0 &&
get_num_mds(STATE_RESOLVE) == 0 &&
- get_num_mds(STATE_REPLAY) == 0 &&
get_num_mds(STATE_FAILED) == 0;
}
+ bool is_stopped() {
+ return
+ get_num_in_mds() == 0 &&
+ get_num_mds(STATE_CREATING) == 0 &&
+ get_num_mds(STATE_STARTING) == 0 &&
+ get_num_mds(STATE_STANDBY) == 0;
+ }
int get_state(int m) {
- if (mds_state.count(m)) return mds_state[m];
- return STATE_OUT;
+ if (mds_state.count(m))
+ return mds_state[m];
+ else
+ return STATE_DNE;
}
// inst
// serialize, unserialize
- void encode(bufferlist& blist) {
- blist.append((char*)&epoch, sizeof(epoch));
- blist.append((char*)&created, sizeof(created));
- blist.append((char*)&same_inst_since, sizeof(same_inst_since));
- blist.append((char*)&anchortable, sizeof(anchortable));
- blist.append((char*)&root, sizeof(root));
-
- ::_encode(mds_state, blist);
- ::_encode(mds_state_seq, blist);
- ::_encode(mds_inst, blist);
- ::_encode(mds_inc, blist);
+ void encode(bufferlist& bl) {
+ ::_encode(epoch, bl);
+ ::_encode(target_num, bl);
+ ::_encode(created, bl);
+ ::_encode(same_inst_since, bl);
+ ::_encode(anchortable, bl);
+ ::_encode(root, bl);
+ ::_encode(mds_state, bl);
+ ::_encode(mds_state_seq, bl);
+ ::_encode(mds_inst, bl);
+ ::_encode(mds_inc, bl);
}
- void decode(bufferlist& blist) {
+ void decode(bufferlist& bl) {
int off = 0;
- blist.copy(off, sizeof(epoch), (char*)&epoch);
- off += sizeof(epoch);
- blist.copy(off, sizeof(created), (char*)&created);
- off += sizeof(created);
- blist.copy(off, sizeof(same_inst_since), (char*)&same_inst_since);
- off += sizeof(same_inst_since);
- blist.copy(off, sizeof(anchortable), (char*)&anchortable);
- off += sizeof(anchortable);
- blist.copy(off, sizeof(root), (char*)&root);
- off += sizeof(root);
-
- ::_decode(mds_state, blist, off);
- ::_decode(mds_state_seq, blist, off);
- ::_decode(mds_inst, blist, off);
- ::_decode(mds_inc, blist, off);
+ ::_decode(epoch, bl, off);
+ ::_decode(target_num, bl, off);
+ ::_decode(created, bl, off);
+ ::_decode(same_inst_since, bl, off);
+ ::_decode(anchortable, bl, off);
+ ::_decode(root, bl, off);
+ ::_decode(mds_state, bl, off);
+ ::_decode(mds_state_seq, bl, off);
+ ::_decode(mds_inst, bl, off);
+ ::_decode(mds_inc, bl, off);
}
#include "messages/MExportDirNotifyAck.h"
#include "messages/MExportDirFinish.h"
-#include "messages/MHashDirDiscover.h"
-#include "messages/MHashDirDiscoverAck.h"
-#include "messages/MHashDirPrep.h"
-#include "messages/MHashDirPrepAck.h"
-#include "messages/MHashDir.h"
-#include "messages/MHashDirNotify.h"
-#include "messages/MHashDirAck.h"
-
-#include "messages/MUnhashDirPrep.h"
-#include "messages/MUnhashDirPrepAck.h"
-#include "messages/MUnhashDir.h"
-#include "messages/MUnhashDirAck.h"
-#include "messages/MUnhashDirNotify.h"
-#include "messages/MUnhashDirNotifyAck.h"
-
-
#include "config.h"
#undef dout
#define dout(l) if (l<=g_conf.debug || l <= g_conf.debug_mds) cout << g_clock.now() << " mds" << mds->get_nodeid() << ".migrator "
case MSG_MDS_EXPORTDIRFINISH:
handle_export_finish((MExportDirFinish*)m);
break;
+ case MSG_MDS_EXPORTDIRCANCEL:
+ handle_export_cancel((MExportDirCancel*)m);
+ break;
// export
case MSG_MDS_EXPORTDIRDISCOVERACK:
// ==========================================================
// mds failure handling
-void Migrator::handle_mds_failure(int who)
+void Migrator::handle_mds_failure_or_stop(int who)
{
- dout(5) << "handle_mds_failure mds" << who << endl;
+ dout(5) << "handle_mds_failure_or_stop mds" << who << endl;
// check my exports
map<CDir*,int>::iterator p = export_state.begin();
// abort exports:
// - that are going to the failed node
- // - that aren't frozen yet (to about auth_pin deadlock)
+ // - that aren't frozen yet (to avoid auth_pin deadlock)
if (export_peer[dir] == who ||
p->second == EXPORT_DISCOVERING || p->second == EXPORT_FREEZING) {
// the guy i'm exporting to failed, or we're just freezing.
dir->auth_unpin();
export_state.erase(dir); // clean up
dir->state_clear(CDir::STATE_EXPORTING);
+ dir->put(CDir::PIN_EXPORTING);
if (export_peer[dir] != who) // tell them.
- mds->send_message_mds(new MExportDirCancel(dir->dirfrag()), who, MDS_PORT_MIGRATOR);
+ mds->send_message_mds(new MExportDirCancel(dir->dirfrag()), export_peer[dir], MDS_PORT_MIGRATOR);
break;
case EXPORT_FREEZING:
dir->unfreeze_tree(); // cancel the freeze
export_state.erase(dir); // clean up
dir->state_clear(CDir::STATE_EXPORTING);
+ dir->put(CDir::PIN_EXPORTING);
if (export_peer[dir] != who) // tell them.
- mds->send_message_mds(new MExportDirCancel(dir->dirfrag()), who, MDS_PORT_MIGRATOR);
+ mds->send_message_mds(new MExportDirCancel(dir->dirfrag()), export_peer[dir], MDS_PORT_MIGRATOR);
break;
// NOTE: state order reversal, warning comes after loggingstart+prepping
cache->try_subtree_merge(dir);
export_state.erase(dir); // clean up
dir->state_clear(CDir::STATE_EXPORTING);
+ dir->put(CDir::PIN_EXPORTING);
break;
case EXPORT_EXPORTING:
export_reverse(dir);
export_state.erase(dir); // clean up
dir->state_clear(CDir::STATE_EXPORTING);
+ dir->put(CDir::PIN_EXPORTING);
break;
case EXPORT_LOGGINGFINISH:
export_finish_waiters.erase(dir);
// send pending import_maps? (these need to go out when all exports have finished.)
- cache->send_pending_import_maps();
+ cache->maybe_send_pending_resolves();
cache->show_subtrees();
}
} else {
// bystander failed.
- if (p->second == EXPORT_WARNING) {
- // exporter waiting for warning acks, let's fake theirs.
- if (export_warning_ack_waiting[dir].count(who)) {
+ if (export_warning_ack_waiting.count(dir) &&
+ export_warning_ack_waiting[dir].count(who)) {
+ export_warning_ack_waiting[dir].erase(who);
+ export_notify_ack_waiting[dir].erase(who); // they won't get a notify either.
+ if (p->second == EXPORT_WARNING) {
+ // exporter waiting for warning acks, let's fake theirs.
dout(10) << "faking export_warning_ack from mds" << who
<< " on " << *dir << " to mds" << export_peer[dir]
<< endl;
- export_warning_ack_waiting[dir].erase(who);
- export_notify_ack_waiting[dir].erase(who); // they won't get a notify either.
if (export_warning_ack_waiting[dir].empty())
export_go(dir);
}
}
- if (p->second == EXPORT_NOTIFYING) {
- // exporter is waiting for notify acks, fake it
- if (export_notify_ack_waiting[dir].count(who)) {
+ if (export_notify_ack_waiting.count(dir) &&
+ export_notify_ack_waiting[dir].count(who)) {
+ export_notify_ack_waiting[dir].erase(who);
+ if (p->second == EXPORT_NOTIFYING) {
+ // exporter is waiting for notify acks, fake it
dout(10) << "faking export_notify_ack from mds" << who
<< " on " << *dir << " to mds" << export_peer[dir]
<< endl;
- export_notify_ack_waiting[dir].erase(who);
if (export_notify_ack_waiting[dir].empty())
export_finish(dir);
}
+void Migrator::show_importing()
+{
+ dout(10) << "show_importing" << endl;
+ for (map<dirfrag_t,int>::iterator p = import_state.begin();
+ p != import_state.end();
+ p++) {
+ CDir *dir = mds->mdcache->get_dirfrag(p->first);
+ if (dir) {
+ dout(10) << " importing to " << import_peer[p->first]
+ << ": (" << p->second << ") " << get_import_statename(p->second)
+ << " " << p->first
+ << " " << *dir
+ << endl;
+ } else {
+ dout(10) << " importing to " << import_peer[p->first]
+ << ": (" << p->second << ") " << get_import_statename(p->second)
+ << " " << p->first
+ << endl;
+ }
+ }
+}
+
+void Migrator::show_exporting()
+{
+ dout(10) << "show_exporting" << endl;
+ for (map<CDir*,int>::iterator p = export_state.begin();
+ p != export_state.end();
+ p++)
+ dout(10) << " exporting to " << export_peer[p->first]
+ << ": (" << p->second << ") " << get_export_statename(p->second)
+ << " " << p->first->dirfrag()
+ << " " << *p->first
+ << endl;
+}
+
void Migrator::audit()
if (g_conf.debug_mds < 5) return; // hrm.
// import_state
+ show_importing();
for (map<dirfrag_t,int>::iterator p = import_state.begin();
p != import_state.end();
p++) {
}
// export_state
+ show_exporting();
for (map<CDir*,int>::iterator p = export_state.begin();
p != export_state.end();
p++) {
// pin path?
vector<CDentry*> trace;
cache->make_trace(trace, dir->inode);
- if (!mds->locker->dentry_can_rdlock_trace(trace, 0)) {
+ if (!mds->locker->dentry_can_rdlock_trace(trace)) {
dout(7) << "export_dir couldn't pin path, failing." << endl;
return;
}
export_peer[dir] = dest;
dir->state_set(CDir::STATE_EXPORTING);
+ dir->get(CDir::PIN_EXPORTING);
// send ExportDirDiscover (ask target)
mds->send_message_mds(new MExportDirDiscover(dir), export_peer[dir], MDS_PORT_MIGRATOR);
// fill export message with cache data
C_Contexts *fin = new C_Contexts; // collect all the waiters
+ map<int,entity_inst_t> exported_client_map;
int num_exported_inodes = encode_export_dir( export_data[dir],
fin,
dir, // base
dir, // recur start point
- dest );
-
+ dest,
+ exported_client_map );
+ bufferlist bl;
+ ::_encode(exported_client_map, bl);
+ export_data[dir].push_front(bl);
+
// send the export data!
MExportDir *req = new MExportDir(dir->dirfrag());
* encode relevant state to be sent over the wire.
* used by: encode_export_dir, file_rename (if foreign)
*/
-void Migrator::encode_export_inode(CInode *in, bufferlist& enc_state, int new_auth)
+void Migrator::encode_export_inode(CInode *in, bufferlist& enc_state, int new_auth,
+ map<int,entity_inst_t>& exported_client_map)
{
// tell (all) clients about migrating caps.. mark STALE
for (map<int, Capability>::iterator it = in->client_caps.begin();
it != in->client_caps.end();
it++) {
dout(7) << "encode_export_inode " << *in << " telling client" << it->first << " stale caps" << endl;
- MClientFileCaps *m = new MClientFileCaps(in->inode,
+ MClientFileCaps *m = new MClientFileCaps(MClientFileCaps::OP_STALE,
+ in->inode,
it->second.get_last_seq(),
it->second.pending(),
- it->second.wanted(),
- MClientFileCaps::OP_STALE);
- mds->messenger->send_message(m, mds->clientmap.get_inst(it->first),
- 0, MDS_PORT_CACHE);
+ it->second.wanted());
+ entity_inst_t inst = mds->clientmap.get_inst(it->first);
+ exported_client_map[it->first] = inst;
+ mds->send_message_client(m, inst);
}
// relax locks?
in->clear_replicas();
// twiddle lock states for auth -> replica transition
- // auth
- in->authlock.clear_gather();
- if (in->authlock.get_state() == LOCK_GLOCKR)
- in->authlock.set_state(LOCK_LOCK);
-
- // link
- in->linklock.clear_gather();
- if (in->linklock.get_state() == LOCK_GLOCKR)
- in->linklock.set_state(LOCK_LOCK);
-
- // dirfragtree
- in->dirfragtreelock.clear_gather();
- if (in->dirfragtreelock.get_state() == LOCK_GLOCKR)
- in->dirfragtreelock.set_state(LOCK_LOCK);
-
- // file : we lost all our caps, so move to stable state!
- in->filelock.clear_gather();
- if (in->filelock.get_state() == LOCK_GLOCKR ||
- in->filelock.get_state() == LOCK_GLOCKM ||
- in->filelock.get_state() == LOCK_GLOCKL ||
- in->filelock.get_state() == LOCK_GLONERR ||
- in->filelock.get_state() == LOCK_GLONERM ||
- in->filelock.get_state() == LOCK_LONER)
- in->filelock.set_state(LOCK_LOCK);
- if (in->filelock.get_state() == LOCK_GMIXEDR)
- in->filelock.set_state(LOCK_MIXED);
- // this looks like a step backwards, but it's what we want!
- if (in->filelock.get_state() == LOCK_GSYNCM)
- in->filelock.set_state(LOCK_MIXED);
- if (in->filelock.get_state() == LOCK_GSYNCL)
- in->filelock.set_state(LOCK_LOCK);
- if (in->filelock.get_state() == LOCK_GMIXEDL)
- in->filelock.set_state(LOCK_LOCK);
- //in->filelock.set_state(LOCK_MIXED);
-
+ in->authlock.export_twiddle();
+ in->linklock.export_twiddle();
+ in->dirfragtreelock.export_twiddle();
+ in->filelock.export_twiddle();
+ in->dirlock.export_twiddle();
+
// mark auth
assert(in->is_auth());
in->state_clear(CInode::STATE_AUTH);
int Migrator::encode_export_dir(list<bufferlist>& dirstatelist,
- C_Contexts *fin,
- CDir *basedir,
- CDir *dir,
- int newauth)
+ C_Contexts *fin,
+ CDir *basedir,
+ CDir *dir,
+ int newauth,
+ map<int,entity_inst_t>& exported_client_map)
{
int num_exported = 0;
// -- inode
enc_dir.append("I", 1); // inode dentry
- encode_export_inode(in, enc_dir, newauth); // encode, and (update state for) export
+ encode_export_inode(in, enc_dir, newauth, exported_client_map); // encode, and (update state for) export
// directory?
list<CDir*> dfs;
// subdirs
for (list<CDir*>::iterator it = subdirs.begin(); it != subdirs.end(); it++)
- num_exported += encode_export_dir(dirstatelist, fin, basedir, *it, newauth);
+ num_exported += encode_export_dir(dirstatelist, fin, basedir, *it, newauth,
+ exported_client_map);
return num_exported;
}
}
// re-import the metadata
- int num_imported_inodes = 0;
-
- for (list<bufferlist>::iterator p = export_data[dir].begin();
- p != export_data[dir].end();
- ++p) {
- num_imported_inodes +=
- decode_import_dir(*p,
- export_peer[dir],
- dir, // import root
- 0);
+ map<int,entity_inst_t> imported_client_map;
+ int off = 0;
+ ::_decode(imported_client_map, export_data[dir].front(), off);
+ export_data[dir].pop_front();
+
+ while (!export_data[dir].empty()) {
+ decode_import_dir(export_data[dir].front(),
+ export_peer[dir],
+ dir, // import root
+ 0,
+ imported_client_map);
+ export_data[dir].pop_front();
}
// process delayed expires
// remove from exporting list, clean up state
dir->state_clear(CDir::STATE_EXPORTING);
+ dir->put(CDir::PIN_EXPORTING);
export_state.erase(dir);
export_peer.erase(dir);
export_bounds.erase(dir);
audit();
// send pending import_maps?
- mds->mdcache->send_pending_import_maps();
+ mds->mdcache->maybe_send_pending_resolves();
}
// must discover it!
filepath fpath(m->get_path());
vector<CDentry*> trace;
- int r = cache->path_traverse(0,
- 0,
- fpath, trace, true,
- m, new C_MDS_RetryMessage(mds, m), // on delay/retry
+ int r = cache->path_traverse(0, m,
+ 0, fpath, trace, true,
MDS_TRAVERSE_DISCOVER);
if (r > 0) return; // wait
if (r < 0) {
cache->verify_subtree_bounds(dir, import_bounds[dir]);
// add this crap to my cache
- int num_imported_inodes = 0;
+ map<int,entity_inst_t> imported_client_map;
+ int off = 0;
+ ::_decode(imported_client_map, m->get_dirstate().front(), off);
+ m->get_dirstate().pop_front();
- for (list<bufferlist>::iterator p = m->get_dirstate().begin();
- p != m->get_dirstate().end();
- ++p) {
+ int num_imported_inodes = 0;
+ while (!m->get_dirstate().empty()) {
num_imported_inodes +=
- decode_import_dir(*p,
- oldauth,
- dir, // import root
- le);
+ decode_import_dir(m->get_dirstate().front(),
+ oldauth,
+ dir, // import root
+ le,
+ imported_client_map);
+ m->get_dirstate().pop_front();
}
dout(10) << " " << m->get_bounds().size() << " imported bounds" << endl;
import_bystanders.erase(dir);
cache->show_subtrees();
- audit();
+ //audit(); // this fails, bc we munge up the subtree map during handle_import_map (resolve phase)
}
dout(5) << "finishing any waiters on imported data" << endl;
dir->finish_waiting(CDir::WAIT_IMPORTED);
- // log it
- if (mds->logger) {
- //mds->logger->set("nex", cache->exports.size());
- //mds->logger->set("nim", cache->imports.size());
- }
cache->show_subtrees();
- audit();
+ //audit(); // this fails, bc we munge up the subtree map during handle_import_map (resolve phase)
+
// is it empty?
if (dir->get_size() == 0 &&
}
-void Migrator::decode_import_inode(CDentry *dn, bufferlist& bl, int& off, int oldauth)
+void Migrator::decode_import_inode(CDentry *dn, bufferlist& bl, int& off, int oldauth,
+ map<int,entity_inst_t>& imported_client_map)
{
dout(15) << "decode_import_inode on " << *dn << endl;
in->remove_replica(mds->get_nodeid());
// twiddle locks
+ /*
if (in->authlock.do_import(oldauth, mds->get_nodeid()))
mds->locker->simple_eval(&in->authlock);
if (in->linklock.do_import(oldauth, mds->get_nodeid()))
mds->locker->simple_eval(&in->linklock);
if (in->dirfragtreelock.do_import(oldauth, mds->get_nodeid()))
mds->locker->simple_eval(&in->dirfragtreelock);
+ if (in->dirlock.do_import(oldauth, mds->get_nodeid()))
+ mds->locker->scatter_eval(&in->dirlock);
+ */
// caps
for (set<int>::iterator it = merged_client_caps.begin();
it != merged_client_caps.end();
it++) {
- MClientFileCaps *caps = new MClientFileCaps(in->inode,
+ MClientFileCaps *caps = new MClientFileCaps(MClientFileCaps::OP_REAP,
+ in->inode,
in->client_caps[*it].get_last_seq(),
in->client_caps[*it].pending(),
- in->client_caps[*it].wanted(),
- MClientFileCaps::OP_REAP);
+ in->client_caps[*it].wanted());
caps->set_mds( oldauth ); // reap from whom?
- mds->messenger->send_message(caps,
- mds->clientmap.get_inst(*it),
- 0, MDS_PORT_CACHE);
+ mds->send_message_client_maybe_open(caps, imported_client_map[*it]);
}
// filelock
+ /*
if (in->filelock.do_import(oldauth, mds->get_nodeid()))
mds->locker->simple_eval(&in->filelock);
+ */
}
int Migrator::decode_import_dir(bufferlist& bl,
int oldauth,
CDir *import_root,
- EImportStart *le)
+ EImportStart *le,
+ map<int,entity_inst_t>& imported_client_map)
{
int off = 0;
// add to journal entry
if (le)
- le->metablob.add_dir(dir, true); // Hmm: false would be okay in some cases
+ le->metablob.add_dir(dir,
+ true, // Hmm: dirty=false would be okay in some cases
+ dir->is_complete());
int num_imported = 0;
}
else if (icode == 'I') {
// inode
- decode_import_inode(dn, bl, off, oldauth);
+ decode_import_inode(dn, bl, off, oldauth, imported_client_map);
}
// add dentry to journal entry
const static int EXPORT_LOGGINGFINISH = 7; // logging EExportFinish
const static int EXPORT_NOTIFYING = 8; // waiting for notifyacks
const static int EXPORT_ABORTING = 9; // notifying bystanders of abort
+ static const char *get_export_statename(int s) {
+ switch (s) {
+ case EXPORT_DISCOVERING: return "discovering";
+ case EXPORT_FREEZING: return "freezing";
+ case EXPORT_PREPPING: return "prepping";
+ case EXPORT_WARNING: return "warning";
+ case EXPORT_EXPORTING: return "exporting";
+ case EXPORT_LOGGINGFINISH: return "loggingfinish";
+ case EXPORT_NOTIFYING: return "notifying";
+ case EXPORT_ABORTING: return "aborting";
+ default: assert(0); return 0;
+ }
+ }
protected:
// export fun
const static int IMPORT_ACKING = 6; // logged EImportStart, sent ack, waiting for finish
//const static int IMPORT_LOGGINGFINISH = 7; // logging EImportFinish
const static int IMPORT_ABORTING = 8; // notifying bystanders of an abort before unfreezing
+ static const char *get_import_statename(int s) {
+ switch (s) {
+ case IMPORT_DISCOVERING: return "discovering";
+ case IMPORT_DISCOVERED: return "discovered";
+ case IMPORT_PREPPING: return "prepping";
+ case IMPORT_PREPPED: return "prepped";
+ case IMPORT_LOGGINGSTART: return "loggingstart";
+ case IMPORT_ACKING: return "acking";
+ case IMPORT_ABORTING: return "aborting";
+ default: assert(0); return 0;
+ }
+ }
protected:
map<dirfrag_t,int> import_state; // FIXME make these dirfrags
void dispatch(Message*);
+ void show_importing();
+ void show_exporting();
// -- status --
int is_exporting(CDir *dir) {
// -- misc --
- void handle_mds_failure(int who);
+ void handle_mds_failure_or_stop(int who);
void audit();
void export_dir(CDir *dir, int dest);
void export_empty_import(CDir *dir);
- void encode_export_inode(CInode *in, bufferlist& enc_state, int newauth);
- void decode_import_inode(CDentry *dn, bufferlist& bl, int &off, int oldauth);
+ void encode_export_inode(CInode *in, bufferlist& enc_state, int newauth,
+ map<int,entity_inst_t>& exported_client_map);
+ int encode_export_dir(list<bufferlist>& dirstatelist,
+ class C_Contexts *fin,
+ CDir *basedir,
+ CDir *dir,
+ int newauth,
+ map<int,entity_inst_t>& exported_client_map);
void add_export_finish_waiter(CDir *dir, Context *c) {
export_finish_waiters[dir].push_back(c);
void export_frozen(CDir *dir);
void handle_export_prep_ack(MExportDirPrepAck *m);
void export_go(CDir *dir);
- int encode_export_dir(list<bufferlist>& dirstatelist,
- class C_Contexts *fin,
- CDir *basedir,
- CDir *dir,
- int newauth);
void export_reverse(CDir *dir);
void handle_export_ack(MExportDirAck *m);
void export_logged_finish(CDir *dir);
void import_discovered(CInode *in, dirfrag_t df);
void handle_export_prep(MExportDirPrep *m);
void handle_export_dir(MExportDir *m);
+
+public:
+ void decode_import_inode(CDentry *dn, bufferlist& bl, int &off, int oldauth,
+ map<int,entity_inst_t>& imported_client_map);
int decode_import_dir(bufferlist& bl,
int oldauth,
CDir *import_root,
- EImportStart *le);
-
+ EImportStart *le,
+ map<int,entity_inst_t>& imported_client_map);
public:
void import_reverse(CDir *dir, bool fix_dir_auth=true);
#include "SimpleLock.h"
-// lock state machine states.
-#define LOCK_SYNC__ // rdlocks allowed (e.g., for stat)
-#define LOCK_GSYNCS -20 // waiting for replicas to gather
-#define LOCK_SCATTER 21 // mtime updates on replicas allowed, no reads.
-#define LOCK_GSCATTERS 22 // waiting for rdlocks to release
+// lock state machine states:
+// Sync -- Lock -- sCatter
+// Tempsync _/
+// auth repl
+#define LOCK_SYNC__ // R . R . rdlocks allowed on auth and replicas
+#define LOCK_GLOCKS -20 // r . r . waiting for replicas+rdlocks (auth), or rdlocks to release (replica)
+#define LOCK_GSCATTERS -28 // r . r .
+
+#define LOCK_GSYNCL__ // . w LOCK on replica.
+#define LOCK_LOCK__ // . W . .
+#define LOCK_GTEMPSYNCL -21 // . w LOCK on replica.
+
+#define LOCK_GLOCKC -22 // . wp . wp waiting for replicas+wrlocks (auth), or wrlocks to release (replica)
+#define LOCK_SCATTER 23 // . Wp . WP mtime updates on replicas allowed, no reads. stable here.
+#define LOCK_GTEMPSYNCC -24 // . wp . wp GLOCKC|LOCK on replica
+
+#define LOCK_GSCATTERT -25 // r . LOCK on replica.
+#define LOCK_GLOCKT -26 // r . LOCK on replica.
+#define LOCK_TEMPSYNC 27 // R . LOCK on replica.
+
inline const char *get_scatterlock_state_name(int s) {
switch(s) {
- case LOCK_SYNC: return "sync";
- case LOCK_GSYNCS: return "gsyncs";
- case LOCK_SCATTER: return "scatter";
- case LOCK_GSCATTERS: return "gscatters";
+ case LOCK_SYNC: return "Sync";
+ case LOCK_GLOCKS: return "gLockS";
+ case LOCK_GSCATTERS: return "gScatterS";
+
+ case LOCK_GSYNCL: return "gSyncL";
+ case LOCK_LOCK: return "Lock";
+ case LOCK_GTEMPSYNCL: return "gTempsyncL";
+
+ case LOCK_GLOCKC: return "gLockC";
+ case LOCK_SCATTER: return "sCatter";
+ case LOCK_GTEMPSYNCC: return "gTempsyncC";
+
+ case LOCK_GSCATTERT: return "gsCatterT";
+ case LOCK_GLOCKT: return "gLockT";
+ case LOCK_TEMPSYNC: return "Tempsync";
+
default: assert(0);
}
}
class ScatterLock : public SimpleLock {
int num_wrlock;
-
+ bool updated;
+
public:
- ScatterLock(MDSCacheObject *o, int t, int wo) : SimpleLock(o, t, wo),
- num_wrlock(0) {}
+ ScatterLock(MDSCacheObject *o, int t, int wo) :
+ SimpleLock(o, t, wo),
+ num_wrlock(0),
+ updated(false) {}
int get_replica_state() {
switch (state) {
case LOCK_SYNC:
- case LOCK_GSYNCS:
- case LOCK_GSCATTERS:
return LOCK_SYNC;
+
+ case LOCK_GSCATTERS: // hrm.
+ case LOCK_GLOCKS:
+ case LOCK_GSYNCL:
+ case LOCK_LOCK:
+ case LOCK_GTEMPSYNCL:
+ case LOCK_GLOCKC:
+ return LOCK_LOCK;
+
case LOCK_SCATTER:
return LOCK_SCATTER;
+
+ case LOCK_GTEMPSYNCC:
+ case LOCK_GSCATTERT:
+ case LOCK_GLOCKT:
+ case LOCK_TEMPSYNC:
+ return LOCK_LOCK;
default:
assert(0);
}
}
+ void set_updated() {
+ if (!updated) {
+ parent->get(MDSCacheObject::PIN_DIRTYSCATTERED);
+ updated = true;
+ }
+ }
+ void clear_updated() {
+ if (updated) {
+ parent->put(MDSCacheObject::PIN_DIRTYSCATTERED);
+ updated = false;
+ }
+ }
+ bool is_updated() { return updated; }
+
void replicate_relax() {
- if (state == LOCK_SYNC && !is_rdlocked())
- state = LOCK_SCATTER;
+ //if (state == LOCK_SYNC && !is_rdlocked())
+ //state = LOCK_SCATTER;
+ }
+
+ void export_twiddle() {
+ clear_gather();
+ state = get_replica_state();
}
// rdlock
bool can_rdlock(MDRequest *mdr) {
- return state == LOCK_SYNC;
+ return state == LOCK_SYNC || state == LOCK_TEMPSYNC;
}
bool can_rdlock_soon() {
- return state == LOCK_SYNC || state == LOCK_GSYNCS;
+ return state == LOCK_GTEMPSYNCC;
+ }
+
+ // xlock
+ bool can_xlock_soon() {
+ if (parent->is_auth())
+ return (state == LOCK_GLOCKC ||
+ state == LOCK_GLOCKS);
+ else
+ return false;
}
// wrlock
bool can_wrlock() {
- return state == LOCK_SCATTER;
+ return state == LOCK_SCATTER || state == LOCK_LOCK;
}
void get_wrlock() {
- assert(state == LOCK_SCATTER);
+ assert(can_wrlock());
+ if (num_wrlock == 0) parent->get(MDSCacheObject::PIN_LOCK);
++num_wrlock;
}
void put_wrlock() {
--num_wrlock;
+ if (num_wrlock == 0) parent->put(MDSCacheObject::PIN_LOCK);
}
bool is_wrlocked() { return num_wrlock > 0; }
int get_num_wrlocks() { return num_wrlock; }
if (!get_gather_set().empty()) out << " g=" << get_gather_set();
if (is_rdlocked())
out << " r=" << get_num_rdlocks();
- //if (l.is_xlocked())
- //out << " x=" << l.get_xlocked_by();
+ if (is_xlocked())
+ out << " x=" << get_xlocked_by();
if (is_wrlocked())
out << " wr=" << get_num_wrlocks();
out << ")";
#include "Migrator.h"
#include "MDBalancer.h"
#include "AnchorClient.h"
+#include "IdAllocator.h"
#include "msg/Messenger.h"
#include "messages/MClientRequest.h"
#include "messages/MClientReply.h"
#include "messages/MClientReconnect.h"
+#include "messages/MClientFileCaps.h"
+
+#include "messages/MMDSSlaveRequest.h"
#include "messages/MLock.h"
#include "messages/MDentryUnlink.h"
-#include "messages/MInodeLink.h"
#include "events/EString.h"
#include "events/EUpdate.h"
+#include "events/ESlaveUpdate.h"
#include "events/ESession.h"
#include "events/EOpen.h"
// active?
if (!mds->is_active()) {
dout(3) << "not active yet, waiting" << endl;
- mds->queue_waitfor_active(new C_MDS_RetryMessage(mds, m));
+ mds->wait_for_active(new C_MDS_RetryMessage(mds, m));
return;
}
case MSG_CLIENT_REQUEST:
handle_client_request((MClientRequest*)m);
return;
+ case MSG_MDS_SLAVE_REQUEST:
+ handle_slave_request((MMDSSlaveRequest*)m);
+ return;
}
dout(1) << "server unknown message " << m->get_type() << endl;
{
dout(3) << "handle_client_session " << *m << " from " << m->get_source() << endl;
int from = m->get_source().num();
- bool open = m->op == MClientSession::OP_OPEN;
+ bool open = m->op == MClientSession::OP_REQUEST_OPEN;
if (open) {
+ if (mds->clientmap.have_session(from)) {
+ dout(10) << "already open, dropping this req" << endl;
+ delete m;
+ return;
+ }
if (mds->clientmap.is_opening(from)) {
dout(10) << "already opening, dropping this req" << endl;
delete m;
delete m;
return;
}
+ if (m->seq < mds->clientmap.get_push_seq(from)) {
+ dout(10) << "old push seq " << m->seq << " < " << mds->clientmap.get_push_seq(from)
+ << ", dropping" << endl;
+ delete m;
+ return;
+ }
+ assert(m->seq == mds->clientmap.get_push_seq(from));
+
mds->clientmap.add_closing(from);
}
// reply
if (open)
- mds->messenger->send_message(new MClientSession(MClientSession::OP_OPEN_ACK), client_inst);
+ mds->messenger->send_message(new MClientSession(MClientSession::OP_OPEN), client_inst);
else
- mds->messenger->send_message(new MClientSession(MClientSession::OP_CLOSE_ACK), client_inst);
+ mds->messenger->send_message(new MClientSession(MClientSession::OP_CLOSE), client_inst);
}
// reconnect with clients
if (mds->clientmap.get_session_set().empty()) {
dout(7) << "reconnect_clients -- no sessions, doing nothing." << endl;
- reconnect_finish();
+ reconnect_gather_finish();
return;
}
dout(7) << "reconnect_clients -- sending mdsmap to clients with sessions" << endl;
- mds->set_want_state(MDSMap::STATE_RECONNECT); // just fyi.
- // send mdsmap to all mounted clients
- mds->bcast_mds_map();
+ mds->bcast_mds_map(); // send mdsmap to all client sessions
// init gather list
reconnect_start = g_clock.now();
new C_MDS_session_finish(mds, mds->clientmap.get_inst(from), false, cmapv));
} else {
-
+
// caps
- for (map<inodeno_t, MClientReconnect::inode_caps_t>::iterator p = m->inode_caps.begin();
+ for (map<inodeno_t, inode_caps_reconnect_t>::iterator p = m->inode_caps.begin();
p != m->inode_caps.end();
++p) {
CInode *in = mdcache->get_inode(p->first);
- if (!in) {
- dout(0) << "missing " << p->first << ", fetching via " << m->inode_path[p->first] << endl;
- assert(0);
+ if (in && in->is_auth()) {
+ // we recovered it, and it's ours. take note.
+ dout(15) << "open caps on " << *in << endl;
+ in->reconnect_cap(from, p->second);
+ reconnected_caps.insert(in);
continue;
}
- dout(10) << " client cap " << cap_string(p->second.wanted)
- << " seq " << p->second.seq
- << " on " << *in << endl;
- Capability cap(p->second.wanted, p->second.seq);
- in->add_client_cap(from, cap);
- in->inode.size = MAX(in->inode.size, p->second.size);
- in->inode.mtime = MAX(in->inode.mtime, p->second.mtime);
- in->inode.atime = MAX(in->inode.atime, p->second.atime);
-
- reconnected_open_files.insert(in);
+ filepath path = m->inode_path[p->first];
+ if ((in && !in->is_auth()) ||
+ !mds->mdcache->path_is_mine(path)) {
+ // not mine.
+ dout(0) << "non-auth " << p->first << " " << m->inode_path[p->first]
+ << ", will pass off to authority" << endl;
+
+ // mark client caps stale.
+ inode_t fake_inode;
+ fake_inode.ino = p->first;
+ MClientFileCaps *stale = new MClientFileCaps(MClientFileCaps::OP_STALE,
+ fake_inode,
+ 0,
+ 0, // doesn't matter.
+ p->second.wanted); // doesn't matter.
+ mds->send_message_client(stale, m->get_source_inst());
+
+ // add to cap export list.
+ mdcache->rejoin_export_caps(p->first, m->inode_path[p->first], from, p->second);
+ } else {
+ // mine. fetch later.
+ dout(0) << "missing " << p->first << " " << m->inode_path[p->first]
+ << " (mine), will load later" << endl;
+ mdcache->rejoin_recovered_caps(p->first, m->inode_path[p->first], from, p->second,
+ -1); // "from" me.
+ }
}
}
// remove from gather set
client_reconnect_gather.erase(from);
- if (client_reconnect_gather.empty()) reconnect_finish();
+ if (client_reconnect_gather.empty()) reconnect_gather_finish();
delete m;
}
-void Server::client_reconnect_failure(int from)
-{
- dout(5) << "client_reconnect_failure on client" << from << endl;
- client_reconnect_gather.erase(from);
- if (client_reconnect_gather.empty())
- reconnect_finish();
-}
-
-void Server::reconnect_finish()
+/*
+ * called by mdcache, late in rejoin (right before acks are sent)
+ */
+void Server::process_reconnected_caps()
{
- dout(7) << "reconnect_finish" << endl;
+ dout(10) << "process_reconnected_caps" << endl;
// adjust filelock state appropriately
- for (set<CInode*>::iterator p = reconnected_open_files.begin();
- p != reconnected_open_files.end();
+ for (set<CInode*>::iterator p = reconnected_caps.begin();
+ p != reconnected_caps.end();
++p) {
CInode *in = *p;
int issued = in->get_caps_issued();
else
in->filelock.set_state(LOCK_SYNC); // might have been lock, previously
}
- dout(10) << " issued " << cap_string(issued)
+ dout(15) << " issued " << cap_string(issued)
<< " chose " << in->filelock
<< " on " << *in << endl;
}
- reconnected_open_files.clear(); // clean up
+ reconnected_caps.clear(); // clean up
+}
- // done
- if (mds->mdsmap->get_num_in_mds() == 1)
- mds->set_want_state(MDSMap::STATE_ACTIVE); // go active
- else
- mds->set_want_state(MDSMap::STATE_REJOIN); // move to rejoin state
+
+void Server::client_reconnect_failure(int from)
+{
+ dout(5) << "client_reconnect_failure on client" << from << endl;
+ client_reconnect_gather.erase(from);
+ if (client_reconnect_gather.empty())
+ reconnect_gather_finish();
+}
+
+void Server::reconnect_gather_finish()
+{
+ dout(7) << "reconnect_gather_finish" << endl;
+ mds->reconnect_done();
}
*/
void Server::reply_request(MDRequest *mdr, int r, CInode *tracei)
{
- MClientRequest *req = mdr->client_request();
- reply_request(mdr, new MClientReply(req, r), tracei);
+ reply_request(mdr, new MClientReply(mdr->client_request, r), tracei);
}
*/
void Server::reply_request(MDRequest *mdr, MClientReply *reply, CInode *tracei)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
dout(10) << "reply_request " << reply->get_result()
<< " (" << strerror(-reply->get_result())
// okay, i want
- CInode *ref = 0;
+ CInode *ref = 0;
// retry?
if (req->get_retry_attempt()) {
}
}
// trim completed_request list
- if (req->get_oldest_client_tid() > 0)
+ if (req->get_oldest_client_tid() > 0) {
+ dout(15) << " oldest_client_tid=" << req->get_oldest_client_tid() << endl;
mds->clientmap.trim_completed_requests(client,
req->get_oldest_client_tid());
+ }
// -----
mdr->pin(ref);
}
- dispatch_request(mdr);
+ dispatch_client_request(mdr);
return;
}
-void Server::dispatch_request(MDRequest *mdr)
+void Server::dispatch_client_request(MDRequest *mdr)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
if (mdr->ref) {
- dout(7) << "dispatch_request " << *req << " ref " << *mdr->ref << endl;
+ dout(7) << "dispatch_client_request " << *req << " ref " << *mdr->ref << endl;
} else {
- dout(7) << "dispatch_request " << *req << endl;
+ dout(7) << "dispatch_client_request " << *req << endl;
}
+ // we shouldn't be waiting on anyone.
+ assert(mdr->waiting_on_slave.empty());
+
switch (req->get_op()) {
// inodes ops.
}
+// ---------------------------------------
+// SLAVE REQUESTS
+
+void Server::handle_slave_request(MMDSSlaveRequest *m)
+{
+ dout(4) << "handle_slave_request " << m->get_reqid() << " from " << m->get_source() << endl;
+ int from = m->get_source().num();
+
+ // reply?
+ if (m->is_reply()) {
+
+ switch (m->get_op()) {
+ case MMDSSlaveRequest::OP_XLOCKACK:
+ {
+ // identify lock, master request
+ SimpleLock *lock = mds->locker->get_lock(m->get_lock_type(),
+ m->get_object_info());
+ MDRequest *mdr = mdcache->request_get(m->get_reqid());
+ mdr->slaves.insert(from);
+ dout(10) << "got remote xlock on " << *lock << " on " << *lock->get_parent() << endl;
+ mdr->xlocks.insert(lock);
+ mdr->locks.insert(lock);
+ lock->get_xlock(mdr);
+ lock->finish_waiters(SimpleLock::WAIT_REMOTEXLOCK);
+ }
+ break;
+
+ case MMDSSlaveRequest::OP_AUTHPINACK:
+ {
+ MDRequest *mdr = mdcache->request_get(m->get_reqid());
+ handle_slave_auth_pin_ack(mdr, m);
+ }
+ break;
+
+ case MMDSSlaveRequest::OP_LINKPREPACK:
+ {
+ MDRequest *mdr = mdcache->request_get(m->get_reqid());
+ handle_slave_link_prep_ack(mdr, m);
+ }
+ break;
+
+ case MMDSSlaveRequest::OP_RENAMEPREPACK:
+ {
+ MDRequest *mdr = mdcache->request_get(m->get_reqid());
+ handle_slave_rename_prep_ack(mdr, m);
+ }
+ break;
+
+ case MMDSSlaveRequest::OP_RENAMEGETINODEACK:
+ {
+ MDRequest *mdr = mdcache->request_get(m->get_reqid());
+ handle_slave_rename_get_inode_ack(mdr, m);
+ }
+ break;
+
+ default:
+ assert(0);
+ }
+
+ // done with reply.
+ delete m;
+ return;
+
+ } else {
+ // am i a new slave?
+ MDRequest *mdr;
+ if (mdcache->have_request(m->get_reqid())) {
+ // existing?
+ mdr = mdcache->request_get(m->get_reqid());
+ if (mdr->slave_to_mds != from) { // may not even be a slave! (e.g. forward race)
+ dout(10) << "local request " << *mdr << " not slave to mds" << from
+ << ", ignoring " << *m << endl;
+ delete m;
+ return;
+ }
+ } else {
+ // new?
+ if (m->get_op() == MMDSSlaveRequest::OP_FINISH) {
+ dout(10) << "missing slave request for " << m->get_reqid()
+ << " OP_FINISH, must have lost race with a forward" << endl;
+ delete m;
+ return;
+ }
+ mdr = mdcache->request_start_slave(m->get_reqid(), m->get_source().num());
+ }
+ assert(mdr->slave_request == 0); // only one at a time, please!
+ mdr->slave_request = m;
+
+ dispatch_slave_request(mdr);
+ }
+}
+
+void Server::dispatch_slave_request(MDRequest *mdr)
+{
+ dout(7) << "dispatch_slave_request " << *mdr << " " << *mdr->slave_request << endl;
+
+ if (mdr->aborted) {
+ dout(7) << " abort flag set, finishing" << endl;
+ mdcache->request_finish(mdr);
+ return;
+ }
+
+ switch (mdr->slave_request->get_op()) {
+ case MMDSSlaveRequest::OP_XLOCK:
+ {
+ // identify object
+ SimpleLock *lock = mds->locker->get_lock(mdr->slave_request->get_lock_type(),
+ mdr->slave_request->get_object_info());
+
+ if (lock && lock->get_parent()->is_auth()) {
+ // xlock.
+ // use acquire_locks so that we get auth_pinning.
+ set<SimpleLock*> rdlocks;
+ set<SimpleLock*> wrlocks;
+ set<SimpleLock*> xlocks = mdr->xlocks;
+ xlocks.insert(lock);
+
+ if (!mds->locker->acquire_locks(mdr, rdlocks, wrlocks, xlocks))
+ return;
+
+ // ack
+ MMDSSlaveRequest *r = new MMDSSlaveRequest(mdr->reqid, MMDSSlaveRequest::OP_XLOCKACK);
+ r->set_lock_type(lock->get_type());
+ lock->get_parent()->set_object_info(r->get_object_info());
+ mds->send_message_mds(r, mdr->slave_request->get_source().num(), MDS_PORT_SERVER);
+ } else {
+ if (lock) {
+ dout(10) << "not auth for remote xlock attempt, dropping on "
+ << *lock << " on " << *lock->get_parent() << endl;
+ } else {
+ dout(10) << "don't have object, dropping" << endl;
+ assert(0); // can this happen, if we auth pinned properly.
+ }
+ }
+
+ // done.
+ delete mdr->slave_request;
+ mdr->slave_request = 0;
+ }
+ break;
+
+ case MMDSSlaveRequest::OP_UNXLOCK:
+ {
+ SimpleLock *lock = mds->locker->get_lock(mdr->slave_request->get_lock_type(),
+ mdr->slave_request->get_object_info());
+ assert(lock);
+ mds->locker->xlock_finish(lock, mdr);
+
+ // done. no ack necessary.
+ delete mdr->slave_request;
+ mdr->slave_request = 0;
+ }
+ break;
+
+ case MMDSSlaveRequest::OP_AUTHPIN:
+ handle_slave_auth_pin(mdr);
+ break;
+
+ case MMDSSlaveRequest::OP_LINKPREP:
+ case MMDSSlaveRequest::OP_UNLINKPREP:
+ handle_slave_link_prep(mdr);
+ break;
+
+ case MMDSSlaveRequest::OP_RENAMEPREP:
+ handle_slave_rename_prep(mdr);
+ break;
+
+ case MMDSSlaveRequest::OP_RENAMEGETINODE:
+ handle_slave_rename_get_inode(mdr);
+ break;
+
+ case MMDSSlaveRequest::OP_FINISH:
+ // finish off request.
+ mdcache->request_finish(mdr);
+ break;
+
+ default:
+ assert(0);
+ }
+}
+
+
+void Server::handle_slave_auth_pin(MDRequest *mdr)
+{
+ dout(10) << "handle_slave_auth_pin " << *mdr << endl;
+
+ // build list of objects
+ list<MDSCacheObject*> objects;
+ bool fail = false;
+
+ for (list<MDSCacheObjectInfo>::iterator p = mdr->slave_request->get_authpins().begin();
+ p != mdr->slave_request->get_authpins().end();
+ ++p) {
+ MDSCacheObject *object = mdcache->get_object(*p);
+ if (!object) {
+ dout(10) << " don't have " << *p << endl;
+ fail = true;
+ break;
+ }
+
+ objects.push_back(object);
+ }
+
+ // can we auth pin them?
+ if (!fail) {
+ for (list<MDSCacheObject*>::iterator p = objects.begin();
+ p != objects.end();
+ ++p) {
+ if (!(*p)->is_auth()) {
+ dout(10) << " not auth for " << **p << endl;
+ fail = true;
+ break;
+ }
+ if (!mdr->is_auth_pinned(*p) &&
+ !(*p)->can_auth_pin()) {
+ // wait
+ dout(10) << " waiting for authpinnable on " << **p << endl;
+ (*p)->add_waiter(CDir::WAIT_AUTHPINNABLE, new C_MDS_RetryRequest(mdcache, mdr));
+ mdr->drop_local_auth_pins();
+ return;
+ }
+ }
+ }
+
+ // auth pin!
+ if (fail) {
+ mdr->drop_local_auth_pins(); // just in case
+ } else {
+ for (list<MDSCacheObject*>::iterator p = objects.begin();
+ p != objects.end();
+ ++p) {
+ dout(10) << "auth_pinning " << **p << endl;
+ mdr->auth_pin(*p);
+ }
+ }
+
+ // ack!
+ MMDSSlaveRequest *reply = new MMDSSlaveRequest(mdr->reqid, MMDSSlaveRequest::OP_AUTHPINACK);
+
+ // return list of my auth_pins (if any)
+ for (set<MDSCacheObject*>::iterator p = mdr->auth_pins.begin();
+ p != mdr->auth_pins.end();
+ ++p) {
+ MDSCacheObjectInfo info;
+ (*p)->set_object_info(info);
+ reply->get_authpins().push_back(info);
+ }
+
+ mds->send_message_mds(reply, mdr->slave_to_mds, MDS_PORT_SERVER);
+
+ // clean up this request
+ delete mdr->slave_request;
+ mdr->slave_request = 0;
+ return;
+}
+
+void Server::handle_slave_auth_pin_ack(MDRequest *mdr, MMDSSlaveRequest *ack)
+{
+ dout(10) << "handle_slave_auth_pin_ack on " << *mdr << " " << *ack << endl;
+ int from = ack->get_source().num();
+
+ // added auth pins?
+ set<MDSCacheObject*> pinned;
+ for (list<MDSCacheObjectInfo>::iterator p = ack->get_authpins().begin();
+ p != ack->get_authpins().end();
+ ++p) {
+ MDSCacheObject *object = mdcache->get_object(*p);
+ assert(object); // we pinned it
+ dout(10) << " remote has pinned " << *object << endl;
+ if (!mdr->is_auth_pinned(object))
+ mdr->auth_pins.insert(object);
+ pinned.insert(object);
+ }
+
+ // removed auth pins?
+ set<MDSCacheObject*>::iterator p = mdr->auth_pins.begin();
+ while (p != mdr->auth_pins.end()) {
+ if ((*p)->authority().first == from &&
+ pinned.count(*p) == 0) {
+ dout(10) << " remote has unpinned " << **p << endl;
+ set<MDSCacheObject*>::iterator o = p;
+ ++p;
+ mdr->auth_pins.erase(o);
+ } else {
+ ++p;
+ }
+ }
+
+ // note slave
+ mdr->slaves.insert(from);
+
+ // clear from waiting list
+ assert(mdr->waiting_on_slave.count(from));
+ mdr->waiting_on_slave.erase(from);
+
+ // go again?
+ if (mdr->waiting_on_slave.empty())
+ dispatch_client_request(mdr);
+ else
+ dout(10) << "still waiting on slaves " << mdr->waiting_on_slave << endl;
+}
+
// ---------------------------------------
// HELPERS
// does it already exist?
CDentry *dn = dir->lookup(dname);
if (dn) {
- if (!dn->lock.can_rdlock(mdr)) {
- dout(10) << "waiting on (existing!) unreadable dentry " << *dn << endl;
+ if (dn->lock.is_xlocked_by_other(mdr)) {
+ dout(10) << "waiting on xlocked dentry " << *dn << endl;
dn->lock.add_waiter(SimpleLock::WAIT_RD, new C_MDS_RetryRequest(mdcache, mdr));
return 0;
}
// create
dn = dir->add_dentry(dname, 0);
+ dn->mark_new();
dout(10) << "prepare_null_dentry added " << *dn << endl;
return dn;
*
* create a new inode. set c/m/atime. hit dir pop.
*/
-CInode* Server::prepare_new_inode(MClientRequest *req, CDir *dir)
+CInode* Server::prepare_new_inode(MDRequest *mdr, CDir *dir)
{
CInode *in = mdcache->create_inode();
- in->inode.uid = req->get_caller_uid();
- in->inode.gid = req->get_caller_gid();
- in->inode.ctime = in->inode.mtime = in->inode.atime = g_clock.real_now(); // now
+ in->inode.uid = mdr->client_request->get_caller_uid();
+ in->inode.gid = mdr->client_request->get_caller_gid();
+ in->inode.ctime = in->inode.mtime = in->inode.atime = mdr->now; // now
dout(10) << "prepare_new_inode " << *in << endl;
// bump modify pop
dout(10) << "traverse_to_auth_dir dirpath " << refpath << " dname " << dname << endl;
// traverse to parent dir
- Context *ondelay = new C_MDS_RetryRequest(mdcache, mdr);
- int r = mdcache->path_traverse(mdr,
- 0,
- refpath, trace, true,
- mdr->request, ondelay,
- MDS_TRAVERSE_FORWARD,
- true); // is MClientRequest
+ int r = mdcache->path_traverse(mdr, mdr->client_request,
+ 0, refpath, trace, true,
+ MDS_TRAVERSE_FORWARD);
if (r > 0) return 0; // delayed
if (r < 0) {
reply_request(mdr, r);
if (mdr->ref)
return mdr->ref;
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
// traverse
filepath refpath = req->get_filepath();
- Context *ondelay = new C_MDS_RetryRequest(mdcache, mdr);
vector<CDentry*> trace;
- int r = mdcache->path_traverse(mdr, 0,
- refpath, trace, req->follow_trailing_symlink(),
- req, ondelay,
- MDS_TRAVERSE_FORWARD,
- true); // is MClientRequest
+ int r = mdcache->path_traverse(mdr, req,
+ 0, refpath,
+ trace, req->follow_trailing_symlink(),
+ MDS_TRAVERSE_FORWARD);
if (r > 0) return false; // delayed
if (r < 0) { // error
reply_request(mdr, r);
else {
CDentry *dn = trace[trace.size()-1];
- // if no inode, fw to dentry auth?
- if (want_auth &&
- dn->is_remote() &&
- !dn->inode &&
- !dn->is_auth()) {
+ // if no inode (null or unattached remote), fw to dentry auth?
+ if (want_auth && !dn->is_auth() &&
+ (dn->is_null() ||
+ (dn->is_remote() && dn->inode))) {
if (dn->is_ambiguous_auth()) {
dout(10) << "waiting for single auth on " << *dn << endl;
- dn->dir->add_waiter(CInode::WAIT_SINGLEAUTH, new C_MDS_RetryMessage(mds, req));
+ dn->dir->add_waiter(CInode::WAIT_SINGLEAUTH, new C_MDS_RetryRequest(mdcache, mdr));
} else {
dout(10) << "fw to auth for " << *dn << endl;
- mds->forward_message_mds(req, dn->authority().first, MDS_PORT_SERVER);
+ mdcache->request_forward(mdr, dn->authority().first);
+ return 0;
}
}
if (want_auth && !ref->is_auth()) {
if (ref->is_ambiguous_auth()) {
dout(10) << "waiting for single auth on " << *ref << endl;
- ref->add_waiter(CInode::WAIT_SINGLEAUTH, new C_MDS_RetryMessage(mds, req));
+ ref->add_waiter(CInode::WAIT_SINGLEAUTH, new C_MDS_RetryRequest(mdcache, mdr));
} else {
dout(10) << "fw to auth for " << *ref << endl;
- mds->forward_message_mds(req, ref->authority().first, MDS_PORT_SERVER);
+ mdcache->request_forward(mdr, ref->authority().first);
}
+ return 0;
}
// auth_pin?
if (want_auth) {
- if (!ref->can_auth_pin()) {
- dout(7) << "waiting for authpinnable on " << *ref << endl;
+ if (ref->is_frozen()) {
+ dout(7) << "waiting for !frozen/authpinnable on " << *ref << endl;
ref->add_waiter(CInode::WAIT_AUTHPINNABLE, new C_MDS_RetryRequest(mdcache, mdr));
return 0;
}
*/
CDentry* Server::rdlock_path_xlock_dentry(MDRequest *mdr, bool okexist, bool mustexist)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
vector<CDentry*> trace;
CDir *dir = traverse_to_auth_dir(mdr, trace, req->get_filepath());
if (!dir) return 0;
dout(10) << "rdlock_path_xlock_dentry dir " << *dir << endl;
- // make sure we can auth_pin dir
- if (!dir->can_auth_pin()) {
- dout(7) << "waiting for authpinnable on " << *dir << endl;
+ // make sure we can auth_pin (or have already authpinned) dir
+ if (dir->is_frozen()) {
+ dout(7) << "waiting for !frozen/authpinnable on " << *dir << endl;
dir->add_waiter(CInode::WAIT_AUTHPINNABLE, new C_MDS_RetryRequest(mdcache, mdr));
return 0;
}
}
// readable?
- if (dn && !dn->lock.can_rdlock(mdr)) {
- dout(10) << "waiting on (existing!) unreadable dentry " << *dn << endl;
+ if (dn && dn->lock.is_xlocked_by_other(mdr)) {
+ dout(10) << "waiting on xlocked dentry " << *dn << endl;
dn->lock.add_waiter(SimpleLock::WAIT_RD, new C_MDS_RetryRequest(mdcache, mdr));
return 0;
}
* predirty the directory inode for a new dentry, if it is auth (and not root)
* BUG: root inode doesn't get dirtied properly, currently. blech.
*/
-version_t Server::predirty_dn_diri(CDentry *dn, EMetaBlob *blob, utime_t mtime)
+version_t Server::predirty_dn_diri(MDRequest *mdr, CDentry *dn, EMetaBlob *blob)
{
version_t dirpv = 0;
CInode *diri = dn->dir->inode;
+
+ if (diri->is_root()) return 0;
- if (diri->is_auth() && !diri->is_root()) {
+ if (diri->is_auth()) {
+ assert(mdr->wrlocks.count(&diri->dirlock));
+
dirpv = diri->pre_dirty();
- inode_t *pi = blob->add_primary_dentry(diri->get_parent_dn(), true);
- pi->version = dirpv;
- pi->ctime = pi->mtime = mtime;
- dout(10) << "predirty_dn_diri ctime/mtime " << mtime << " pv " << dirpv << " on " << *diri << endl;
- }
+ dout(10) << "predirty_dn_diri ctime/mtime " << mdr->now << " pv " << dirpv << " on " << *diri << endl;
+
+ // predirty+journal
+ inode_t *pi = diri->project_inode();
+ if (dirpv) pi->version = dirpv;
+ pi->ctime = pi->mtime = mdr->now;
+ blob->add_dir_context(diri->get_parent_dir());
+ blob->add_primary_dentry(diri->get_parent_dn(), true, 0, pi);
+ } else {
+ // journal the mtime change anyway.
+ inode_t *ji = blob->add_primary_dentry(diri->get_parent_dn(), true);
+ ji->ctime = ji->mtime = mdr->now;
+ dout(10) << "predirty_dn_diri (non-auth) ctime/mtime " << mdr->now << " on " << *diri << endl;
+
+ blob->add_dirtied_inode_mtime(diri->ino(), mdr->now);
+ }
+
return dirpv;
}
{
CInode *diri = dn->dir->inode;
- // make the udpate
- diri->inode.ctime = diri->inode.mtime = mtime;
+ if (diri->is_root()) return;
- if (diri->is_auth() && !diri->is_root()) {
- // we're auth.
- diri->mark_dirty(dirpv);
+ if (dirpv) {
+ // we journaled and predirtied.
+ assert(diri->is_auth() && !diri->is_root());
+ diri->pop_and_dirty_projected_inode();
dout(10) << "dirty_dn_diri ctime/mtime " << mtime << " v " << diri->inode.version << " on " << *diri << endl;
} else {
- // we're not auth. dirlock scatterlock will propagate the update.
+ // dirlock scatterlock will propagate the update.
+ diri->inode.ctime = diri->inode.mtime = mtime;
+ diri->dirlock.set_updated();
+ dout(10) << "dirty_dn_diri (non-dirty) ctime/mtime " << mtime << " on " << *diri << endl;
}
}
-
-
-
// ===============================================================================
// STAT
void Server::handle_client_stat(MDRequest *mdr)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
CInode *ref = rdlock_path_pin_ref(mdr, false);
if (!ref) return;
/*
- * finisher: do a inode_file_write_finish and reply.
+ * finisher for basic inode updates
*/
-class C_MDS_utime_finish : public Context {
+class C_MDS_inode_update_finish : public Context {
MDS *mds;
MDRequest *mdr;
CInode *in;
- version_t pv;
- utime_t mtime, atime;
public:
- C_MDS_utime_finish(MDS *m, MDRequest *r, CInode *i, version_t pdv, utime_t mt, utime_t at) :
- mds(m), mdr(r), in(i),
- pv(pdv),
- mtime(mt), atime(at) { }
+ C_MDS_inode_update_finish(MDS *m, MDRequest *r, CInode *i) :
+ mds(m), mdr(r), in(i) { }
void finish(int r) {
assert(r == 0);
// apply
- in->inode.mtime = mtime;
- in->inode.atime = atime;
- in->mark_dirty(pv);
+ in->pop_and_dirty_projected_inode();
// reply
- MClientReply *reply = new MClientReply(mdr->client_request(), 0);
+ MClientReply *reply = new MClientReply(mdr->client_request, 0);
reply->set_result(0);
mds->server->reply_request(mdr, reply, in);
}
void Server::handle_client_utime(MDRequest *mdr)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
CInode *cur = rdlock_path_pin_ref(mdr, true);
if (!cur) return;
- // write
- if (!mds->locker->xlock_start(&cur->filelock, mdr))
+ // xlock inode
+ set<SimpleLock*> rdlocks = mdr->rdlocks;
+ set<SimpleLock*> wrlocks = mdr->wrlocks;
+ set<SimpleLock*> xlocks = mdr->xlocks;
+ xlocks.insert(&cur->filelock);
+ if (!mds->locker->acquire_locks(mdr, rdlocks, wrlocks, xlocks))
return;
mds->balancer->hit_inode(cur, META_POP_IWR);
- // prepare
- version_t pdv = cur->pre_dirty();
- utime_t mtime = req->args.utime.mtime;
- utime_t atime = req->args.utime.atime;
- C_MDS_utime_finish *fin = new C_MDS_utime_finish(mds, mdr, cur, pdv,
- mtime, atime);
+ // project update
+ inode_t *pi = cur->project_inode();
+ pi->mtime = req->args.utime.mtime;
+ pi->atime = req->args.utime.atime;
+ pi->version = cur->pre_dirty();
+ pi->ctime = g_clock.real_now();
// log + wait
EUpdate *le = new EUpdate("utime");
le->metablob.add_client_req(req->get_reqid());
le->metablob.add_dir_context(cur->get_parent_dir());
- inode_t *pi = le->metablob.add_dentry(cur->parent, true);
- pi->mtime = mtime;
- pi->atime = mtime;
- pi->ctime = g_clock.real_now();
- pi->version = pdv;
+ le->metablob.add_primary_dentry(cur->parent, true, 0, pi);
mdlog->submit_entry(le);
- mdlog->wait_for_sync(fin);
+ mdlog->wait_for_sync(new C_MDS_inode_update_finish(mds, mdr, cur));
}
-// --------------
-
-/*
- * finisher: do a inode_hard_xlock_finish and reply.
- */
-class C_MDS_chmod_finish : public Context {
- MDS *mds;
- MDRequest *mdr;
- CInode *in;
- version_t pv;
- int mode;
-public:
- C_MDS_chmod_finish(MDS *m, MDRequest *r, CInode *i, version_t pdv, int mo) :
- mds(m), mdr(r), in(i), pv(pdv), mode(mo) { }
- void finish(int r) {
- assert(r == 0);
-
- // apply
- in->inode.mode &= ~04777;
- in->inode.mode |= (mode & 04777);
- in->mark_dirty(pv);
-
- // reply
- MClientReply *reply = new MClientReply(mdr->client_request(), 0);
- reply->set_result(0);
- mds->server->reply_request(mdr, reply, in);
- }
-};
-
-
// chmod
void Server::handle_client_chmod(MDRequest *mdr)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
CInode *cur = rdlock_path_pin_ref(mdr, true);
if (!cur) return;
// write
- if (!mds->locker->xlock_start(&cur->authlock, mdr))
+ set<SimpleLock*> rdlocks = mdr->rdlocks;
+ set<SimpleLock*> wrlocks = mdr->wrlocks;
+ set<SimpleLock*> xlocks = mdr->xlocks;
+ xlocks.insert(&cur->authlock);
+ if (!mds->locker->acquire_locks(mdr, rdlocks, wrlocks, xlocks))
return;
mds->balancer->hit_inode(cur, META_POP_IWR);
- // prepare
- version_t pdv = cur->pre_dirty();
- int mode = req->args.chmod.mode;
- C_MDS_chmod_finish *fin = new C_MDS_chmod_finish(mds, mdr, cur, pdv,
- mode);
+ // project update
+ inode_t *pi = cur->project_inode();
+ pi->mode = req->args.chmod.mode & 04777;
+ pi->version = cur->pre_dirty();
+ pi->ctime = g_clock.real_now();
// log + wait
EUpdate *le = new EUpdate("chmod");
le->metablob.add_client_req(req->get_reqid());
le->metablob.add_dir_context(cur->get_parent_dir());
- inode_t *pi = le->metablob.add_dentry(cur->parent, true);
- pi->mode = mode;
- pi->version = pdv;
- pi->ctime = g_clock.real_now();
-
+ le->metablob.add_primary_dentry(cur->parent, true, 0, pi);
+
mdlog->submit_entry(le);
- mdlog->wait_for_sync(fin);
+ mdlog->wait_for_sync(new C_MDS_inode_update_finish(mds, mdr, cur));
}
// chown
-class C_MDS_chown_finish : public Context {
- MDS *mds;
- MDRequest *mdr;
- CInode *in;
- version_t pv;
- int uid, gid;
-public:
- C_MDS_chown_finish(MDS *m, MDRequest *r, CInode *i, version_t pdv, int u, int g) :
- mds(m), mdr(r), in(i), pv(pdv), uid(u), gid(g) { }
- void finish(int r) {
- assert(r == 0);
-
- // apply
- if (uid >= 0) in->inode.uid = uid;
- if (gid >= 0) in->inode.gid = gid;
- in->mark_dirty(pv);
-
- // reply
- MClientReply *reply = new MClientReply(mdr->client_request(), 0);
- reply->set_result(0);
- mds->server->reply_request(mdr, reply, in);
- }
-};
-
-
void Server::handle_client_chown(MDRequest *mdr)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
CInode *cur = rdlock_path_pin_ref(mdr, true);
if (!cur) return;
// write
- if (!mds->locker->xlock_start(&cur->authlock, mdr))
+ set<SimpleLock*> rdlocks = mdr->rdlocks;
+ set<SimpleLock*> wrlocks = mdr->wrlocks;
+ set<SimpleLock*> xlocks = mdr->xlocks;
+ xlocks.insert(&cur->authlock);
+ if (!mds->locker->acquire_locks(mdr, rdlocks, wrlocks, xlocks))
return;
mds->balancer->hit_inode(cur, META_POP_IWR);
- // prepare
- version_t pdv = cur->pre_dirty();
- int uid = req->args.chown.uid;
- int gid = req->args.chown.gid;
- C_MDS_chown_finish *fin = new C_MDS_chown_finish(mds, mdr, cur, pdv,
- uid, gid);
-
+ // project update
+ inode_t *pi = cur->project_inode();
+ pi->uid = MAX(req->args.chown.uid, 0);
+ pi->gid = MAX(req->args.chown.gid, 0);
+ pi->version = cur->pre_dirty();
+ pi->ctime = g_clock.real_now();
+
// log + wait
EUpdate *le = new EUpdate("chown");
le->metablob.add_client_req(req->get_reqid());
le->metablob.add_dir_context(cur->get_parent_dir());
- inode_t *pi = le->metablob.add_dentry(cur->parent, true);
- if (uid >= 0) pi->uid = uid;
- if (gid >= 0) pi->gid = gid;
- pi->version = pdv;
- pi->ctime = g_clock.real_now();
+ le->metablob.add_primary_dentry(cur->parent, true, 0, pi);
mdlog->submit_entry(le);
- mdlog->wait_for_sync(fin);
+ mdlog->wait_for_sync(new C_MDS_inode_update_finish(mds, mdr, cur));
}
void Server::handle_client_readdir(MDRequest *mdr)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
CInode *diri = rdlock_path_pin_ref(mdr, false);
if (!diri) return;
MDRequest *mdr;
CDentry *dn;
CInode *newi;
- version_t pv;
version_t dirpv;
public:
C_MDS_mknod_finish(MDS *m, MDRequest *r, CDentry *d, CInode *ni, version_t dirpv_) :
mds(m), mdr(r), dn(d), newi(ni),
- pv(d->get_projected_version()), dirpv(dirpv_) {}
+ dirpv(dirpv_) {}
void finish(int r) {
assert(r == 0);
dn->get_dir()->link_inode(dn, newi);
// dirty inode, dn, dir
- newi->mark_dirty(pv);
+ newi->mark_dirty(newi->inode.version + 1);
// dir inode's mtime
mds->server->dirty_dn_diri(dn, dirpv, newi->inode.ctime);
mds->balancer->hit_inode(newi, META_POP_IWR);
// reply
- MClientReply *reply = new MClientReply(mdr->client_request(), 0);
+ MClientReply *reply = new MClientReply(mdr->client_request, 0);
reply->set_result(0);
mds->server->reply_request(mdr, reply, newi);
}
};
-
void Server::handle_client_mknod(MDRequest *mdr)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
CDentry *dn = rdlock_path_xlock_dentry(mdr, false, false);
if (!dn) return;
- CInode *newi = prepare_new_inode(req, dn->dir);
+ mdr->now = g_clock.real_now();
+ CInode *newi = prepare_new_inode(mdr, dn->dir);
assert(newi);
// it's a file.
- dn->pre_dirty();
newi->inode.mode = req->args.mknod.mode;
newi->inode.mode &= ~INODE_TYPE_MASK;
newi->inode.mode |= INODE_MODE_FILE;
+ newi->inode.version = dn->pre_dirty() - 1;
// prepare finisher
EUpdate *le = new EUpdate("mknod");
le->metablob.add_client_req(req->get_reqid());
-
- version_t dirpv = predirty_dn_diri(dn, &le->metablob, newi->inode.ctime); // dir mtime too
-
+ le->metablob.add_allocated_ino(newi->ino(), mds->idalloc->get_version());
+ version_t dirpv = predirty_dn_diri(mdr, dn, &le->metablob); // dir mtime too
le->metablob.add_dir_context(dn->dir);
- inode_t *pi = le->metablob.add_primary_dentry(dn, true, newi);
- pi->version = dn->get_projected_version();
+ le->metablob.add_primary_dentry(dn, true, newi, &newi->inode);
// log + wait
- C_MDS_mknod_finish *fin = new C_MDS_mknod_finish(mds, mdr, dn, newi, dirpv);
mdlog->submit_entry(le);
- mdlog->wait_for_sync(fin);
+ mdlog->wait_for_sync(new C_MDS_mknod_finish(mds, mdr, dn, newi, dirpv));
}
void Server::handle_client_mkdir(MDRequest *mdr)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
CDentry *dn = rdlock_path_xlock_dentry(mdr, false, false);
if (!dn) return;
// new inode
- CInode *newi = prepare_new_inode(req, dn->dir);
+ mdr->now = g_clock.real_now();
+ CInode *newi = prepare_new_inode(mdr, dn->dir);
assert(newi);
// it's a directory.
- dn->pre_dirty();
newi->inode.mode = req->args.mkdir.mode;
newi->inode.mode &= ~INODE_TYPE_MASK;
newi->inode.mode |= INODE_MODE_DIR;
newi->inode.layout = g_OSD_MDDirLayout;
+ newi->inode.version = dn->pre_dirty() - 1;
// ...and that new dir is empty.
CDir *newdir = newi->get_or_open_dirfrag(mds->mdcache, frag_t());
// prepare finisher
EUpdate *le = new EUpdate("mkdir");
le->metablob.add_client_req(req->get_reqid());
- version_t dirpv = predirty_dn_diri(dn, &le->metablob, newi->inode.ctime); // dir mtime too
+ le->metablob.add_allocated_ino(newi->ino(), mds->idalloc->get_version());
+ version_t dirpv = predirty_dn_diri(mdr, dn, &le->metablob); // dir mtime too
le->metablob.add_dir_context(dn->dir);
- inode_t *pi = le->metablob.add_primary_dentry(dn, true, newi);
- pi->version = dn->get_projected_version();
- le->metablob.add_dir(newdir, true);
+ le->metablob.add_primary_dentry(dn, true, newi, &newi->inode);
+ le->metablob.add_dir(newdir, true, true); // dirty AND complete
// log + wait
- C_MDS_mknod_finish *fin = new C_MDS_mknod_finish(mds, mdr, dn, newi, dirpv);
mdlog->submit_entry(le);
- mdlog->wait_for_sync(fin);
+ mdlog->wait_for_sync(new C_MDS_mknod_finish(mds, mdr, dn, newi, dirpv));
/* old export heuristic. pbly need to reimplement this at some point.
void Server::handle_client_symlink(MDRequest *mdr)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
CDentry *dn = rdlock_path_xlock_dentry(mdr, false, false);
if (!dn) return;
- CInode *newi = prepare_new_inode(req, dn->dir);
+ mdr->now = g_clock.real_now();
+ CInode *newi = prepare_new_inode(mdr, dn->dir);
assert(newi);
// it's a symlink
- dn->pre_dirty();
newi->inode.mode &= ~INODE_TYPE_MASK;
newi->inode.mode |= INODE_MODE_SYMLINK;
newi->symlink = req->get_sarg();
+ newi->inode.version = dn->pre_dirty() - 1;
// prepare finisher
EUpdate *le = new EUpdate("symlink");
le->metablob.add_client_req(req->get_reqid());
- version_t dirpv = predirty_dn_diri(dn, &le->metablob, newi->inode.ctime); // dir mtime too
+ le->metablob.add_allocated_ino(newi->ino(), mds->idalloc->get_version());
+ version_t dirpv = predirty_dn_diri(mdr, dn, &le->metablob); // dir mtime too
le->metablob.add_dir_context(dn->dir);
- inode_t *pi = le->metablob.add_primary_dentry(dn, true, newi);
- pi->version = dn->get_projected_version();
+ le->metablob.add_primary_dentry(dn, true, newi, &newi->inode);
// log + wait
- C_MDS_mknod_finish *fin = new C_MDS_mknod_finish(mds, mdr, dn, newi, dirpv);
mdlog->submit_entry(le);
- mdlog->wait_for_sync(fin);
+ mdlog->wait_for_sync(new C_MDS_mknod_finish(mds, mdr, dn, newi, dirpv));
}
void Server::handle_client_link(MDRequest *mdr)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
dout(7) << "handle_client_link " << req->get_filepath()
<< " to " << req->get_sarg()
// traverse to link target
filepath targetpath = req->get_sarg();
dout(7) << "handle_client_link discovering target " << targetpath << endl;
- Context *ondelay = new C_MDS_RetryRequest(mdcache, mdr);
vector<CDentry*> targettrace;
- int r = mdcache->path_traverse(mdr, 0,
- targetpath, targettrace, false,
- req, ondelay,
+ int r = mdcache->path_traverse(mdr, req,
+ 0, targetpath, targettrace, false,
MDS_TRAVERSE_DISCOVER);
if (r > 0) return; // wait
- if (targettrace.empty()) r = -EINVAL;
+ if (targettrace.empty()) r = -EINVAL;
if (r < 0) {
reply_request(mdr, r);
return;
return;
}
- // does the target need an anchor?
- if (targeti->is_auth()) {
- /*if (targeti->get_parent_dir() == dn->dir) {
- dout(7) << "target is in the same dirfrag, sweet" << endl;
- }
- else
- */
- if (targeti->is_anchored() && !targeti->is_unanchoring()) {
- dout(7) << "target anchored already (nlink=" << targeti->inode.nlink << "), sweet" << endl;
- }
- else {
- dout(7) << "target needs anchor, nlink=" << targeti->inode.nlink << ", creating anchor" << endl;
-
- mdcache->anchor_create(targeti,
- new C_MDS_RetryRequest(mdcache, mdr));
- return;
- }
- }
-
- // can we create the dentry?
- CDentry *dn = 0;
-
- // make null link dentry
- dn = prepare_null_dentry(mdr, dir, dname, false);
+ // get/make null link dentry
+ CDentry *dn = prepare_null_dentry(mdr, dir, dname, false);
if (!dn) return;
// create lock lists
if (!mds->locker->acquire_locks(mdr, rdlocks, wrlocks, xlocks))
return;
+ mdr->done_locking = true; // avoid wrlock moving target issues.
+
+ // pick mtime
+ if (mdr->now == utime_t())
+ mdr->now = g_clock.real_now();
+
+ // does the target need an anchor?
+ if (targeti->is_auth()) {
+ /*if (targeti->get_parent_dir() == dn->dir) {
+ dout(7) << "target is in the same dirfrag, sweet" << endl;
+ }
+ else
+ */
+ if (targeti->is_anchored() && !targeti->is_unanchoring()) {
+ dout(7) << "target anchored already (nlink=" << targeti->inode.nlink << "), sweet" << endl;
+ }
+ else {
+ dout(7) << "target needs anchor, nlink=" << targeti->inode.nlink << ", creating anchor" << endl;
+
+ mdcache->anchor_create(mdr, targeti,
+ new C_MDS_RetryRequest(mdcache, mdr));
+ return;
+ }
+ }
+
// go!
// local or remote?
MDRequest *mdr;
CDentry *dn;
CInode *targeti;
- version_t dpv;
- utime_t tctime;
- version_t tpv;
+ version_t dnpv;
+ version_t tipv;
version_t dirpv;
public:
- C_MDS_link_local_finish(MDS *m, MDRequest *r, CDentry *d, CInode *ti, version_t dirpv_, utime_t ct) :
+ C_MDS_link_local_finish(MDS *m, MDRequest *r, CDentry *d, CInode *ti,
+ version_t dnpv_, version_t tipv_, version_t dirpv_) :
mds(m), mdr(r), dn(d), targeti(ti),
- dpv(d->get_projected_version()),
- tctime(ct),
- tpv(targeti->get_parent_dn()->get_projected_version()),
- dirpv(dirpv_) { }
+ dnpv(dnpv_), tipv(tipv_), dirpv(dirpv_) { }
void finish(int r) {
assert(r == 0);
- mds->server->_link_local_finish(mdr, dn, targeti, dpv, tctime, tpv, dirpv);
+ mds->server->_link_local_finish(mdr, dn, targeti, dnpv, tipv, dirpv);
}
};
{
dout(10) << "_link_local " << *dn << " to " << *targeti << endl;
- // ok, let's do it.
- // prepare log entry
+ // predirty NEW dentry
+ version_t dnpv = dn->pre_dirty();
+ version_t tipv = targeti->pre_dirty();
+
+ // project inode update
+ inode_t *pi = targeti->project_inode();
+ pi->nlink++;
+ pi->ctime = mdr->now;
+ pi->version = tipv;
+
+ // log + wait
EUpdate *le = new EUpdate("link_local");
le->metablob.add_client_req(mdr->reqid);
-
- // predirty
- dn->pre_dirty();
- version_t tpdv = targeti->pre_dirty();
-
- // add to event
- utime_t now = g_clock.real_now();
- version_t dirpv = predirty_dn_diri(dn, &le->metablob, now); // dir inode's mtime
+ version_t dirpv = predirty_dn_diri(mdr, dn, &le->metablob); // dir inode's mtime
le->metablob.add_dir_context(dn->get_dir());
le->metablob.add_remote_dentry(dn, true, targeti->ino()); // new remote
le->metablob.add_dir_context(targeti->get_parent_dir());
- inode_t *pi = le->metablob.add_primary_dentry(targeti->parent, true, targeti); // update old primary
+ le->metablob.add_primary_dentry(targeti->parent, true, targeti, pi); // update old primary
- // update journaled target inode
- pi->nlink++;
- pi->ctime = now;
- pi->version = tpdv;
-
- // finisher
- C_MDS_link_local_finish *fin = new C_MDS_link_local_finish(mds, mdr, dn, targeti, dirpv, now);
-
- // log + wait
mdlog->submit_entry(le);
- mdlog->wait_for_sync(fin);
+ mdlog->wait_for_sync(new C_MDS_link_local_finish(mds, mdr, dn, targeti, dnpv, tipv, dirpv));
}
void Server::_link_local_finish(MDRequest *mdr, CDentry *dn, CInode *targeti,
- version_t dpv, utime_t tctime, version_t tpv, version_t dirpv)
+ version_t dnpv, version_t tipv, version_t dirpv)
{
dout(10) << "_link_local_finish " << *dn << " to " << *targeti << endl;
- // link and unlock the new dentry
+ // link and unlock the NEW dentry
dn->dir->link_inode(dn, targeti->ino());
- dn->set_version(dpv);
- dn->mark_dirty(dpv);
+ dn->mark_dirty(dnpv);
- // update the target
- targeti->inode.nlink++;
- targeti->inode.ctime = tctime;
- targeti->mark_dirty(tpv);
+ // target inode
+ targeti->pop_and_dirty_projected_inode();
- // dir inode's mtime
- dirty_dn_diri(dn, dirpv, tctime);
+ // new dentry dir mtime
+ dirty_dn_diri(dn, dirpv, mdr->now);
// bump target popularity
mds->balancer->hit_inode(targeti, META_POP_IWR);
// reply
- MClientReply *reply = new MClientReply(mdr->client_request(), 0);
+ MClientReply *reply = new MClientReply(mdr->client_request, 0);
reply_request(mdr, reply, dn->get_dir()->get_inode()); // FIXME: imprecise ref
}
+// remote
+
+class C_MDS_link_remote_finish : public Context {
+ MDS *mds;
+ MDRequest *mdr;
+ CDentry *dn;
+ CInode *targeti;
+ version_t dpv;
+ version_t dirpv;
+public:
+ C_MDS_link_remote_finish(MDS *m, MDRequest *r, CDentry *d, CInode *ti, version_t dirpv_) :
+ mds(m), mdr(r), dn(d), targeti(ti),
+ dpv(d->get_projected_version()),
+ dirpv(dirpv_) { }
+ void finish(int r) {
+ assert(r == 0);
+ mds->server->_link_remote_finish(mdr, dn, targeti, dpv, dirpv);
+ }
+};
void Server::_link_remote(MDRequest *mdr, CDentry *dn, CInode *targeti)
{
dout(10) << "_link_remote " << *dn << " to " << *targeti << endl;
-
+
// 1. send LinkPrepare to dest (journal nlink++ prepare)
- // 2. create+journal new dentry, as with link_local.
- // 3. send LinkCommit to dest (journals commit)
+ int linkauth = targeti->authority().first;
+ if (mdr->witnessed.count(linkauth) == 0) {
+ dout(10) << " targeti auth must prepare nlink++" << endl;
- // IMPLEMENT ME
- reply_request(mdr, -EXDEV);
-}
+ MMDSSlaveRequest *req = new MMDSSlaveRequest(mdr->reqid, MMDSSlaveRequest::OP_LINKPREP);
+ targeti->set_object_info(req->get_object_info());
+ req->now = mdr->now;
+ mds->send_message_mds(req, linkauth, MDS_PORT_SERVER);
+ assert(mdr->waiting_on_slave.count(linkauth) == 0);
+ mdr->waiting_on_slave.insert(linkauth);
+ return;
+ }
+ dout(10) << " targeti auth has prepared nlink++" << endl;
-/*
-void Server::handle_client_link_finish(MClientRequest *req, CInode *ref,
- CDentry *dn, CInode *targeti)
+ // go.
+ // predirty dentry
+ dn->pre_dirty();
+
+ // add to event
+ EUpdate *le = new EUpdate("link_remote");
+ le->metablob.add_client_req(mdr->reqid);
+ version_t dirpv = predirty_dn_diri(mdr, dn, &le->metablob); // dir inode's mtime
+ le->metablob.add_dir_context(dn->get_dir());
+ le->metablob.add_remote_dentry(dn, true, targeti->ino()); // new remote
+
+ // mark committing (needed for proper recovery)
+ mdr->committing = true;
+
+ // log + wait
+ mdlog->submit_entry(le);
+ mdlog->wait_for_sync(new C_MDS_link_remote_finish(mds, mdr, dn, targeti, dirpv));
+}
+
+void Server::_link_remote_finish(MDRequest *mdr, CDentry *dn, CInode *targeti,
+ version_t dpv, version_t dirpv)
{
- // create remote link
+ dout(10) << "_link_remote_finish " << *dn << " to " << *targeti << endl;
+
+ // link the new dentry
dn->dir->link_inode(dn, targeti->ino());
- dn->link_remote( targeti ); // since we have it
- dn->_mark_dirty(); // fixme
+ dn->mark_dirty(dpv);
+
+ // dir inode's mtime
+ dirty_dn_diri(dn, dirpv, mdr->now);
- mds->balancer->hit_dir(dn->dir, META_POP_DWR);
+ // bump target popularity
+ mds->balancer->hit_inode(targeti, META_POP_IWR);
- // done!
- commit_request(req, new MClientReply(req, 0), ref,
- 0); // FIXME i should log something
+ // reply
+ MClientReply *reply = new MClientReply(mdr->client_request, 0);
+ reply_request(mdr, reply, dn->get_dir()->get_inode()); // FIXME: imprecise ref
}
-*/
-/*
-class C_MDS_RemoteLink : public Context {
+
+// remote linking/unlinking
+
+class C_MDS_SlaveLinkPrep : public Context {
Server *server;
- MClientRequest *req;
- CInode *ref;
- CDentry *dn;
+ MDRequest *mdr;
CInode *targeti;
+ utime_t old_ctime;
+ bool inc;
public:
- C_MDS_RemoteLink(Server *server, MClientRequest *req, CInode *ref, CDentry *dn, CInode *targeti) {
- this->server = server;
- this->req = req;
- this->ref = ref;
- this->dn = dn;
- this->targeti = targeti;
- }
+ C_MDS_SlaveLinkPrep(Server *s, MDRequest *r, CInode *t, utime_t oct, bool in) :
+ server(s), mdr(r), targeti(t), old_ctime(oct), inc(in) { }
void finish(int r) {
- if (r > 0) { // success
- // yay
- server->handle_client_link_finish(req, ref, dn, targeti);
+ assert(r == 0);
+ server->_logged_slave_link(mdr, targeti, old_ctime, inc);
+ }
+};
+
+void Server::handle_slave_link_prep(MDRequest *mdr)
+{
+ dout(10) << "handle_slave_link_prep " << *mdr
+ << " on " << mdr->slave_request->get_object_info()
+ << endl;
+
+ CInode *targeti = mdcache->get_inode(mdr->slave_request->get_object_info().ino);
+ assert(targeti);
+ dout(10) << "targeti " << *targeti << endl;
+ CDentry *dn = targeti->get_parent_dn();
+ assert(dn->is_primary());
+
+ mdr->now = mdr->slave_request->now;
+
+ // anchor?
+ if (mdr->slave_request->get_op() == MMDSSlaveRequest::OP_LINKPREP) {
+ if (targeti->is_anchored() && !targeti->is_unanchoring()) {
+ dout(7) << "target anchored already (nlink=" << targeti->inode.nlink << "), sweet" << endl;
}
- else if (r == 0) {
- // huh? retry!
- assert(0);
- server->dispatch_request(req, ref);
- } else {
- // link failed
- server->reply_request(req, r);
+ else {
+ dout(7) << "target needs anchor, nlink=" << targeti->inode.nlink << ", creating anchor" << endl;
+ mdcache->anchor_create(mdr, targeti,
+ new C_MDS_RetryRequest(mdcache, mdr));
+ return;
}
}
+
+
+ inode_t *pi = dn->inode->project_inode();
+
+ // update journaled target inode
+ bool inc;
+ if (mdr->slave_request->get_op() == MMDSSlaveRequest::OP_LINKPREP) {
+ inc = true;
+ pi->nlink++;
+ } else {
+ inc = false;
+ pi->nlink--;
+ }
+ utime_t old_ctime = pi->ctime;
+ pi->ctime = mdr->now;
+ pi->version = targeti->pre_dirty();
+
+ dout(10) << " projected inode " << pi << " v " << pi->version << endl;
+
+ // journal it
+ ESlaveUpdate *le = new ESlaveUpdate("slave_link_prep", mdr->reqid, mdr->slave_to_mds, ESlaveUpdate::OP_PREPARE);
+ le->metablob.add_dir_context(targeti->get_parent_dir());
+ le->metablob.add_primary_dentry(dn, true, targeti, pi); // update old primary
+ mds->mdlog->submit_entry(le, new C_MDS_SlaveLinkPrep(this, mdr, targeti, old_ctime, inc));
+}
+
+class C_MDS_SlaveLinkCommit : public Context {
+ Server *server;
+ MDRequest *mdr;
+ CInode *targeti;
+ utime_t old_ctime;
+ version_t old_version;
+ bool inc;
+public:
+ C_MDS_SlaveLinkCommit(Server *s, MDRequest *r, CInode *t, utime_t oct, version_t ov, bool in) :
+ server(s), mdr(r), targeti(t), old_ctime(oct), old_version(ov), inc(in) { }
+ void finish(int r) {
+ server->_commit_slave_link(mdr, r, targeti,
+ old_ctime, old_version, inc);
+ }
};
+void Server::_logged_slave_link(MDRequest *mdr, CInode *targeti, utime_t old_ctime, bool inc)
+{
+ dout(10) << "_logged_slave_link " << *mdr
+ << " inc=" << inc
+ << " " << *targeti << endl;
+
+ version_t old_version = targeti->inode.version;
+ // update the target
+ targeti->pop_and_dirty_projected_inode();
+
+ // ack
+ MMDSSlaveRequest *reply = new MMDSSlaveRequest(mdr->reqid, MMDSSlaveRequest::OP_LINKPREPACK);
+ mds->send_message_mds(reply, mdr->slave_to_mds, MDS_PORT_SERVER);
+
+ // set up commit waiter
+ mdr->slave_commit = new C_MDS_SlaveLinkCommit(this, mdr, targeti, old_ctime, old_version, inc);
+
+ // done.
+ delete mdr->slave_request;
+ mdr->slave_request = 0;
+}
+
+
+void Server::_commit_slave_link(MDRequest *mdr, int r, CInode *targeti,
+ utime_t old_ctime, version_t old_version, bool inc)
+{
+ dout(10) << "_commit_slave_link " << *mdr
+ << " r=" << r
+ << " inc=" << inc
+ << " " << *targeti << endl;
+
+ ESlaveUpdate *le;
+ if (r == 0) {
+ // write a commit to the journal
+ le = new ESlaveUpdate("slave_link_commit", mdr->reqid, mdr->slave_to_mds, ESlaveUpdate::OP_COMMIT);
} else {
- // remote: send nlink++ request, wait
- dout(7) << "target is remote, sending InodeLink" << endl;
- mds->send_message_mds(new MInodeLink(targeti->ino(), mds->get_nodeid()), targeti->authority().first, MDS_PORT_CACHE);
-
- // wait
- targeti->add_waiter(CInode::WAIT_LINK, new C_MDS_RemoteLink(this, req, diri, dn, targeti));
- return;
+ le = new ESlaveUpdate("slave_link_rollback", mdr->reqid, mdr->slave_to_mds, ESlaveUpdate::OP_ROLLBACK);
+
+ // -- rollback in memory --
+ assert(targeti->inode.ctime == mdr->now);
+ assert(targeti->projected_inode.empty()); // we're holding the version lock.
+
+ targeti->inode.ctime = old_ctime;
+ targeti->inode.version = old_version;
+ if (inc)
+ targeti->inode.nlink++;
+ else
+ targeti->inode.nlink--;
}
-*/
+ mds->mdlog->submit_entry(le);
+}
+
+
+
+void Server::handle_slave_link_prep_ack(MDRequest *mdr, MMDSSlaveRequest *m)
+{
+ dout(10) << "handle_slave_link_prep_ack " << *mdr
+ << " " << *m << endl;
+ int from = m->get_source().num();
+
+ // note slave
+ mdr->slaves.insert(from);
+
+ // witnessed!
+ assert(mdr->witnessed.count(from) == 0);
+ mdr->witnessed.insert(from);
+
+ // remove from waiting list
+ assert(mdr->waiting_on_slave.count(from));
+ mdr->waiting_on_slave.erase(from);
+
+ assert(mdr->waiting_on_slave.empty());
+
+ dispatch_client_request(mdr); // go again!
+}
void Server::handle_client_unlink(MDRequest *mdr)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
// traverse to path
vector<CDentry*> trace;
- Context *ondelay = new C_MDS_RetryRequest(mdcache, mdr);
- int r = mdcache->path_traverse(mdr, 0,
- req->get_filepath(), trace, false,
- req, ondelay,
+ int r = mdcache->path_traverse(mdr, req,
+ 0, req->get_filepath(), trace, false,
MDS_TRAVERSE_FORWARD);
if (r > 0) return;
if (trace.empty()) r = -EINVAL; // can't unlink root
CDentry *dn = trace[trace.size()-1];
assert(dn);
+
+ // is it my dentry?
+ if (!dn->is_auth()) {
+ // fw to auth
+ mdcache->request_forward(mdr, dn->authority().first);
+ return;
+ }
// rmdir or unlink?
bool rmdir = false;
}
// readable?
- if (!dn->lock.can_rdlock(mdr)) {
- dout(10) << "waiting on unreadable dentry " << *dn << endl;
+ if (dn->lock.is_xlocked_by_other(mdr)) {
+ dout(10) << "waiting on xlocked dentry " << *dn << endl;
dn->lock.add_waiter(SimpleLock::WAIT_RD, new C_MDS_RetryRequest(mdcache, mdr));
return;
}
if (!mds->locker->acquire_locks(mdr, rdlocks, wrlocks, xlocks))
return;
+ // yay!
+ mdr->done_locking = true; // avoid wrlock racing
+ if (mdr->now == utime_t())
+ mdr->now = g_clock.real_now();
+
+ // get stray dn ready?
+ CDentry *straydn = 0;
+ if (dn->is_primary()) {
+ straydn = mdcache->get_or_create_stray_dentry(dn->inode);
+ dout(10) << " straydn is " << *straydn << endl;
+
+ if (!mdr->dst_reanchor_atid &&
+ dn->inode->is_anchored()) {
+ dout(10) << "reanchoring to stray " << *dn->inode << endl;
+ vector<Anchor> trace;
+ straydn->make_anchor_trace(trace, dn->inode);
+ mds->anchorclient->prepare_update(dn->inode->ino(), trace, &mdr->dst_reanchor_atid,
+ new C_MDS_RetryRequest(mdcache, mdr));
+ return;
+ }
+ }
+
// ok!
if (dn->is_remote() && !dn->inode->is_auth())
_unlink_remote(mdr, dn);
else
- _unlink_local(mdr, dn);
+ _unlink_local(mdr, dn, straydn);
}
MDRequest *mdr;
CDentry *dn;
CDentry *straydn;
- version_t ipv; // referred inode
- utime_t ictime;
version_t dnpv; // deleted dentry
version_t dirpv;
public:
C_MDS_unlink_local_finish(MDS *m, MDRequest *r, CDentry *d, CDentry *sd,
- version_t v, version_t dirpv_, utime_t ct) :
+ version_t dirpv_) :
mds(m), mdr(r), dn(d), straydn(sd),
- ipv(v), ictime(ct),
dnpv(d->get_projected_version()), dirpv(dirpv_) { }
void finish(int r) {
assert(r == 0);
- mds->server->_unlink_local_finish(mdr, dn, straydn, ipv, ictime, dnpv, dirpv);
+ mds->server->_unlink_local_finish(mdr, dn, straydn, dnpv, dirpv);
}
};
-void Server::_unlink_local(MDRequest *mdr, CDentry *dn)
+void Server::_unlink_local(MDRequest *mdr, CDentry *dn, CDentry *straydn)
{
dout(10) << "_unlink_local " << *dn << endl;
- // get stray dn ready?
- CDentry *straydn = 0;
- if (dn->is_primary()) {
- string straydname;
- dn->inode->name_stray_dentry(straydname);
- frag_t fg = mdcache->get_stray()->pick_dirfrag(straydname);
- CDir *straydir = mdcache->get_stray()->get_or_open_dirfrag(mdcache, fg);
- straydn = straydir->add_dentry(straydname, 0);
- dout(10) << "_unlink_local straydn is " << *straydn << endl;
- }
-
-
// ok, let's do it.
// prepare log entry
EUpdate *le = new EUpdate("unlink_local");
le->metablob.add_client_req(mdr->reqid);
version_t ipv = 0; // dirty inode version
- inode_t *pi = 0; // the inode
-
+ inode_t *ji = 0; // journaled projected inode
if (dn->is_primary()) {
// primary link. add stray dentry.
assert(straydn);
ipv = straydn->pre_dirty(dn->inode->inode.version);
le->metablob.add_dir_context(straydn->dir);
- pi = le->metablob.add_primary_dentry(straydn, true, dn->inode);
+ ji = le->metablob.add_primary_dentry(straydn, true, dn->inode);
} else {
// remote link. update remote inode.
ipv = dn->inode->pre_dirty();
le->metablob.add_dir_context(dn->inode->get_parent_dir());
- pi = le->metablob.add_primary_dentry(dn->inode->parent, true, dn->inode); // update primary
+ ji = le->metablob.add_primary_dentry(dn->inode->parent, true, dn->inode);
}
+ // update journaled target inode
+ inode_t *pi = dn->inode->project_inode();
+ pi->nlink--;
+ pi->ctime = mdr->now;
+ pi->version = ipv;
+ *ji = *pi; // copy into journal
+
// the unlinked dentry
- utime_t now = g_clock.real_now();
dn->pre_dirty();
- version_t dirpv = predirty_dn_diri(dn, &le->metablob, now);
+ version_t dirpv = predirty_dn_diri(mdr, dn, &le->metablob);
le->metablob.add_dir_context(dn->get_dir());
le->metablob.add_null_dentry(dn, true);
- // update journaled target inode
- pi->nlink--;
- pi->ctime = now;
- pi->version = ipv;
-
+ if (mdr->dst_reanchor_atid)
+ le->metablob.add_anchor_transaction(mdr->dst_reanchor_atid);
+
// finisher
C_MDS_unlink_local_finish *fin = new C_MDS_unlink_local_finish(mds, mdr, dn, straydn,
- ipv, dirpv, now);
+ dirpv);
journal_opens(); // journal pending opens, just in case
void Server::_unlink_local_finish(MDRequest *mdr,
CDentry *dn, CDentry *straydn,
- version_t ipv, utime_t ictime, version_t dnpv, version_t dirpv)
+ version_t dnpv, version_t dirpv)
{
- dout(10) << "_unlink_local " << *dn << endl;
+ dout(10) << "_unlink_local_finish " << *dn << endl;
// unlink main dentry
CInode *in = dn->inode;
// relink as stray? (i.e. was primary link?)
if (straydn) straydn->dir->link_inode(straydn, in);
- // nlink--
- in->inode.ctime = ictime;
- in->inode.nlink--;
- in->mark_dirty(ipv); // dirty inode
- dn->mark_dirty(dnpv); // dirty old dentry
+ // nlink--, dirty old dentry
+ in->pop_and_dirty_projected_inode();
+ dn->mark_dirty(dnpv);
// dir inode's mtime
- dirty_dn_diri(dn, dirpv, ictime);
+ dirty_dn_diri(dn, dirpv, mdr->now);
+ // bump target popularity
+ mds->balancer->hit_dir(dn->dir, META_POP_DWR);
+
// share unlink news with replicas
for (map<int,int>::iterator it = dn->replicas_begin();
it != dn->replicas_end();
}
mds->send_message_mds(unlink, it->first, MDS_PORT_CACHE);
}
-
- // bump target popularity
- mds->balancer->hit_dir(dn->dir, META_POP_DWR);
+
+ // commit anchor update?
+ if (mdr->dst_reanchor_atid)
+ mds->anchorclient->commit(mdr->dst_reanchor_atid);
// reply
- MClientReply *reply = new MClientReply(mdr->client_request(), 0);
+ MClientReply *reply = new MClientReply(mdr->client_request, 0);
reply_request(mdr, reply, dn->dir->get_inode()); // FIXME: imprecise ref
-
+
+ // clean up?
if (straydn)
mdcache->eval_stray(straydn);
+
+ // removing a new dn?
+ dn->dir->try_remove_unlinked_dn(dn);
}
+class C_MDS_unlink_remote_finish : public Context {
+ MDS *mds;
+ MDRequest *mdr;
+ CDentry *dn;
+ version_t dnpv; // deleted dentry
+ version_t dirpv;
+public:
+ C_MDS_unlink_remote_finish(MDS *m, MDRequest *r, CDentry *d,
+ version_t dirpv_) :
+ mds(m), mdr(r), dn(d),
+ dnpv(d->get_projected_version()), dirpv(dirpv_) { }
+ void finish(int r) {
+ assert(r == 0);
+ mds->server->_unlink_remote_finish(mdr, dn, dnpv, dirpv);
+ }
+};
void Server::_unlink_remote(MDRequest *mdr, CDentry *dn)
{
- // IMPLEMENT ME
- reply_request(mdr, -EXDEV);
+ dout(10) << "_unlink_remote " << *dn << " " << *dn->inode << endl;
+
+ // 1. send LinkPrepare to dest (journal nlink-- prepare)
+ int inauth = dn->inode->authority().first;
+ if (mdr->witnessed.count(inauth) == 0) {
+ dout(10) << " inode auth must prepare nlink--" << endl;
+
+ MMDSSlaveRequest *req = new MMDSSlaveRequest(mdr->reqid, MMDSSlaveRequest::OP_UNLINKPREP);
+ dn->inode->set_object_info(req->get_object_info());
+ req->now = mdr->now;
+ mds->send_message_mds(req, inauth, MDS_PORT_SERVER);
+
+ assert(mdr->waiting_on_slave.count(inauth) == 0);
+ mdr->waiting_on_slave.insert(inauth);
+ return;
+ }
+ dout(10) << " inode auth has prepared nlink--" << endl;
+
+ // ok, let's do it.
+ // prepare log entry
+ EUpdate *le = new EUpdate("unlink_remote");
+ le->metablob.add_client_req(mdr->reqid);
+
+ // the unlinked dentry
+ dn->pre_dirty();
+ version_t dirpv = predirty_dn_diri(mdr, dn, &le->metablob);
+ le->metablob.add_dir_context(dn->get_dir());
+ le->metablob.add_null_dentry(dn, true);
+
+ if (mdr->dst_reanchor_atid)
+ le->metablob.add_anchor_transaction(mdr->dst_reanchor_atid);
+
+ // finisher
+ C_MDS_unlink_remote_finish *fin = new C_MDS_unlink_remote_finish(mds, mdr, dn, dirpv);
+
+ journal_opens(); // journal pending opens, just in case
+
+ // mark committing (needed for proper recovery)
+ mdr->committing = true;
+
+ // log + wait
+ mdlog->submit_entry(le);
+ mdlog->wait_for_sync(fin);
+
+ mds->balancer->hit_dir(dn->dir, META_POP_DWR);
+}
+
+void Server::_unlink_remote_finish(MDRequest *mdr,
+ CDentry *dn,
+ version_t dnpv, version_t dirpv)
+{
+ dout(10) << "_unlink_remote_finish " << *dn << endl;
+
+ // unlink main dentry
+ dn->dir->unlink_inode(dn);
+ dn->mark_dirty(dnpv); // dirty old dentry
+
+ // dir inode's mtime
+ dirty_dn_diri(dn, dirpv, mdr->now);
+
+ // bump target popularity
+ mds->balancer->hit_dir(dn->dir, META_POP_DWR);
+
+ // share unlink news with replicas
+ for (map<int,int>::iterator it = dn->replicas_begin();
+ it != dn->replicas_end();
+ it++) {
+ dout(7) << "_unlink_remote_finish sending MDentryUnlink to mds" << it->first << endl;
+ MDentryUnlink *unlink = new MDentryUnlink(dn->dir->dirfrag(), dn->name);
+ mds->send_message_mds(unlink, it->first, MDS_PORT_CACHE);
+ }
+
+ // commit anchor update?
+ if (mdr->dst_reanchor_atid)
+ mds->anchorclient->commit(mdr->dst_reanchor_atid);
+
+ // reply
+ MClientReply *reply = new MClientReply(mdr->client_request, 0);
+ reply_request(mdr, reply, dn->dir->get_inode()); // FIXME: imprecise ref
+
+ // removing a new dn?
+ dn->dir->try_remove_unlinked_dn(dn);
}
+// ======================================================
-// RENAME
-class C_MDS_RenameTraverseDst : public Context {
- Server *server;
+class C_MDS_rename_finish : public Context {
+ MDS *mds;
MDRequest *mdr;
- CInode *srci;
- CDir *srcdir;
CDentry *srcdn;
- filepath destpath;
+ CDentry *destdn;
+ CDentry *straydn;
public:
- vector<CDentry*> trace;
-
- C_MDS_RenameTraverseDst(Server *server,
- MDRequest *r,
- CDentry *srcdn,
- filepath& destpath) {
- this->server = server;
- this->mdr = r;
- this->srcdn = srcdn;
- this->destpath = destpath;
- }
+ C_MDS_rename_finish(MDS *m, MDRequest *r,
+ CDentry *sdn, CDentry *ddn, CDentry *stdn) :
+ mds(m), mdr(r),
+ srcdn(sdn), destdn(ddn), straydn(stdn) { }
void finish(int r) {
- server->handle_client_rename_2(mdr,
- srcdn, destpath,
- trace, r);
+ assert(r == 0);
+ mds->server->_rename_finish(mdr, srcdn, destdn, straydn);
}
};
/** handle_client_rename
*
- * NOTE: caller did not path_pin the ref (srcdir) inode, as it normally does.
- *
-
- weirdness iwith rename:
- - ref inode is what was originally srcdiri, but that may change by the time
- the rename actually happens. for all practical purpose, ref is useless except
- for C_MDS_RetryRequest
-
*/
-
-bool Server::_rename_open_dn(CDir *dir, CDentry *dn, bool mustexist, MDRequest *mdr)
-{
- // xlocked?
- if (dn && !dn->lock.can_rdlock(mdr)) {
- dout(10) << "_rename_open_dn waiting on " << *dn << endl;
- dn->lock.add_waiter(SimpleLock::WAIT_RD, new C_MDS_RetryRequest(mdcache, mdr));
- return false;
- }
-
- if (mustexist &&
- ((dn && dn->is_null()) ||
- (!dn && dir->is_complete()))) {
- dout(10) << "_rename_open_dn dn dne in " << *dir << endl;
- reply_request(mdr, -ENOENT);
- return false;
- }
-
- if (!dn && !dir->is_complete()) {
- dout(10) << "_rename_open_dn readding incomplete dir" << endl;
- dir->fetch(new C_MDS_RetryRequest(mdcache, mdr));
- return false;
- }
- assert(dn && !dn->is_null());
-
- dout(10) << "_rename_open_dn dn is " << *dn << endl;
- CInode *in = mdcache->get_dentry_inode(dn, mdr);
- if (!in) return false;
- dout(10) << "_rename_open_dn inode is " << *in << endl;
-
- return true;
-}
-
void Server::handle_client_rename(MDRequest *mdr)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
dout(7) << "handle_client_rename " << *req << endl;
// traverse to dest dir (not dest)
// traverse to src
filepath srcpath = req->get_filepath();
vector<CDentry*> srctrace;
- Context *ondelay = new C_MDS_RetryRequest(mdcache, mdr);
- int r = mdcache->path_traverse(mdr, 0,
- srcpath, srctrace, false,
- req, ondelay,
+ int r = mdcache->path_traverse(mdr, req,
+ 0, srcpath, srctrace, false,
MDS_TRAVERSE_DISCOVER);
if (r > 0) return;
if (srctrace.empty()) r = -EINVAL; // can't rename root
return;
}
CDentry *srcdn = srctrace[srctrace.size()-1];
- dout(10) << "srcdn is " << *srcdn << endl;
+ dout(10) << " srcdn " << *srcdn << endl;
CInode *srci = mdcache->get_dentry_inode(srcdn, mdr);
- dout(10) << "srci is " << *srci << endl;
+ dout(10) << " srci " << *srci << endl;
// -- some sanity checks --
// src == dest?
// identify/create dest dentry
CDentry *destdn = destdir->lookup(destname);
- if (destdn && !destdn->lock.can_rdlock(mdr)) {
+ if (destdn && destdn->lock.is_xlocked_by_other(mdr)) {
destdn->lock.add_waiter(SimpleLock::WAIT_RD, new C_MDS_RetryRequest(mdcache, mdr));
return;
}
CInode *oldin = 0;
if (destdn && !destdn->is_null()) {
- dout(10) << "dest dn exists " << *destdn << endl;
+ //dout(10) << "dest dn exists " << *destdn << endl;
oldin = mdcache->get_dentry_inode(destdn, mdr);
if (!oldin) return;
- dout(10) << "oldin " << *oldin << endl;
+ dout(10) << " oldin " << *oldin << endl;
// mv /some/thing /to/some/existing_other_thing
if (oldin->is_dir() && !srci->is_dir()) {
if (!destdn) return;
}
- dout(10) << "destdn " << *destdn << endl;
+ dout(10) << " destdn " << *destdn << endl;
// -- locks --
xlocks.insert(&destdn->lock);
wrlocks.insert(&destdn->dir->inode->dirlock);
- // xlock oldin
+ // xlock oldin (for nlink--)
if (oldin) xlocks.insert(&oldin->linklock);
if (!mds->locker->acquire_locks(mdr, rdlocks, wrlocks, xlocks))
return;
-
- // ok go!
- if (srcdn->is_auth() && destdn->is_auth())
- _rename_local(mdr, srcdn, destdn);
- else {
- // _rename_remote(mdr, srcdn, destdn);
- reply_request(mdr, -EXDEV);
- return;
+ // set done_locking flag, to avoid problems with wrlock moving auth target
+ mdr->done_locking = true;
+
+ // -- open all srcdn inode frags, if any --
+ // we need these open so that auth can properly delegate from inode to dirfrags
+ // after the inode is _ours_.
+ if (srcdn->is_primary() &&
+ !srcdn->is_auth() &&
+ srci->is_dir()) {
+ dout(10) << "srci is remote dir, opening all frags" << endl;
+ list<frag_t> frags;
+ srci->dirfragtree.get_leaves(frags);
+ for (list<frag_t>::iterator p = frags.begin();
+ p != frags.end();
+ ++p) {
+ CDir *dir = srci->get_dirfrag(*p);
+ if (dir) {
+ dout(10) << " opened " << *dir << endl;
+ mdr->pin(dir);
+ } else {
+ mdcache->open_remote_dir(srci, *p, new C_MDS_RetryRequest(mdcache, mdr));
+ return;
+ }
+ }
}
-}
+ // -- declare now --
+ if (mdr->now == utime_t())
+ mdr->now = g_clock.real_now();
+ // -- create stray dentry? --
+ CDentry *straydn = 0;
+ if (destdn->is_primary()) {
+ straydn = mdcache->get_or_create_stray_dentry(destdn->inode);
+ dout(10) << "straydn is " << *straydn << endl;
+ }
+ // -- prepare witnesses --
+ set<int> witnesses = mdr->extra_witnesses;
+ if (srcdn->is_auth())
+ srcdn->list_replicas(witnesses);
+ else
+ witnesses.insert(srcdn->authority().first);
+ destdn->list_replicas(witnesses);
-class C_MDS_rename_local_finish : public Context {
- MDS *mds;
- MDRequest *mdr;
- CDentry *srcdn;
- CDentry *destdn;
- CDentry *straydn;
- version_t ipv;
- version_t straypv;
- version_t destpv;
- version_t srcpv;
- version_t ddirpv, sdirpv;
- utime_t ictime;
-public:
- version_t atid1;
- version_t atid2;
- C_MDS_rename_local_finish(MDS *m, MDRequest *r,
- CDentry *sdn, CDentry *ddn, CDentry *stdn,
- version_t v, version_t ddirpv_, version_t sdirpv_, utime_t ct) :
- mds(m), mdr(r),
- srcdn(sdn), destdn(ddn), straydn(stdn),
- ipv(v),
- straypv(straydn ? straydn->get_projected_version():0),
- destpv(destdn->get_projected_version()),
- srcpv(srcdn->get_projected_version()),
- ddirpv(ddirpv_), sdirpv(sdirpv_),
- ictime(ct),
- atid1(0), atid2(0) { }
- void finish(int r) {
- assert(r == 0);
- mds->server->_rename_local_finish(mdr, srcdn, destdn, straydn,
- srcpv, destpv, straypv, ipv, ddirpv, sdirpv, ictime,
- atid1, atid2);
- }
-};
+ for (set<int>::iterator p = witnesses.begin();
+ p != witnesses.end();
+ ++p) {
+ if (mdr->witnessed.count(*p)) {
+ dout(10) << " already witnessed by mds" << *p << endl;
+ } else {
+ dout(10) << " not yet witnessed by mds" << *p << ", sending prepare" << endl;
+ MMDSSlaveRequest *req = new MMDSSlaveRequest(mdr->reqid, MMDSSlaveRequest::OP_RENAMEPREP);
+ srcdn->make_path(req->srcdnpath);
+ destdn->make_path(req->destdnpath);
+ req->now = mdr->now;
+
+ if (straydn) {
+ CInodeDiscover *indis = straydn->dir->inode->replicate_to(*p);
+ CDirDiscover *dirdis = straydn->dir->replicate_to(*p);
+ CDentryDiscover *dndis = straydn->replicate_to(*p);
+ indis->_encode(req->stray);
+ dirdis->_encode(req->stray);
+ dndis->_encode(req->stray);
+ delete indis;
+ delete dirdis;
+ delete dndis;
+ }
-class C_MDS_rename_local_anchor : public Context {
- Server *server;
-public:
- LogEvent *le;
- C_MDS_rename_local_finish *fin;
- version_t atid1;
- version_t atid2;
-
- C_MDS_rename_local_anchor(Server *s) : server(s), le(0), fin(0), atid1(0), atid2(0) { }
- void finish(int r) {
- server->_rename_local_reanchored(le, fin, atid1, atid2);
+ mds->send_message_mds(req, *p, MDS_PORT_SERVER);
+
+ assert(mdr->waiting_on_slave.count(*p) == 0);
+ mdr->waiting_on_slave.insert(*p);
+ }
}
-};
+ if (!mdr->waiting_on_slave.empty())
+ return; // we're waiting for a witness.
+
+ // -- inode migration? --
+ if (!srcdn->is_auth() &&
+ srcdn->is_primary()) {
+ if (mdr->inode_import.length() == 0) {
+ // get inode
+ int auth = srcdn->authority().first;
+ dout(10) << " requesting inode export from srcdn auth mds" << auth << endl;
+ MMDSSlaveRequest *req = new MMDSSlaveRequest(mdr->reqid, MMDSSlaveRequest::OP_RENAMEGETINODE);
+ srcdn->make_path(req->srcdnpath);
+ mds->send_message_mds(req, auth, MDS_PORT_SERVER);
+
+ assert(mdr->waiting_on_slave.count(auth) == 0);
+ mdr->waiting_on_slave.insert(auth);
+ return;
+ } else {
+ dout(10) << " already (just!) got inode export from srcdn auth" << endl;
+ /*int off = 0;
+ mdcache->migrator->decode_import_inode(destdn, mdr->inode_import, off,
+ srcdn->authority().first);
+ srcdn->inode->force_auth.first = srcdn->authority().first;
+ */
+ }
+ }
+
+ // -- prepare anchor updates --
+ bool linkmerge = (srcdn->inode == destdn->inode &&
+ (srcdn->is_primary() || destdn->is_primary()));
-void Server::_rename_local(MDRequest *mdr,
- CDentry *srcdn,
- CDentry *destdn)
-{
- dout(10) << "_rename_local " << *srcdn << " to " << *destdn << endl;
+ if (!linkmerge) {
+ C_Gather *anchorgather = 0;
+
+ if (srcdn->is_primary() && srcdn->inode->is_anchored() &&
+ srcdn->dir != destdn->dir &&
+ !mdr->src_reanchor_atid) {
+ dout(10) << "reanchoring src->dst " << *srcdn->inode << endl;
+ vector<Anchor> trace;
+ destdn->make_anchor_trace(trace, srcdn->inode);
+
+ anchorgather = new C_Gather(new C_MDS_RetryRequest(mdcache, mdr));
+ mds->anchorclient->prepare_update(srcdn->inode->ino(), trace, &mdr->src_reanchor_atid,
+ anchorgather->new_sub());
+ }
+ if (destdn->is_primary() &&
+ destdn->inode->is_anchored() &&
+ !mdr->dst_reanchor_atid) {
+ dout(10) << "reanchoring dst->stray " << *destdn->inode << endl;
+
+ assert(straydn);
+ vector<Anchor> trace;
+ straydn->make_anchor_trace(trace, destdn->inode);
+
+ if (!anchorgather)
+ anchorgather = new C_Gather(new C_MDS_RetryRequest(mdcache, mdr));
+ mds->anchorclient->prepare_update(destdn->inode->ino(), trace, &mdr->dst_reanchor_atid,
+ anchorgather->new_sub());
+ }
+
+ if (anchorgather)
+ return; // waiting for anchor prepares
+ }
- // let's go.
- EUpdate *le = new EUpdate("rename_local");
+ // -- prepare journal entry --
+ EUpdate *le = new EUpdate("rename");
le->metablob.add_client_req(mdr->reqid);
+
+ _rename_prepare(mdr, &le->metablob, srcdn, destdn, straydn);
- utime_t now = g_clock.real_now();
+ // -- commit locally --
+ C_MDS_rename_finish *fin = new C_MDS_rename_finish(mds, mdr, srcdn, destdn, straydn);
- CDentry *straydn = 0;
- inode_t *pi = 0;
- version_t ipv = 0;
+ journal_opens(); // journal pending opens, just in case
+
+ // mark committing (needed for proper recovery)
+ mdr->committing = true;
+
+ // log + wait
+ mdlog->submit_entry(le);
+ mdlog->wait_for_sync(fin);
+}
+
+
+void Server::_rename_finish(MDRequest *mdr, CDentry *srcdn, CDentry *destdn, CDentry *straydn)
+{
+ dout(10) << "_rename_finish " << *mdr << endl;
+
+ // apply
+ _rename_apply(mdr, srcdn, destdn, straydn);
+
+ // commit anchor updates?
+ if (mdr->src_reanchor_atid) mds->anchorclient->commit(mdr->src_reanchor_atid);
+ if (mdr->dst_reanchor_atid) mds->anchorclient->commit(mdr->dst_reanchor_atid);
+
+ // reply
+ MClientReply *reply = new MClientReply(mdr->client_request, 0);
+ reply_request(mdr, reply, destdn->dir->get_inode()); // FIXME: imprecise ref
- C_MDS_rename_local_anchor *anchorfin = 0;
- C_Gather *anchorgather = 0;
+ // clean up?
+ if (straydn)
+ mdcache->eval_stray(straydn);
+}
+
+
+
+// helpers
+
+void Server::_rename_prepare(MDRequest *mdr,
+ EMetaBlob *metablob,
+ CDentry *srcdn, CDentry *destdn, CDentry *straydn)
+{
+ dout(10) << "_rename_prepare " << *mdr << " " << *srcdn << " " << *destdn << endl;
// primary+remote link merge?
bool linkmerge = (srcdn->inode == destdn->inode &&
(srcdn->is_primary() || destdn->is_primary()));
- // dir mtimes
- version_t ddirpv = predirty_dn_diri(destdn, &le->metablob, now);
- version_t sdirpv = predirty_dn_diri(srcdn, &le->metablob, now);
+ if (mdr->is_master()) {
+ mdr->pvmap[destdn->dir->inode] = predirty_dn_diri(mdr, destdn, metablob);
+ if (destdn->dir != srcdn->dir)
+ mdr->pvmap[srcdn->dir->inode] = predirty_dn_diri(mdr, srcdn, metablob);
+ }
+ inode_t *ji = 0; // journaled inode getting nlink--
+ version_t ipv; // it's version
+
if (linkmerge) {
dout(10) << "will merge remote+primary links" << endl;
// destdn -> primary
- le->metablob.add_dir_context(destdn->dir);
- ipv = destdn->pre_dirty(destdn->inode->inode.version);
- pi = le->metablob.add_primary_dentry(destdn, true, destdn->inode);
+ metablob->add_dir_context(destdn->dir);
+ if (destdn->is_auth())
+ ipv = mdr->pvmap[destdn] = destdn->pre_dirty(destdn->inode->inode.version);
+ ji = metablob->add_primary_dentry(destdn, true, destdn->inode);
// do src dentry
- le->metablob.add_dir_context(srcdn->dir);
- srcdn->pre_dirty();
- le->metablob.add_null_dentry(srcdn, true);
-
- // anchor update?
- if (srcdn->is_primary() && srcdn->inode->is_anchored() &&
- srcdn->dir != destdn->dir) {
- dout(10) << "reanchoring src->dst " << *srcdn->inode << endl;
- vector<Anchor> trace;
- destdn->make_anchor_trace(trace, srcdn->inode);
- anchorfin = new C_MDS_rename_local_anchor(this);
- mds->anchorclient->prepare_update(srcdn->inode->ino(), trace, &anchorfin->atid1, anchorfin);
- }
+ metablob->add_dir_context(srcdn->dir);
+ if (srcdn->is_auth())
+ mdr->pvmap[srcdn] = srcdn->pre_dirty();
+ metablob->add_null_dentry(srcdn, true);
} else {
// move to stray?
if (destdn->is_primary()) {
- // primary.
- // move inode to stray dir.
- string straydname;
- destdn->inode->name_stray_dentry(straydname);
- frag_t fg = mdcache->get_stray()->pick_dirfrag(straydname);
- CDir *straydir = mdcache->get_stray()->get_or_open_dirfrag(mdcache, fg);
- straydn = straydir->add_dentry(straydname, 0);
- dout(10) << "straydn is " << *straydn << endl;
-
- // renanchor?
- if (destdn->inode->is_anchored()) {
- dout(10) << "reanchoring dst->stray " << *destdn->inode << endl;
- vector<Anchor> trace;
- straydn->make_anchor_trace(trace, destdn->inode);
- anchorfin = new C_MDS_rename_local_anchor(this);
- anchorgather = new C_Gather(anchorfin);
- mds->anchorclient->prepare_update(destdn->inode->ino(), trace, &anchorfin->atid1,
- anchorgather->new_sub());
- }
+ // primary. we'll move inode to stray dir.
+ assert(straydn);
// link-- inode, move to stray dir.
- le->metablob.add_dir_context(straydn->dir);
- ipv = straydn->pre_dirty(destdn->inode->inode.version);
- pi = le->metablob.add_primary_dentry(straydn, true, destdn->inode);
+ metablob->add_dir_context(straydn->dir);
+ if (straydn->is_auth())
+ ipv = mdr->pvmap[straydn] = straydn->pre_dirty(destdn->inode->inode.version);
+ ji = metablob->add_primary_dentry(straydn, true, destdn->inode);
}
else if (destdn->is_remote()) {
// remote.
// nlink-- targeti
- le->metablob.add_dir_context(destdn->inode->get_parent_dir());
- ipv = destdn->inode->pre_dirty();
- pi = le->metablob.add_primary_dentry(destdn->inode->parent, true, destdn->inode); // update primary
+ metablob->add_dir_context(destdn->inode->get_parent_dir());
+ if (destdn->inode->is_auth())
+ ipv = mdr->pvmap[destdn->inode] = destdn->inode->pre_dirty();
+ ji = metablob->add_primary_dentry(destdn->inode->parent, true, destdn->inode); // update primary
dout(10) << "remote targeti (nlink--) is " << *destdn->inode << endl;
}
else {
}
// add dest dentry
- le->metablob.add_dir_context(destdn->dir);
+ metablob->add_dir_context(destdn->dir);
if (srcdn->is_primary()) {
dout(10) << "src is a primary dentry" << endl;
- destdn->pre_dirty(srcdn->inode->inode.version);
- le->metablob.add_primary_dentry(destdn, true, srcdn->inode);
-
- if (srcdn->inode->is_anchored()) {
- dout(10) << "reanchoring src->dst " << *srcdn->inode << endl;
- vector<Anchor> trace;
- destdn->make_anchor_trace(trace, srcdn->inode);
- if (!anchorfin) anchorfin = new C_MDS_rename_local_anchor(this);
- if (!anchorgather) anchorgather = new C_Gather(anchorfin);
- mds->anchorclient->prepare_update(srcdn->inode->ino(), trace, &anchorfin->atid2,
- anchorgather->new_sub());
-
+ if (destdn->is_auth()) {
+ version_t siv;
+ if (srcdn->is_auth())
+ siv = srcdn->inode->get_projected_version();
+ else
+ siv = mdr->inode_import_v;
+ mdr->pvmap[destdn] = destdn->pre_dirty(siv+1);
}
+ metablob->add_primary_dentry(destdn, true, srcdn->inode);
+
} else {
assert(srcdn->is_remote());
dout(10) << "src is a remote dentry" << endl;
- destdn->pre_dirty();
- le->metablob.add_remote_dentry(destdn, true, srcdn->get_remote_ino());
+ if (destdn->is_auth())
+ mdr->pvmap[destdn] = destdn->pre_dirty();
+ metablob->add_remote_dentry(destdn, true, srcdn->get_remote_ino());
}
// remove src dentry
- le->metablob.add_dir_context(srcdn->dir);
- srcdn->pre_dirty();
- le->metablob.add_null_dentry(srcdn, true);
- }
-
- if (pi) {
+ metablob->add_dir_context(srcdn->dir);
+ if (srcdn->is_auth())
+ mdr->pvmap[srcdn] = srcdn->pre_dirty();
+ metablob->add_null_dentry(srcdn, true);
+
+ // new subtree?
+ if (srcdn->is_primary() &&
+ srcdn->inode->is_dir()) {
+ list<CDir*> ls;
+ srcdn->inode->get_nested_dirfrags(ls);
+ int auth = srcdn->authority().first;
+ for (list<CDir*>::iterator p = ls.begin(); p != ls.end(); ++p)
+ mdcache->adjust_subtree_auth(*p, auth, auth);
+ }
+ }
+
+ if (ji) {
// update journaled target inode
+ inode_t *pi = destdn->inode->project_inode();
pi->nlink--;
- pi->ctime = now;
+ pi->ctime = mdr->now;
pi->version = ipv;
+ *ji = *pi; // copy into journal
}
- C_MDS_rename_local_finish *fin = new C_MDS_rename_local_finish(mds, mdr,
- srcdn, destdn, straydn,
- ipv, ddirpv, sdirpv, now);
-
- journal_opens(); // journal pending opens, just in case
-
- if (anchorfin) {
- // doing anchor update prepare first
- anchorfin->fin = fin;
- anchorfin->le = le;
- } else {
- // log + wait
- mdlog->submit_entry(le);
- mdlog->wait_for_sync(fin);
- }
-}
-
-
-void Server::_rename_local_reanchored(LogEvent *le, C_MDS_rename_local_finish *fin,
- version_t atid1, version_t atid2)
-{
- dout(10) << "_rename_local_reanchored, logging " << *le << endl;
-
- // note anchor transaction ids
- fin->atid1 = atid1;
- fin->atid2 = atid2;
-
- // log + wait
- mdlog->submit_entry(le);
- mdlog->wait_for_sync(fin);
+ // anchor updates?
+ if (mdr->src_reanchor_atid)
+ metablob->add_anchor_transaction(mdr->src_reanchor_atid);
+ if (mdr->dst_reanchor_atid)
+ metablob->add_anchor_transaction(mdr->dst_reanchor_atid);
}
-void Server::_rename_local_finish(MDRequest *mdr,
- CDentry *srcdn, CDentry *destdn, CDentry *straydn,
- version_t srcpv, version_t destpv, version_t straypv, version_t ipv,
- version_t ddirpv, version_t sdirpv,
- utime_t ictime,
- version_t atid1, version_t atid2)
+void Server::_rename_apply(MDRequest *mdr, CDentry *srcdn, CDentry *destdn, CDentry *straydn)
{
- MClientRequest *req = mdr->client_request();
- dout(10) << "_rename_local_finish " << *req << endl;
+ dout(10) << "_rename_apply " << *mdr << " " << *srcdn << " " << *destdn << endl;
+ dout(10) << " pvs " << mdr->pvmap << endl;
CInode *oldin = destdn->inode;
(srcdn->is_primary() || destdn->is_primary()));
// dir mtimes
- dirty_dn_diri(destdn, ddirpv, ictime);
- dirty_dn_diri(srcdn, sdirpv, ictime);
-
- if (linkmerge) {
- assert(ipv);
+ if (mdr->is_master()) {
+ dirty_dn_diri(destdn, mdr->pvmap[destdn->dir->inode], mdr->now);
+ if (destdn->dir != srcdn->dir)
+ dirty_dn_diri(srcdn, mdr->pvmap[srcdn->dir->inode], mdr->now);
+ }
+ if (linkmerge) {
if (destdn->is_primary()) {
dout(10) << "merging remote onto primary link" << endl;
// nlink-- in place
destdn->inode->inode.nlink--;
- destdn->inode->inode.ctime = ictime;
- destdn->inode->mark_dirty(destpv);
+ destdn->inode->inode.ctime = mdr->now;
+ if (destdn->inode->is_auth())
+ destdn->inode->mark_dirty(mdr->pvmap[destdn]);
// unlink srcdn
srcdn->dir->unlink_inode(srcdn);
- srcdn->mark_dirty(srcpv);
+ if (srcdn->is_auth())
+ srcdn->mark_dirty(mdr->pvmap[srcdn]);
} else {
dout(10) << "merging primary onto remote link" << endl;
assert(srcdn->is_primary());
// nlink--
destdn->inode->inode.nlink--;
- destdn->inode->inode.ctime = ictime;
- destdn->inode->mark_dirty(destpv);
+ destdn->inode->inode.ctime = mdr->now;
+ if (destdn->inode->is_auth())
+ destdn->inode->mark_dirty(mdr->pvmap[destdn]);
// mark src dirty
- srcdn->mark_dirty(srcpv);
+ if (srcdn->is_auth())
+ srcdn->mark_dirty(mdr->pvmap[srcdn]);
}
}
else {
// unlink destdn?
if (!destdn->is_null())
destdn->dir->unlink_inode(destdn);
-
+
if (straydn) {
- // relink oldin to stray dir
+ dout(10) << "straydn is " << *straydn << endl;
+
+ // relink oldin to stray dir. destdn was primary.
assert(oldin);
straydn->dir->link_inode(straydn, oldin);
- assert(straypv == ipv);
+ //assert(straypv == ipv);
+
+ // nlink-- in stray dir.
+ oldin->inode.nlink--;
+ oldin->inode.ctime = mdr->now;
+ if (oldin->is_auth())
+ oldin->mark_dirty(mdr->pvmap[straydn]);
}
-
- if (oldin) {
- // nlink--
+ else if (oldin) {
+ // nlink-- remote. destdn was remote.
oldin->inode.nlink--;
- oldin->inode.ctime = ictime;
- oldin->mark_dirty(ipv);
+ oldin->inode.ctime = mdr->now;
+ if (oldin->is_auth())
+ oldin->mark_dirty(mdr->pvmap[oldin]);
}
CInode *in = srcdn->inode;
assert(in);
if (srcdn->is_remote()) {
+ // srcdn was remote.
srcdn->dir->unlink_inode(srcdn);
destdn->dir->link_inode(destdn, in->ino());
+ if (destdn->is_auth())
+ destdn->mark_dirty(mdr->pvmap[destdn]);
} else {
+ // srcdn was primary.
srcdn->dir->unlink_inode(srcdn);
destdn->dir->link_inode(destdn, in);
+
+ // srcdn inode import?
+ if (!srcdn->is_auth() && destdn->is_auth()) {
+ assert(mdr->inode_import.length() > 0);
+ int off = 0;
+ map<int,entity_inst_t> imported_client_map;
+ ::_decode(imported_client_map, mdr->inode_import, off);
+ mdcache->migrator->decode_import_inode(destdn, mdr->inode_import, off,
+ srcdn->authority().first,
+ imported_client_map);
+ }
+ if (destdn->inode->is_auth())
+ destdn->inode->mark_dirty(mdr->pvmap[destdn]);
}
- destdn->mark_dirty(destpv);
- srcdn->mark_dirty(srcpv);
- }
- // commit anchor updates?
- if (atid1) mds->anchorclient->commit(atid1);
- if (atid2) mds->anchorclient->commit(atid2);
+ if (srcdn->is_auth())
+ srcdn->mark_dirty(mdr->pvmap[srcdn]);
+ }
// update subtree map?
- if (destdn->inode->is_dir())
+ if (destdn->is_primary() && destdn->inode->is_dir())
mdcache->adjust_subtree_after_rename(destdn->inode, srcdn->dir);
- // share news with replicas
- // ***
+ // removing a new dn?
+ srcdn->dir->try_remove_unlinked_dn(srcdn);
+}
- // reply
- MClientReply *reply = new MClientReply(req, 0);
- reply_request(mdr, reply, destdn->dir->get_inode()); // FIXME: imprecise ref
- // clean up?
- if (straydn)
- mdcache->eval_stray(straydn);
-}
+// ------------
+// SLAVE
-/*
-void Server::handle_client_rename_local(MClientRequest *req,
- CInode *ref,
- const string& srcpath,
- CInode *srcdiri,
- CDentry *srcdn,
- const string& destpath,
- CDir *destdir,
- CDentry *destdn,
- const string& destname)
+class C_MDS_SlaveRenamePrep : public Context {
+ Server *server;
+ MDRequest *mdr;
+ CDentry *srcdn, *destdn, *straydn;
+public:
+ C_MDS_SlaveRenamePrep(Server *s, MDRequest *m, CDentry *sr, CDentry *de, CDentry *st) :
+ server(s), mdr(m), srcdn(sr), destdn(de), straydn(st) {}
+ void finish(int r) {
+ server->_logged_slave_rename(mdr, srcdn, destdn, straydn);
+ }
+};
+
+class C_MDS_SlaveRenameCommit : public Context {
+ Server *server;
+ MDRequest *mdr;
+ CDentry *srcdn, *destdn, *straydn;
+public:
+ C_MDS_SlaveRenameCommit(Server *s, MDRequest *m, CDentry *sr, CDentry *de, CDentry *st) :
+ server(s), mdr(m), srcdn(sr), destdn(de), straydn(st) {}
+ void finish(int r) {
+ server->_commit_slave_rename(mdr, r, srcdn, destdn, straydn);
+ }
+};
+
+void Server::handle_slave_rename_prep(MDRequest *mdr)
{
-*/
- //bool everybody = false;
- //if (true || srcdn->inode->is_dir()) {
- /* overkill warning: lock w/ everyone for simplicity. FIXME someday! along with the foreign rename crap!
- i could limit this to cases where something beneath me is exported.
- could possibly limit the list. (maybe.)
- Underlying constraint is that, regardless of the order i do the xlocks, and whatever
- imports/exports might happen in the process, the destdir _must_ exist on any node
- importing something beneath me when rename finishes, or else mayhem ensues when
- their import is dangling in the cache.
- */
- /*
- having made a proper mess of this on the first pass, here is my plan:
+ dout(10) << "handle_slave_rename_prep " << *mdr
+ << " " << mdr->slave_request->srcdnpath
+ << " to " << mdr->slave_request->destdnpath
+ << endl;
+
+ // discover destdn
+ filepath destpath(mdr->slave_request->destdnpath);
+ dout(10) << " dest " << destpath << endl;
+ vector<CDentry*> trace;
+ int r = mdcache->path_traverse(mdr, mdr->slave_request,
+ 0, destpath, trace, false,
+ MDS_TRAVERSE_DISCOVERXLOCK);
+ if (r > 0) return;
+ assert(r == 0); // we shouldn't get an error here!
- - xlocks of src, dest are done in lex order
- - xlock is optional.. if you have the dentry, lock it, if not, don't.
- - if you discover an xlocked dentry, you get the xlock.
-
- possible trouble:
- - you have an import beneath the source, and don't have the dest dir.
- - when the actual rename happens, you discover the dest
- - actually, do this on any open dir, so we don't detach whole swaths
- of our cache.
+ CDentry *destdn = trace[trace.size()-1];
+ dout(10) << " destdn " << *destdn << endl;
+ mdr->pin(destdn);
+
- notes:
- - xlocks are initiated from authority, as are discover_replies, so replicas are
- guaranteed to either not have dentry, or to have it xlocked.
- -
- - foreign xlocks are eventually unraveled by the initiator on success or failure.
-
- todo to make this work:
- - hose bool everybody param crap
- /- make handle_lock_dn not discover, clean up cases
- /- put dest path in MRenameNotify
- /- make rename_notify discover if its a dir
- / - this will catch nested imports too, obviously
- /- notify goes to merged list on local rename
- /- notify goes to everybody on a foreign rename
- /- handle_notify needs to gracefully ignore spurious notifies
- */
- //dout(7) << "handle_client_rename_local: overkill? doing xlocks with _all_ nodes" << endl;
- //everybody = true;
- //}
-/*
- bool srclocal = srcdn->dir->dentry_authority(srcdn->name).first == mds->get_nodeid();
- bool destlocal = destdir->dentry_authority(destname).first == mds->get_nodeid();
+ // discover srcdn
+ filepath srcpath(mdr->slave_request->srcdnpath);
+ dout(10) << " src " << srcpath << endl;
+ r = mdcache->path_traverse(mdr, mdr->slave_request,
+ 0, srcpath, trace, false,
+ MDS_TRAVERSE_DISCOVERXLOCK);
+ if (r > 0) return;
+ assert(r == 0); // we shouldn't get an error here!
+
+ CDentry *srcdn = trace[trace.size()-1];
+ dout(10) << " srcdn " << *srcdn << endl;
+ mdr->pin(srcdn);
+ assert(srcdn->inode);
+ mdr->pin(srcdn->inode);
- dout(7) << "handle_client_rename_local: src local=" << srclocal << " " << *srcdn << endl;
- if (destdn) {
- dout(7) << "handle_client_rename_local: dest local=" << destlocal << " " << *destdn << endl;
+ // stray?
+ CDentry *straydn = 0;
+ if (destdn->is_primary()) {
+ assert(mdr->slave_request->stray.length() > 0);
+ straydn = mdcache->add_replica_stray(mdr->slave_request->stray,
+ destdn->inode, mdr->slave_to_mds);
+ assert(straydn);
+ mdr->pin(straydn);
+ }
+
+ mdr->now = mdr->slave_request->now;
+
+ // journal it?
+ if (srcdn->is_auth() ||
+ destdn->inode->is_auth() ||
+ srcdn->inode->is_any_caps()) {
+ // journal.
+ ESlaveUpdate *le = new ESlaveUpdate("slave_rename_prep", mdr->reqid, mdr->slave_to_mds, ESlaveUpdate::OP_PREPARE);
+ _rename_prepare(mdr, &le->metablob, srcdn, destdn, straydn);
+ mds->mdlog->submit_entry(le, new C_MDS_SlaveRenamePrep(this, mdr, srcdn, destdn, straydn));
} else {
- dout(7) << "handle_client_rename_local: dest local=" << destlocal << " dn dne yet" << endl;
+ // don't journal.
+ dout(10) << "not journaling, i'm not auth for anything, and srci isn't open" << endl;
+ _logged_slave_rename(mdr, srcdn, destdn, straydn);
}
+}
- // lock source and dest dentries, in lexicographic order.
- bool dosrc = srcpath < destpath;
- for (int i=0; i<2; i++) {
- if (dosrc) {
+void Server::_logged_slave_rename(MDRequest *mdr,
+ CDentry *srcdn, CDentry *destdn, CDentry *straydn)
+{
+ dout(10) << "_logged_slave_rename " << *mdr << endl;
- // src
- if (srclocal) {
- if (!srcdn->is_xlockedbyme(req) &&
- !mds->locker->dentry_xlock_start(srcdn, req, ref))
- return;
- } else {
- if (!srcdn || srcdn->xlockedby != req) {
- mds->locker->dentry_xlock_request(srcdn->dir, srcdn->name, false, req, new C_MDS_RetryRequest(mds, req, ref));
- return;
- }
- }
- dout(7) << "handle_client_rename_local: srcdn is xlock " << *srcdn << endl;
-
- } else {
+ // ack
+ MMDSSlaveRequest *reply = new MMDSSlaveRequest(mdr->reqid, MMDSSlaveRequest::OP_RENAMEPREPACK);
+ if (srcdn->is_auth()) {
+ // share the replica list, so that they can all witness the rename.
+ srcdn->list_replicas(reply->srcdn_replicas);
- if (destlocal) {
- // dest
- if (!destdn) destdn = destdir->add_dentry(destname);
- if (!destdn->is_xlockedbyme(req) &&
- !mds->locker->dentry_xlock_start(destdn, req, ref)) {
- if (destdn->is_clean() && destdn->is_null() && destdn->is_sync()) destdir->remove_dentry(destdn);
- return;
- }
- } else {
- if (!destdn || destdn->xlockedby != req) {
- // NOTE: require that my xlocked item be a leaf/file, NOT a dir. in case
- // my traverse and determination of dest vs dest/srcfilename was out of date.
- mds->locker->dentry_xlock_request(destdir, destname, true, req, new C_MDS_RetryRequest(mds, req, ref));
- return;
- }
- }
- dout(7) << "handle_client_rename_local: destdn is xlock " << *destdn << endl;
+ // note srcdn, we'll get asked for inode momentarily
+ mdr->srcdn = srcdn;
+ }
- }
+ mds->send_message_mds(reply, mdr->slave_to_mds, MDS_PORT_SERVER);
+
+ // set up commit waiter
+ mdr->slave_commit = new C_MDS_SlaveRenameCommit(this, mdr, srcdn, destdn, straydn);
+
+ // done.
+ delete mdr->slave_request;
+ mdr->slave_request = 0;
+}
+
+void Server::_commit_slave_rename(MDRequest *mdr, int r,
+ CDentry *srcdn, CDentry *destdn, CDentry *straydn)
+{
+ dout(10) << "_commit_slave_rename " << *mdr << " r=" << r << endl;
+
+ ESlaveUpdate *le;
+ if (r == 0) {
+ // commit
+ _rename_apply(mdr, srcdn, destdn, straydn);
- dosrc = !dosrc;
+ // write a commit to the journal
+ le = new ESlaveUpdate("slave_rename_commit", mdr->reqid, mdr->slave_to_mds, ESlaveUpdate::OP_COMMIT);
+ } else {
+ // abort
+ le = new ESlaveUpdate("slave_rename_abort", mdr->reqid, mdr->slave_to_mds, ESlaveUpdate::OP_ROLLBACK);
}
+ mds->mdlog->submit_entry(le);
+}
-
- // final check: verify if dest exists that src is a file
+void Server::handle_slave_rename_prep_ack(MDRequest *mdr, MMDSSlaveRequest *m)
+{
+ dout(10) << "handle_slave_rename_prep_ack " << *mdr
+ << " witnessed by " << m->get_source()
+ << " " << *m << endl;
+ int from = m->get_source().num();
- // FIXME: is this necessary?
+ // note slave
+ mdr->slaves.insert(from);
- if (destdn->inode) {
- if (destdn->inode->is_dir()) {
- dout(7) << "handle_client_rename_local failing, dest exists and is a dir: " << *destdn->inode << endl;
- assert(0);
- reply_request(req, -EINVAL);
- return;
- }
- if (srcdn->inode->is_dir()) {
- dout(7) << "handle_client_rename_local failing, dest exists and src is a dir: " << *destdn->inode << endl;
- assert(0);
- reply_request(req, -EINVAL);
- return;
- }
- } else {
- // if destdn->inode is null, then we know it's a non-existent dest,
- // why? because if it's local, it dne. and if it's remote, we xlocked with
- // REQXLOCKC, which will only allow you to lock a file.
- // so we know dest is a file, or non-existent
- if (!destlocal) {
- if (srcdn->inode->is_dir()) {
- // help: maybe the dest exists and is a file? ..... FIXME
- } else {
- // we're fine, src is file, dest is file|dne
- }
- }
+ // witnessed!
+ assert(mdr->witnessed.count(from) == 0);
+ mdr->witnessed.insert(from);
+
+
+ // add extra witnesses?
+ if (!m->srcdn_replicas.empty()) {
+ dout(10) << " extra witnesses (srcdn replicas) are " << m->srcdn_replicas << endl;
+ mdr->extra_witnesses = m->srcdn_replicas;
+ mdr->extra_witnesses.erase(mds->get_nodeid()); // not me!
}
-
- mds->balancer->hit_dir(srcdn->dir, META_POP_DWR);
- mds->balancer->hit_dir(destdn->dir, META_POP_DWR);
- // we're golden.
- // everything is xlocked by us, we rule, etc.
- MClientReply *reply = new MClientReply(req, 0);
- mdcache->renamer->file_rename( srcdn, destdn,
- new C_MDS_CommitRequest(this, req, reply, srcdn->inode,
- new EString("file rename fixme")) );
+ // remove from waiting list
+ assert(mdr->waiting_on_slave.count(from));
+ mdr->waiting_on_slave.erase(from);
+
+ if (mdr->waiting_on_slave.empty())
+ dispatch_client_request(mdr); // go again!
+ else
+ dout(10) << "still waiting on slaves " << mdr->waiting_on_slave << endl;
}
-*/
+void Server::handle_slave_rename_get_inode(MDRequest *mdr)
+{
+ dout(10) << "handle_slave_rename_get_inode " << *mdr << endl;
+
+ assert(mdr->srcdn);
+ assert(mdr->srcdn->is_auth());
+ assert(mdr->srcdn->is_primary());
+
+ // reply
+ MMDSSlaveRequest *reply = new MMDSSlaveRequest(mdr->reqid, MMDSSlaveRequest::OP_RENAMEGETINODEACK);
+ dout(10) << " replying with inode export info " << *mdr->srcdn->inode << endl;
+
+ map<int,entity_inst_t> exported_client_map;
+ bufferlist inodebl;
+ mdcache->migrator->encode_export_inode(mdr->srcdn->inode, inodebl, mdr->slave_to_mds,
+ exported_client_map);
+ ::_encode(exported_client_map, reply->inode_export);
+ reply->inode_export.claim_append(inodebl);
+
+ reply->inode_export_v = mdr->srcdn->inode->inode.version;
+
+ mdr->inode_import = reply->inode_export; // keep a copy locally, in case we have to rollback
+
+ mds->send_message_mds(reply, mdr->slave_to_mds, MDS_PORT_SERVER);
+ // clean up.
+ delete mdr->slave_request;
+ mdr->slave_request = 0;
+}
+
+void Server::handle_slave_rename_get_inode_ack(MDRequest *mdr, MMDSSlaveRequest *m)
+{
+ dout(10) << "handle_slave_rename_get_inode_ack " << *mdr
+ << " " << *m << endl;
+ int from = m->get_source().num();
+
+ assert(m->inode_export.length());
+ dout(10) << " got inode export, saving in " << *mdr << endl;
+ mdr->inode_import.claim(m->inode_export);
+ mdr->inode_import_v = m->inode_export_v;
+
+ assert(mdr->waiting_on_slave.count(from));
+ mdr->waiting_on_slave.erase(from);
+
+ if (mdr->waiting_on_slave.empty())
+ dispatch_client_request(mdr); // go again!
+ else
+ dout(10) << "still waiting on slaves " << mdr->waiting_on_slave << endl;
+}
void Server::handle_client_truncate(MDRequest *mdr)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
CInode *cur = rdlock_path_pin_ref(mdr, true);
if (!cur) return;
// check permissions?
// xlock inode
- if (!mds->locker->xlock_start(&cur->filelock, mdr))
- return; // fw or (wait for) lock
+ set<SimpleLock*> rdlocks = mdr->rdlocks;
+ set<SimpleLock*> wrlocks = mdr->wrlocks;
+ set<SimpleLock*> xlocks = mdr->xlocks;
+ xlocks.insert(&cur->filelock);
+ if (!mds->locker->acquire_locks(mdr, rdlocks, wrlocks, xlocks))
+ return;
// already small enough?
if (cur->inode.size >= req->args.truncate.length) {
void Server::handle_client_open(MDRequest *mdr)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
int flags = req->args.open.flags;
int cmode = req->get_open_file_mode();
assert(cur->is_auth());
// xlock file size
- if (!mds->locker->xlock_start(&cur->filelock, mdr))
+ set<SimpleLock*> rdlocks = mdr->rdlocks;
+ set<SimpleLock*> wrlocks = mdr->wrlocks;
+ set<SimpleLock*> xlocks = mdr->xlocks;
+ xlocks.insert(&cur->filelock);
+ if (!mds->locker->acquire_locks(mdr, rdlocks, wrlocks, xlocks))
return;
-
+
if (cur->inode.size > 0) {
handle_client_opent(mdr);
return;
void Server::_do_open(MDRequest *mdr, CInode *cur)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
int cmode = req->get_open_file_mode();
// can we issue the caps they want?
void Server::handle_client_openc(MDRequest *mdr)
{
- MClientRequest *req = mdr->client_request();
+ MClientRequest *req = mdr->client_request;
dout(7) << "open w/ O_CREAT on " << req->get_filepath() << endl;
// created null dn.
// create inode.
- CInode *in = prepare_new_inode(req, dn->dir);
+ mdr->now = g_clock.real_now();
+ CInode *in = prepare_new_inode(mdr, dn->dir);
assert(in);
// it's a file.
dn->pre_dirty();
in->inode.mode = req->args.open.mode;
in->inode.mode |= INODE_MODE_FILE;
+ in->inode.version = dn->get_projected_version();
// prepare finisher
C_MDS_openc_finish *fin = new C_MDS_openc_finish(mds, mdr, dn, in);
EUpdate *le = new EUpdate("openc");
le->metablob.add_client_req(req->get_reqid());
+ le->metablob.add_allocated_ino(in->ino(), mds->idalloc->get_version());
le->metablob.add_dir_context(dn->dir);
- inode_t *pi = le->metablob.add_primary_dentry(dn, true, in);
- pi->version = dn->get_projected_version();
+ le->metablob.add_primary_dentry(dn, true, in, &in->inode);
// log + wait
mdlog->submit_entry(le);
#include "MDS.h"
class LogEvent;
-class C_MDS_rename_local_finish;
+class C_MDS_rename_finish;
class MDRequest;
+class EMetaBlob;
+class PVList;
+class MMDSSlaveRequest;
class Server {
MDS *mds;
// -- sessions and recovery --
utime_t reconnect_start;
set<int> client_reconnect_gather; // clients i need a reconnect msg from.
- set<CInode*> reconnected_open_files;
-
+ set<CInode*> reconnected_caps;
+
void handle_client_session(class MClientSession *m);
void _session_logged(entity_inst_t ci, bool open, version_t cmapv);
+ void terminate_sessions();
void reconnect_clients();
void handle_client_reconnect(class MClientReconnect *m);
+ void process_reconnect_cap(CInode *in, int from, inode_caps_reconnect_t& capinfo);
+ void add_reconnected_cap_inode(CInode *in) {
+ reconnected_caps.insert(in);
+ }
+ void process_reconnected_caps();
void client_reconnect_failure(int from);
- void reconnect_finish();
- void terminate_sessions();
+ void reconnect_gather_finish();
+
// -- requests --
void handle_client_request(MClientRequest *m);
- void dispatch_request(MDRequest *mdr);
+ void dispatch_client_request(MDRequest *mdr);
void reply_request(MDRequest *mdr, int r = 0, CInode *tracei = 0);
void reply_request(MDRequest *mdr, MClientReply *reply, CInode *tracei);
+ void handle_slave_request(MMDSSlaveRequest *m);
+ void dispatch_slave_request(MDRequest *mdr);
+ void handle_slave_auth_pin(MDRequest *mdr);
+ void handle_slave_auth_pin_ack(MDRequest *mdr, MMDSSlaveRequest *ack);
+
// some helpers
CDir *validate_dentry_dir(MDRequest *mdr, CInode *diri, const string& dname);
CDir *traverse_to_auth_dir(MDRequest *mdr, vector<CDentry*> &trace, filepath refpath);
CDentry *prepare_null_dentry(MDRequest *mdr, CDir *dir, const string& dname, bool okexist=false);
- CInode* prepare_new_inode(MClientRequest *req, CDir *dir);
+ CInode* prepare_new_inode(MDRequest *mdr, CDir *dir);
CInode* rdlock_path_pin_ref(MDRequest *mdr, bool want_auth);
CDentry* rdlock_path_xlock_dentry(MDRequest *mdr, bool okexist, bool mustexist);
CDir* try_open_auth_dir(CInode *diri, frag_t fg, MDRequest *mdr);
//CDir* try_open_dir(CInode *diri, frag_t fg, MDRequest *mdr);
- version_t predirty_dn_diri(CDentry *dn, class EMetaBlob *blob, utime_t mtime);
+ version_t predirty_dn_diri(MDRequest *mdr, CDentry *dn, class EMetaBlob *blob);
void dirty_dn_diri(CDentry *dn, version_t dirpv, utime_t mtime);
+
// requests on existing inodes.
void handle_client_stat(MDRequest *mdr);
void handle_client_utime(MDRequest *mdr);
void _link_local(MDRequest *mdr, CDentry *dn, CInode *targeti);
void _link_local_finish(MDRequest *mdr,
CDentry *dn, CInode *targeti,
- version_t, utime_t, version_t, version_t);
+ version_t, version_t, version_t);
+
void _link_remote(MDRequest *mdr, CDentry *dn, CInode *targeti);
+ void _link_remote_finish(MDRequest *mdr, CDentry *dn, CInode *targeti,
+ version_t, version_t);
+
+ void handle_slave_link_prep(MDRequest *mdr);
+ void _logged_slave_link(MDRequest *mdr, CInode *targeti, utime_t old_ctime, bool inc);
+ void _commit_slave_link(MDRequest *mdr, int r, CInode *targeti,
+ utime_t old_ctime, version_t old_version, bool inc);
+ void handle_slave_link_prep_ack(MDRequest *mdr, MMDSSlaveRequest *m);
// unlink
void handle_client_unlink(MDRequest *mdr);
bool _verify_rmdir(MDRequest *mdr, CInode *rmdiri);
- void _unlink_local(MDRequest *mdr, CDentry *dn);
+ void _unlink_local(MDRequest *mdr, CDentry *dn, CDentry *straydn);
void _unlink_local_finish(MDRequest *mdr,
CDentry *dn, CDentry *straydn,
- version_t, utime_t, version_t, version_t);
+ version_t, version_t);
+
void _unlink_remote(MDRequest *mdr, CDentry *dn);
+ void _unlink_remote_finish(MDRequest *mdr,
+ CDentry *dn,
+ version_t, version_t);
// rename
- bool _rename_open_dn(CDir *dir, CDentry *dn, bool mustexist, MDRequest *mdr);
void handle_client_rename(MDRequest *mdr);
- void handle_client_rename_2(MDRequest *mdr,
- CDentry *srcdn,
- filepath& destpath,
- vector<CDentry*>& trace,
- int r);
- void _rename_local(MDRequest *mdr, CDentry *srcdn, CDentry *destdn);
- void _rename_local_reanchored(LogEvent *le, C_MDS_rename_local_finish *fin,
- version_t atid1, version_t atid2);
- void _rename_local_finish(MDRequest *mdr,
- CDentry *srcdn, CDentry *destdn, CDentry *straydn,
- version_t srcpv, version_t destpv, version_t straypv, version_t ipv,
- version_t ddirpv, version_t sdirpv, utime_t ictime,
- version_t atid1, version_t atid2);
+ void _rename_finish(MDRequest *mdr,
+ CDentry *srcdn, CDentry *destdn, CDentry *straydn);
+
+ // helpers
+ void _rename_prepare(MDRequest *mdr,
+ EMetaBlob *metablob,
+ CDentry *srcdn, CDentry *destdn, CDentry *straydn);
+ void _rename_apply(MDRequest *mdr, CDentry *srcdn, CDentry *destdn, CDentry *straydn);
+
+ // slaving
+ void handle_slave_rename_prep(MDRequest *mdr);
+ void handle_slave_rename_prep_ack(MDRequest *mdr, MMDSSlaveRequest *m);
+ void _logged_slave_rename(MDRequest *mdr, CDentry *srcdn, CDentry *destdn, CDentry *straydn);
+ void _commit_slave_rename(MDRequest *mdr, int r, CDentry *srcdn, CDentry *destdn, CDentry *straydn);
+ void handle_slave_rename_get_inode(MDRequest *mdr);
+ void handle_slave_rename_get_inode_ack(MDRequest *mdr, MMDSSlaveRequest *m);
+
};
// NOTE: this also defines the lock ordering!
#define LOCK_OTYPE_DN 1
-#define LOCK_OTYPE_IFILE 2
-#define LOCK_OTYPE_IAUTH 3
-#define LOCK_OTYPE_ILINK 4
-#define LOCK_OTYPE_IDIRFRAGTREE 5
-#define LOCK_OTYPE_IDIR 6
+#define LOCK_OTYPE_IVERSION 2
+#define LOCK_OTYPE_IFILE 3
+#define LOCK_OTYPE_IAUTH 4
+#define LOCK_OTYPE_ILINK 5
+#define LOCK_OTYPE_IDIRFRAGTREE 6
+#define LOCK_OTYPE_IDIR 7
//#define LOCK_OTYPE_DIR 7 // not used
inline const char *get_lock_type_name(int t) {
switch (t) {
case LOCK_OTYPE_DN: return "dn";
+ case LOCK_OTYPE_IVERSION: return "iversion";
case LOCK_OTYPE_IFILE: return "ifile";
case LOCK_OTYPE_IAUTH: return "iauth";
case LOCK_OTYPE_ILINK: return "ilink";
}
// -- lock states --
+// sync <-> lock
#define LOCK_UNDEF 0
// auth rep
#define LOCK_SYNC 1 // AR R . R .
public:
static const int WAIT_RD = (1<<0); // to read
static const int WAIT_WR = (1<<1); // to write
- static const int WAIT_SINGLEAUTH = (1<<2);
- static const int WAIT_STABLE = (1<<3); // for a stable state
- static const int WAIT_REMOTEXLOCK = (1<<4); // for a remote xlock
- static const int WAIT_BITS = 5;
+ static const int WAIT_XLOCK = (1<<2); // to xlock (** dup)
+ static const int WAIT_STABLE = (1<<2); // for a stable state
+ static const int WAIT_REMOTEXLOCK = (1<<3); // for a remote xlock
+ static const int WAIT_BITS = 4;
+ static const int WAIT_ALL = ((1<<WAIT_BITS)-1);
protected:
// parent (what i lock)
int wait_offset;
// lock state
- int state;
- set<__int32_t> gather_set; // auth
+ int state;
+ set<int32_t> gather_set; // auth
// local state
- int num_rdlock;
+ int num_rdlock;
MDRequest *xlock_by;
public:
struct ptr_lt {
bool operator()(const SimpleLock* l, const SimpleLock* r) const {
- if (l->type < r->type) return true;
- if (l->type == r->type) return l->parent->is_lt(r->parent);
+ // first sort by object type (dn < inode)
+ if ((l->type>LOCK_OTYPE_DN) < (r->type>LOCK_OTYPE_DN)) return true;
+ if ((l->type>LOCK_OTYPE_DN) == (r->type>LOCK_OTYPE_DN)) {
+ // then sort by object
+ if (l->parent->is_lt(r->parent)) return true;
+ if (l->parent == r->parent) {
+ // then sort by (inode) lock type
+ if (l->type < r->type) return true;
+ }
+ }
return false;
}
};
void finish_waiters(int mask, int r=0) {
parent->finish_waiting(mask << wait_offset, r);
}
+ void take_waiting(int mask, list<Context*>& ls) {
+ parent->take_waiting(mask << wait_offset, ls);
+ }
void add_waiter(int mask, Context *c) {
parent->add_waiter(mask << wait_offset, c);
}
// ref counting
bool is_rdlocked() { return num_rdlock > 0; }
- int get_rdlock() { return ++num_rdlock; }
+ int get_rdlock() {
+ if (!num_rdlock) parent->get(MDSCacheObject::PIN_LOCK);
+ return ++num_rdlock;
+ }
int put_rdlock() {
assert(num_rdlock>0);
- return --num_rdlock;
+ --num_rdlock;
+ if (num_rdlock == 0) parent->put(MDSCacheObject::PIN_LOCK);
+ return num_rdlock;
}
int get_num_rdlocks() { return num_rdlock; }
void get_xlock(MDRequest *who) {
assert(xlock_by == 0);
+ parent->get(MDSCacheObject::PIN_LOCK);
xlock_by = who;
}
void put_xlock() {
assert(xlock_by);
+ parent->put(MDSCacheObject::PIN_LOCK);
xlock_by = 0;
}
bool is_xlocked() { return xlock_by ? true:false; }
+ bool is_xlocked_by_other(MDRequest *mdr) {
+ return is_xlocked() && xlock_by != mdr;
+ }
MDRequest *get_xlocked_by() { return xlock_by; }
bool is_used() {
return is_xlocked() || is_rdlocked();
}
return 0;
}
+ void export_twiddle() {
+ clear_gather();
+ state = get_replica_state();
+ }
+
/** replicate_relax
* called on first replica creation.
*/
if (!is_gathering())
return true;
}
+ if (!is_stable() && !is_gathering())
+ return true;
return false;
}
bool can_rdlock(MDRequest *mdr) {
- if (state == LOCK_SYNC)
- return true;
- if (state == LOCK_LOCK && mdr && xlock_by == mdr)
- return true;
- return false;
+ //if (state == LOCK_LOCK && mdr && xlock_by == mdr) return true; // xlocked by me. (actually, is this right?)
+ //if (state == LOCK_LOCK && !xlock_by && parent->is_auth()) return true;
+ return (state == LOCK_SYNC);
}
bool can_xlock(MDRequest *mdr) {
- if (!parent->is_auth()) return false;
- if (state != LOCK_LOCK) return false;
- if (xlock_by == 0 ||
- (mdr && xlock_by == mdr)) return true;
+ if (mdr && xlock_by == mdr) {
+ assert(state == LOCK_LOCK);
+ return true; // auth or replica! xlocked by me.
+ }
+ if (state == LOCK_LOCK && parent->is_auth() && !xlock_by) return true;
return false;
}
bool can_xlock_soon() {
if (is_rdlocked())
out << " r=" << get_num_rdlocks();
if (is_xlocked())
- out << " w=" << get_xlocked_by();
+ out << " x=" << get_xlocked_by();
out << ")";
}
};
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-#ifndef __MDS_EALLOC_H
-#define __MDS_EALLOC_H
-
-#include <assert.h>
-#include "config.h"
-#include "include/types.h"
-
-#include "../LogEvent.h"
-#include "../IdAllocator.h"
-
-#define EALLOC_EV_ALLOC 1
-#define EALLOC_EV_FREE 2
-
-class EAlloc : public LogEvent {
- protected:
- int idtype;
- idno_t id;
- int what; // alloc or dealloc
- version_t table_version;
-
- public:
- EAlloc() : LogEvent(EVENT_ALLOC) { }
- EAlloc(int idtype, idno_t id, int what, version_t v) :
- LogEvent(EVENT_ALLOC) {
- this->idtype = idtype;
- this->id = id;
- this->what = what;
- this->table_version = v;
- }
-
- void encode_payload(bufferlist& bl) {
- bl.append((char*)&idtype, sizeof(idtype));
- bl.append((char*)&id, sizeof(id));
- bl.append((char*)&what, sizeof(what));
- bl.append((char*)&table_version, sizeof(table_version));
- }
- void decode_payload(bufferlist& bl, int& off) {
- bl.copy(off, sizeof(idtype), (char*)&idtype);
- off += sizeof(idtype);
- bl.copy(off, sizeof(id), (char*)&id);
- off += sizeof(id);
- bl.copy(off, sizeof(what), (char*)&what);
- off += sizeof(what);
- bl.copy(off, sizeof(table_version), (char*)&table_version);
- off += sizeof(table_version);
- }
-
-
- void print(ostream& out) {
- if (what == EALLOC_EV_ALLOC)
- out << "EAlloc alloc " << hex << id << dec << " tablev " << table_version;
- else
- out << "EAlloc dealloc " << hex << id << dec << " tablev " << table_version;
- }
-
-
- bool has_expired(MDS *mds);
- void expire(MDS *mds, Context *c);
- void replay(MDS *mds);
-
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-#ifndef __MDS_ECLIENTMAP_H
-#define __MDS_ECLIENTMAP_H
-
-#include <assert.h>
-#include "config.h"
-#include "include/types.h"
-
-#include "../LogEvent.h"
-
-class EClientMap : public LogEvent {
- protected:
- bufferlist mapbl;
- version_t cmapv; // client map version
-
- public:
- EClientMap() : LogEvent(EVENT_CLIENTMAP) { }
- EClientMap(bufferlist& bl, version_t v) :
- LogEvent(EVENT_CLIENTMAP),
- cmapv(v) {
- mapbl.claim(bl);
- }
-
- void encode_payload(bufferlist& bl) {
- bl.append((char*)&cmapv, sizeof(cmapv));
- ::_encode(mapbl, bl);
- }
- void decode_payload(bufferlist& bl, int& off) {
- bl.copy(off, sizeof(cmapv), (char*)&cmapv);
- off += sizeof(cmapv);
- ::_decode(mapbl, bl, off);
- }
-
-
- void print(ostream& out) {
- out << "EClientMap v " << cmapv;
- }
-
-
- bool has_expired(MDS *mds);
- void expire(MDS *mds, Context *c);
- void replay(MDS *mds);
-
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-#ifndef __MDS_EIMPORTMAP_H
-#define __MDS_EIMPORTMAP_H
-
-#include "../LogEvent.h"
-#include "EMetaBlob.h"
-
-class EImportMap : public LogEvent {
-public:
- EMetaBlob metablob;
- set<dirfrag_t> imports;
- map<dirfrag_t, set<dirfrag_t> > bounds;
-
- EImportMap() : LogEvent(EVENT_IMPORTMAP) { }
-
- void print(ostream& out) {
- out << "import_map " << imports.size() << " imports "
- << metablob;
- }
-
- void encode_payload(bufferlist& bl) {
- metablob._encode(bl);
- ::_encode(imports, bl);
- for (set<dirfrag_t>::iterator p = imports.begin();
- p != imports.end();
- ++p) {
- ::_encode(bounds[*p], bl);
- if (bounds[*p].empty())
- bounds.erase(*p);
- }
- }
- void decode_payload(bufferlist& bl, int& off) {
- metablob._decode(bl, off);
- ::_decode(imports, bl, off);
- for (set<dirfrag_t>::iterator p = imports.begin();
- p != imports.end();
- ++p) {
- ::_decode(bounds[*p], bl, off);
- if (bounds[*p].empty())
- bounds.erase(*p);
- }
- }
-
- bool has_expired(MDS *mds);
- void expire(MDS *mds, Context *c);
- void replay(MDS *mds);
-};
-
-#endif
bl.copy(off, sizeof(dirty), (char*)&dirty);
off += sizeof(dirty);
}
+ void print(ostream& out) {
+ out << " fullbit dn " << dn << " dnv " << dnv
+ << " inode " << inode.ino
+ << " dirty=" << dirty << endl;
+ }
};
/* remotebit - a dentry + remote inode link (i.e. just an ino)
bl.copy(off, sizeof(dirty), (char*)&dirty);
off += sizeof(dirty);
}
+ void print(ostream& out) {
+ out << " remotebit dn " << dn << " dnv " << dnv
+ << " ino " << ino
+ << " dirty=" << dirty << endl;
+ }
};
/*
bl.copy(off, sizeof(dirty), (char*)&dirty);
off += sizeof(dirty);
}
+ void print(ostream& out) {
+ out << " nullbit dn " << dn << " dnv " << dnv
+ << " dirty=" << dirty << endl;
+ }
};
static const int STATE_COMPLETE = (1<<1);
static const int STATE_DIRTY = (1<<2); // dirty due to THIS journal item, that is!
- dirfrag_t dirfrag;
version_t dirv;
int state;
int nfull, nremote, nnull;
- bufferlist bfull, bremote, bnull;
private:
+ bufferlist dnbl;
bool dn_decoded;
list<fullbit> dfull;
list<remotebit> dremote;
list<remotebit> &get_dremote() { return dremote; }
list<nullbit> &get_dnull() { return dnull; }
+ void print(dirfrag_t dirfrag, ostream& out) {
+ out << "dirlump " << dirfrag << " dirv " << dirv
+ << " state " << state
+ << " num " << nfull << "/" << nremote << "/" << nnull
+ << endl;
+ _decode_bits();
+ for (list<fullbit>::iterator p = dfull.begin(); p != dfull.end(); ++p)
+ p->print(out);
+ for (list<remotebit>::iterator p = dremote.begin(); p != dremote.end(); ++p)
+ p->print(out);
+ for (list<nullbit>::iterator p = dnull.begin(); p != dnull.end(); ++p)
+ p->print(out);
+ }
+
void _encode_bits() {
for (list<fullbit>::iterator p = dfull.begin(); p != dfull.end(); ++p)
- p->_encode(bfull);
+ p->_encode(dnbl);
for (list<remotebit>::iterator p = dremote.begin(); p != dremote.end(); ++p)
- p->_encode(bremote);
+ p->_encode(dnbl);
for (list<nullbit>::iterator p = dnull.begin(); p != dnull.end(); ++p)
- p->_encode(bnull);
+ p->_encode(dnbl);
}
void _decode_bits() {
if (dn_decoded) return;
int off = 0;
for (int i=0; i<nfull; i++)
- dfull.push_back(fullbit(bfull, off));
- off = 0;
+ dfull.push_back(fullbit(dnbl, off));
for (int i=0; i<nremote; i++)
- dremote.push_back(remotebit(bremote, off));
- off = 0;
+ dremote.push_back(remotebit(dnbl, off));
for (int i=0; i<nnull; i++)
- dnull.push_back(nullbit(bnull, off));
+ dnull.push_back(nullbit(dnbl, off));
dn_decoded = true;
}
void _encode(bufferlist& bl) {
- bl.append((char*)&dirfrag, sizeof(dirfrag));
- bl.append((char*)&dirv, sizeof(dirv));
- bl.append((char*)&state, sizeof(state));
- bl.append((char*)&nfull, sizeof(nfull));
- bl.append((char*)&nremote, sizeof(nremote));
- bl.append((char*)&nnull, sizeof(nnull));
+ ::_encode(dirv, bl);
+ ::_encode(state, bl);
+ ::_encode(nfull, bl);
+ ::_encode(nremote, bl);
+ ::_encode(nnull, bl);
_encode_bits();
- ::_encode(bfull, bl);
- ::_encode(bremote, bl);
- ::_encode(bnull, bl);
+ ::_encode(dnbl, bl);
}
void _decode(bufferlist& bl, int& off) {
- bl.copy(off, sizeof(dirfrag), (char*)&dirfrag); off += sizeof(dirfrag);
- bl.copy(off, sizeof(dirv), (char*)&dirv); off += sizeof(dirv);
- bl.copy(off, sizeof(state), (char*)&state); off += sizeof(state);
- bl.copy(off, sizeof(nfull), (char*)&nfull); off += sizeof(nfull);
- bl.copy(off, sizeof(nremote), (char*)&nremote); off += sizeof(nremote);
- bl.copy(off, sizeof(nnull), (char*)&nnull); off += sizeof(nnull);
- ::_decode(bfull, bl, off);
- ::_decode(bremote, bl, off);
- ::_decode(bnull, bl, off);
- // don't decode bits unless we need them.
- dn_decoded = false;
+ ::_decode(dirv, bl, off);
+ ::_decode(state, bl, off);
+ ::_decode(nfull, bl, off);
+ ::_decode(nremote, bl, off);
+ ::_decode(nnull, bl, off);
+ ::_decode(dnbl, bl, off);
+ dn_decoded = false; // don't decode bits unless we need them.
}
};
// anchor transactions included in this update.
list<version_t> atids;
+ // inode dirlocks (scatterlocks) i've touched.
+ map<inodeno_t, utime_t> dirty_inode_mtimes;
+
+ // ino's i've allocated
+ list<inodeno_t> allocated_inos;
+ version_t alloc_tablev;
+
// inodes i've destroyed.
list< pair<inode_t,off_t> > truncated_inodes;
list<metareqid_t> client_reqs;
public:
+ void print(ostream& out) {
+ for (list<dirfrag_t>::iterator p = lump_order.begin();
+ p != lump_order.end();
+ ++p) {
+ lump_map[*p].print(*p, out);
+ }
+ }
void add_client_req(metareqid_t r) {
client_reqs.push_back(r);
atids.push_back(atid);
}
+ void add_dirtied_inode_mtime(inodeno_t ino, utime_t ctime) {
+ dirty_inode_mtimes[ino] = ctime;
+ }
+
+ void add_allocated_ino(inodeno_t ino, version_t tablev) {
+ allocated_inos.push_back(ino);
+ alloc_tablev = tablev;
+ }
+
void add_inode_truncate(const inode_t& inode, off_t newsize) {
truncated_inodes.push_back(pair<inode_t,off_t>(inode, newsize));
}
}
// return remote pointer to to-be-journaled inode
- inode_t *add_primary_dentry(CDentry *dn, bool dirty, CInode *in=0) {
+ inode_t *add_primary_dentry(CDentry *dn, bool dirty, CInode *in=0, inode_t *pi=0, fragtree_t *pdft=0) {
if (!in) in = dn->get_inode();
dirlump& lump = add_dir(dn->get_dir(), false);
dn->get_projected_version(),
in->inode, in->symlink,
dirty));
+ if (pi) lump.get_dfull().front().inode = *pi;
return &lump.get_dfull().front().inode;
} else {
lump.get_dfull().push_back(fullbit(dn->get_name(),
dn->get_projected_version(),
in->inode, in->symlink,
dirty));
+ if (pi) lump.get_dfull().back().inode = *pi;
return &lump.get_dfull().back().inode;
}
}
}
- dirlump& add_dir(CDir *dir, bool dirty) {
+ dirlump& add_dir(CDir *dir, bool dirty, bool complete=false) {
dirfrag_t df = dir->dirfrag();
if (lump_map.count(df) == 0) {
lump_order.push_back(df);
lump_map[df].dirv = dir->get_projected_version();
}
dirlump& l = lump_map[df];
- if (dir->is_complete()) l.mark_complete();
+ if (complete) l.mark_complete();
if (dirty) l.mark_dirty();
return l;
}
-
- void add_dir_context(CDir *dir, bool toroot=false) {
+
+ static const int TO_AUTH_SUBTREE_ROOT = 0; // default.
+ static const int TO_ROOT = 1;
+
+ void add_dir_context(CDir *dir, int mode = TO_AUTH_SUBTREE_ROOT) {
// already have this dir? (we must always add in order)
if (lump_map.count(dir->dirfrag()))
return;
+ // stop at subtree root?
+ if (mode == TO_AUTH_SUBTREE_ROOT &&
+ dir->is_subtree_root() && dir->is_auth())
+ return;
+
+ // stop at root/stray
CInode *diri = dir->get_inode();
- if (!toroot && dir->is_subtree_root() && dir->is_auth())
- return; // stop at subtree root
- if (!dir->get_inode()->get_parent_dn())
+ if (!diri->get_parent_dn())
return;
+ // add parent dn
CDentry *parent = diri->get_parent_dn();
- add_dir_context(parent->get_dir(), toroot);
+ add_dir_context(parent->get_dir(), mode);
add_dentry(parent, false);
}
// encoding
void _encode(bufferlist& bl) {
- int n = lump_map.size();
- bl.append((char*)&n, sizeof(n));
+ int32_t n = lump_map.size();
+ ::_encode(n, bl);
for (list<dirfrag_t>::iterator i = lump_order.begin();
i != lump_order.end();
++i) {
- bl.append((char*)&(*i), sizeof(*i));
+ dirfrag_t dirfrag = *i;
+ ::_encode(dirfrag, bl);
lump_map[*i]._encode(bl);
}
::_encode(atids, bl);
+ ::_encode(dirty_inode_mtimes, bl);
+ ::_encode(allocated_inos, bl);
+ if (!allocated_inos.empty())
+ ::_encode(alloc_tablev, bl);
::_encode(truncated_inodes, bl);
::_encode(client_reqs, bl);
}
void _decode(bufferlist& bl, int& off) {
- int n;
- bl.copy(off, sizeof(n), (char*)&n);
- off += sizeof(n);
+ int32_t n;
+ ::_decode(n, bl, off);
for (int i=0; i<n; i++) {
- dirfrag_t dirfrag;
- bl.copy(off, sizeof(dirfrag), (char*)&dirfrag);
- off += sizeof(dirfrag);
+ dirfrag_t dirfrag;
+ ::_decode(dirfrag, bl, off);
lump_order.push_back(dirfrag);
lump_map[dirfrag]._decode(bl, off);
}
::_decode(atids, bl, off);
+ ::_decode(dirty_inode_mtimes, bl, off);
+ ::_decode(allocated_inos, bl, off);
+ if (!allocated_inos.empty())
+ ::_decode(alloc_tablev, bl, off);
::_decode(truncated_inodes, bl, off);
::_decode(client_reqs, bl, off);
}
void print(ostream& out) const {
out << "[metablob";
if (!lump_order.empty())
- out << lump_order.front() << ", " << lump_map.size() << " dirs";
+ out << " " << lump_order.front() << ", " << lump_map.size() << " dirs";
if (!atids.empty())
- out << " atids " << atids;
+ out << " atids=" << atids;
+ if (!allocated_inos.empty())
+ out << " inos=" << allocated_inos << " v" << alloc_tablev;
out << "]";
}
class ESlaveUpdate : public LogEvent {
public:
+ const static int OP_PREPARE = 1;
+ const static int OP_COMMIT = 2;
+ const static int OP_ROLLBACK = 3;
+
string type;
metareqid_t reqid;
+ int master;
int op; // prepare, commit, abort
EMetaBlob metablob;
ESlaveUpdate() : LogEvent(EVENT_SLAVEUPDATE) { }
- ESlaveUpdate(const char *s, metareqid_t ri, int o) :
+ ESlaveUpdate(const char *s, metareqid_t ri, int mastermds, int o) :
LogEvent(EVENT_SLAVEUPDATE),
type(s),
reqid(ri),
+ master(mastermds),
op(o) { }
void print(ostream& out) {
out << type << " ";
out << " " << op;
out << " " << reqid;
+ out << " for mds" << master;
out << metablob;
}
void encode_payload(bufferlist& bl) {
::_encode(type, bl);
::_encode(reqid, bl);
+ ::_encode(master, bl);
::_encode(op, bl);
metablob._encode(bl);
}
void decode_payload(bufferlist& bl, int& off) {
::_decode(type, bl, off);
::_decode(reqid, bl, off);
+ ::_decode(master, bl, off);
::_decode(op, bl, off);
metablob._decode(bl, off);
}
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation. See file COPYING.
+ *
+ */
+
+#ifndef __MDS_ESUBTREEMAP_H
+#define __MDS_ESUBTREEMAP_H
+
+#include "../LogEvent.h"
+#include "EMetaBlob.h"
+
+class ESubtreeMap : public LogEvent {
+public:
+ EMetaBlob metablob;
+ map<dirfrag_t, list<dirfrag_t> > subtrees;
+
+ ESubtreeMap() : LogEvent(EVENT_SUBTREEMAP) { }
+
+ void print(ostream& out) {
+ out << "subtree_map " << subtrees.size() << " subtrees "
+ << metablob;
+ }
+
+ void encode_payload(bufferlist& bl) {
+ metablob._encode(bl);
+ ::_encode(subtrees, bl);
+ }
+ void decode_payload(bufferlist& bl, int& off) {
+ metablob._decode(bl, off);
+ ::_decode(subtrees, bl, off);
+ }
+
+ bool has_expired(MDS *mds);
+ void expire(MDS *mds, Context *c);
+ void replay(MDS *mds);
+};
+
+#endif
*/
#include "events/EString.h"
-#include "events/EImportMap.h"
+#include "events/ESubtreeMap.h"
#include "events/ESession.h"
-#include "events/EClientMap.h"
#include "events/EMetaBlob.h"
#include "events/ESlaveUpdate.h"
#include "events/EOpen.h"
-#include "events/EAlloc.h"
#include "events/EPurgeFinish.h"
#include "events/EExport.h"
#include "Migrator.h"
#include "AnchorTable.h"
#include "AnchorClient.h"
+#include "IdAllocator.h"
#include "config.h"
#undef dout
return false;
}
}
+
+ if (!dirty_inode_mtimes.empty())
+ for (map<inodeno_t,utime_t>::iterator p = dirty_inode_mtimes.begin();
+ p != dirty_inode_mtimes.end();
+ ++p) {
+ CInode *in = mds->mdcache->get_inode(p->first);
+ if (in) {
+ if (in->inode.ctime == p->second &&
+ in->dirlock.is_updated()) {
+ dout(10) << "EMetaBlob.has_expired dirty mtime dirlock hasn't flushed on " << *in << endl;
+ return false;
+ }
+ }
+ }
+
+ // allocated_ios
+ if (!allocated_inos.empty()) {
+ version_t cv = mds->idalloc->get_committed_version();
+ if (cv < alloc_tablev) {
+ dout(10) << "EMetaBlob.has_expired idalloc tablev " << alloc_tablev << " > " << cv
+ << ", still dirty" << endl;
+ return false; // still dirty
+ } else {
+ dout(10) << "EMetaBlob.has_expired idalloc tablev " << alloc_tablev << " <= " << cv
+ << ", already flushed" << endl;
+ }
+ }
+
// truncated inodes
for (list< pair<inode_t,off_t> >::iterator p = truncated_inodes.begin();
}
}
+ // dirtied inode mtimes
+ if (!dirty_inode_mtimes.empty())
+ for (map<inodeno_t,utime_t>::iterator p = dirty_inode_mtimes.begin();
+ p != dirty_inode_mtimes.end();
+ ++p) {
+ CInode *in = mds->mdcache->get_inode(p->first);
+ if (in) {
+ if (in->inode.ctime == p->second &&
+ in->dirlock.is_updated()) {
+ dout(10) << "EMetaBlob.expire dirty mtime dirlock hasn't flushed, waiting on "
+ << *in << endl;
+ in->dirlock.add_waiter(SimpleLock::WAIT_STABLE, gather->new_sub());
+ }
+ }
+ }
+
+ // allocated_inos
+ if (!allocated_inos.empty()) {
+ version_t cv = mds->idalloc->get_committed_version();
+ if (cv < alloc_tablev) {
+ dout(10) << "EMetaBlob.expire saving idalloc table, need " << alloc_tablev << endl;
+ mds->idalloc->save(gather->new_sub(), alloc_tablev);
+ }
+ }
+
// truncated inodes
for (list< pair<inode_t,off_t> >::iterator p = truncated_inodes.begin();
p != truncated_inodes.end();
}
// create the dirfrag
dir = diri->get_or_open_dirfrag(mds->mdcache, (*lp).frag);
- if ((*lp).ino == 1)
- dir->set_dir_auth(CDIR_AUTH_UNKNOWN); // FIXME: can root dir be fragmented? hrm.
+
+ if ((*lp).ino < MDS_INO_BASE)
+ mds->mdcache->adjust_subtree_auth(dir, CDIR_AUTH_UNKNOWN);
+
dout(10) << "EMetaBlob.replay added dir " << *dir << endl;
}
dir->set_version( lump.dirv );
mds->anchorclient->got_journaled_agree(*p);
}
+ // dirtied inode mtimes
+ if (!dirty_inode_mtimes.empty())
+ for (map<inodeno_t,utime_t>::iterator p = dirty_inode_mtimes.begin();
+ p != dirty_inode_mtimes.end();
+ ++p) {
+ CInode *in = mds->mdcache->get_inode(p->first);
+ dout(10) << "EMetaBlob.replay setting dirlock updated flag on " << *in << endl;
+ in->dirlock.set_updated();
+ }
+
+ // allocated_inos
+ if (!allocated_inos.empty()) {
+ if (mds->idalloc->get_version() >= alloc_tablev) {
+ dout(10) << "EMetaBlob.replay idalloc tablev " << alloc_tablev
+ << " <= table " << mds->idalloc->get_version() << endl;
+ } else {
+ for (list<inodeno_t>::iterator p = allocated_inos.begin();
+ p != allocated_inos.end();
+ ++p) {
+ dout(10) << " EMetaBlob.replay idalloc " << *p << " tablev " << alloc_tablev
+ << " - 1 == table " << mds->idalloc->get_version() << endl;
+ assert(alloc_tablev-1 == mds->idalloc->get_version());
+
+ inodeno_t ino = mds->idalloc->alloc_id();
+ assert(ino == *p); // this should match.
+
+ assert(alloc_tablev == mds->idalloc->get_version());
+ }
+ }
+ }
+
// truncated inodes
for (list< pair<inode_t,off_t> >::iterator p = truncated_inodes.begin();
p != truncated_inodes.end();
}
// -----------------------
-// EClientMap
-
-bool EClientMap::has_expired(MDS *mds)
-{
- if (mds->clientmap.get_committed() >= cmapv) {
- dout(10) << "EClientMap.has_expired newer clientmap " << mds->clientmap.get_committed()
- << " >= " << cmapv << " has committed" << endl;
- return true;
- } else if (mds->clientmap.get_committing() >= cmapv) {
- dout(10) << "EClientMap.has_expired newer clientmap " << mds->clientmap.get_committing()
- << " >= " << cmapv << " is still committing" << endl;
- return false;
- } else {
- dout(10) << "EClientMap.has_expired clientmap " << mds->clientmap.get_version()
- << " not empty" << endl;
- return false;
- }
-}
-
-void EClientMap::expire(MDS *mds, Context *c)
-{
- if (mds->clientmap.get_committing() >= cmapv) {
- dout(10) << "EClientMap.expire logging clientmap" << endl;
- assert(mds->clientmap.get_committing() > mds->clientmap.get_committed());
- mds->clientmap.add_commit_waiter(c);
- } else {
- dout(10) << "EClientMap.expire logging clientmap" << endl;
- mds->log_clientmap(c);
- }
-}
-
-void EClientMap::replay(MDS *mds)
-{
- dout(10) << "EClientMap.replay v " << cmapv << endl;
- int off = 0;
- mds->clientmap.decode(mapbl, off);
- mds->clientmap.set_committed(mds->clientmap.get_version());
- mds->clientmap.set_committing(mds->clientmap.get_version());
-}
-
-
// ESession
bool ESession::has_expired(MDS *mds)
{
return false;
} else {
dout(10) << "ESession.has_expired clientmap " << mds->clientmap.get_version()
- << " not empty" << endl;
+ << " > " << cmapv << ", need to save" << endl;
return false;
}
}
void ESession::expire(MDS *mds, Context *c)
-{
- if (mds->clientmap.get_committing() >= cmapv) {
- dout(10) << "ESession.expire logging clientmap" << endl;
- assert(mds->clientmap.get_committing() > mds->clientmap.get_committed());
- mds->clientmap.add_commit_waiter(c);
- } else {
- dout(10) << "ESession.expire logging clientmap" << endl;
- mds->log_clientmap(c);
- }
+{
+ dout(10) << "ESession.expire saving clientmap" << endl;
+ mds->clientmap.save(c, cmapv);
}
void ESession::replay(MDS *mds)
{
- dout(10) << "ESession.replay" << endl;
- if (open)
- mds->clientmap.open_session(client_inst);
- else
- mds->clientmap.close_session(client_inst.name.num());
- mds->clientmap.reset_projected(); // make it follow version.
-}
+ if (mds->clientmap.get_version() >= cmapv) {
+ dout(10) << "ESession.replay clientmap " << mds->clientmap.get_version()
+ << " >= " << cmapv << ", noop" << endl;
+ // hrm, this isn't very pretty.
+ if (!open)
+ mds->clientmap.trim_completed_requests(client_inst.name.num(), 0);
-
-// -----------------------
-// EAlloc
-
-bool EAlloc::has_expired(MDS *mds)
-{
- version_t cv = mds->idalloc->get_committed_version();
- if (cv < table_version) {
- dout(10) << "EAlloc.has_expired v " << table_version << " > " << cv
- << ", still dirty" << endl;
- return false; // still dirty
} else {
- dout(10) << "EAlloc.has_expired v " << table_version << " <= " << cv
- << ", already flushed" << endl;
- return true; // already flushed
+ dout(10) << "ESession.replay clientmap " << mds->clientmap.get_version()
+ << " < " << cmapv << endl;
+ assert(mds->clientmap.get_version() + 1 == cmapv);
+ if (open) {
+ mds->clientmap.open_session(client_inst);
+ } else {
+ mds->clientmap.close_session(client_inst.name.num());
+ mds->clientmap.trim_completed_requests(client_inst.name.num(), 0);
+ }
+ mds->clientmap.reset_projected(); // make it follow version.
}
}
-void EAlloc::expire(MDS *mds, Context *c)
-{
- dout(10) << "EAlloc.expire saving idalloc table" << endl;
- mds->idalloc->save(c, table_version);
-}
-
-void EAlloc::replay(MDS *mds)
-{
- if (mds->idalloc->get_version() >= table_version) {
- dout(10) << "EAlloc.replay event " << table_version
- << " <= table " << mds->idalloc->get_version() << endl;
- } else {
- dout(10) << " EAlloc.replay event " << table_version
- << " - 1 == table " << mds->idalloc->get_version() << endl;
- assert(table_version-1 == mds->idalloc->get_version());
-
- if (what == EALLOC_EV_ALLOC) {
- idno_t nid = mds->idalloc->alloc_id(true);
- assert(nid == id); // this should match.
- }
- else if (what == EALLOC_EV_FREE) {
- mds->idalloc->reclaim_id(id, true);
- }
- else
- assert(0);
-
- assert(table_version == mds->idalloc->get_version());
- }
-}
// -----------------------
bool ESlaveUpdate::has_expired(MDS *mds)
{
- return true;
- //return metablob.has_expired(mds);
+ switch (op) {
+ case ESlaveUpdate::OP_PREPARE:
+ if (mds->mdcache->ambiguous_slave_updates.count(reqid) == 0) {
+ dout(10) << "ESlaveUpdate.has_expired prepare " << reqid << " for mds" << master
+ << ": haven't yet seen commit|rollback" << endl;
+ return false;
+ }
+ else if (mds->mdcache->ambiguous_slave_updates[reqid]) {
+ dout(10) << "ESlaveUpdate.has_expired prepare " << reqid << " for mds" << master
+ << ": committed, checking metablob" << endl;
+ bool exp = metablob.has_expired(mds);
+ if (exp)
+ mds->mdcache->ambiguous_slave_updates.erase(reqid);
+ return exp;
+ }
+ else {
+ dout(10) << "ESlaveUpdate.has_expired prepare " << reqid << " for mds" << master
+ << ": aborted" << endl;
+ mds->mdcache->ambiguous_slave_updates.erase(reqid);
+ return true;
+ }
+
+ case ESlaveUpdate::OP_COMMIT:
+ case ESlaveUpdate::OP_ROLLBACK:
+ if (mds->mdcache->waiting_for_slave_update_commit.count(reqid)) {
+ dout(10) << "ESlaveUpdate.has_expired "
+ << ((op == ESlaveUpdate::OP_COMMIT) ? "commit ":"rollback ")
+ << reqid << " for mds" << master
+ << ": noting commit, kicking prepare waiter" << endl;
+ mds->mdcache->ambiguous_slave_updates[reqid] = (op == ESlaveUpdate::OP_COMMIT);
+ mds->mdcache->waiting_for_slave_update_commit[reqid]->finish(0);
+ delete mds->mdcache->waiting_for_slave_update_commit[reqid];
+ mds->mdcache->waiting_for_slave_update_commit.erase(reqid);
+ } else {
+ dout(10) << "ESlaveUpdate.has_expired "
+ << ((op == ESlaveUpdate::OP_COMMIT) ? "commit ":"rollback ")
+ << reqid << " for mds" << master
+ << ": no prepare waiter, ignoring" << endl;
+ }
+ return true;
+
+ default:
+ assert(0);
+ }
}
void ESlaveUpdate::expire(MDS *mds, Context *c)
{
- metablob.expire(mds, c);
+ assert(op == ESlaveUpdate::OP_PREPARE);
+
+ if (mds->mdcache->ambiguous_slave_updates.count(reqid) == 0) {
+ // wait
+ dout(10) << "ESlaveUpdate.expire prepare " << reqid << " for mds" << master
+ << ": waiting for commit|rollback" << endl;
+ mds->mdcache->waiting_for_slave_update_commit[reqid] = c;
+ } else {
+ // we committed.. expire the metablob
+ assert(mds->mdcache->ambiguous_slave_updates[reqid] == true);
+ dout(10) << "ESlaveUpdate.expire prepare " << reqid << " for mds" << master
+ << ": waiting for metablob to expire" << endl;
+ metablob.expire(mds, c);
+ }
}
void ESlaveUpdate::replay(MDS *mds)
{
- //metablob.replay(mds);
+ switch (op) {
+ case ESlaveUpdate::OP_PREPARE:
+ // FIXME: horribly inefficient copy; EMetaBlob needs a swap() or something
+ dout(10) << "ESlaveUpdate.replay prepare " << reqid << " for mds" << master
+ << ": saving blob for later commit" << endl;
+ assert(mds->mdcache->uncommitted_slave_updates[master].count(reqid) == 0);
+ mds->mdcache->uncommitted_slave_updates[master][reqid] = metablob;
+ break;
+
+ case ESlaveUpdate::OP_COMMIT:
+ if (mds->mdcache->uncommitted_slave_updates[master].count(reqid)) {
+ dout(10) << "ESlaveUpdate.replay commit " << reqid << " for mds" << master
+ << ": applying previously saved blob" << endl;
+ mds->mdcache->uncommitted_slave_updates[master][reqid].replay(mds);
+ mds->mdcache->uncommitted_slave_updates[master].erase(reqid);
+ } else {
+ dout(10) << "ESlaveUpdate.replay commit " << reqid << " for mds" << master
+ << ": ignoring, no previously saved blob" << endl;
+ }
+ break;
+
+ case ESlaveUpdate::OP_ROLLBACK:
+ if (mds->mdcache->uncommitted_slave_updates[master].count(reqid)) {
+ dout(10) << "ESlaveUpdate.replay abort " << reqid << " for mds" << master
+ << ": discarding previously saved blob" << endl;
+ assert(mds->mdcache->uncommitted_slave_updates[master].count(reqid));
+ mds->mdcache->uncommitted_slave_updates[master].erase(reqid);
+ } else {
+ dout(10) << "ESlaveUpdate.replay abort " << reqid << " for mds" << master
+ << ": ignoring, no previously saved blob" << endl;
+ }
+ break;
+
+ default:
+ assert(0);
+ }
}
// -----------------------
-// EImportMap
+// ESubtreeMap
-bool EImportMap::has_expired(MDS *mds)
+bool ESubtreeMap::has_expired(MDS *mds)
{
- if (mds->mdlog->last_import_map > get_end_off()) {
- dout(10) << "EImportMap.has_expired -- there's a newer map" << endl;
+ if (mds->mdlog->get_last_subtree_map_offset() > get_start_off()) {
+ dout(10) << "ESubtreeMap.has_expired -- there's a newer map" << endl;
return true;
- }
- else if (mds->mdlog->is_capped()) {
- dout(10) << "EImportMap.has_expired -- log is capped, allowing map to expire" << endl;
+ } else if (mds->mdlog->is_capped()) {
+ dout(10) << "ESubtreeMap.has_expired -- log is capped, allowing map to expire" << endl;
return true;
} else {
- dout(10) << "EImportMap.has_expired -- not until there's a newer map written" << endl;
+ dout(10) << "ESubtreeMap.has_expired -- not until there's a newer map written"
+ << " (" << get_start_off() << " >= " << mds->mdlog->get_last_subtree_map_offset() << ")"
+ << endl;
return false;
}
}
-/*
-class C_MDS_ImportMapFlush : public Context {
- MDS *mds;
- off_t end_off;
-public:
- C_MDS_ImportMapFlush(MDS *m, off_t eo) : mds(m), end_off(eo) { }
- void finish(int r) {
- // am i the last thing in the log?
- if (mds->mdlog->get_write_pos() == end_off) {
- // yes. we're good.
- } else {
- // no. submit another import_map so that we can go away.
- }
- }
-};
-*/
-
-void EImportMap::expire(MDS *mds, Context *c)
+void ESubtreeMap::expire(MDS *mds, Context *c)
{
- dout(10) << "EImportMap.has_expire -- waiting for a newer map to be written (or for shutdown)" << endl;
- mds->mdlog->import_map_expire_waiters.push_back(c);
+ dout(10) << "ESubtreeMap.has_expire -- waiting for a newer map to be written (or for shutdown)" << endl;
+ mds->mdlog->add_subtree_map_expire_waiter(c);
}
-void EImportMap::replay(MDS *mds)
+void ESubtreeMap::replay(MDS *mds)
{
if (mds->mdcache->is_subtrees()) {
- dout(10) << "EImportMap.replay -- ignoring, already have import map" << endl;
+ dout(10) << "ESubtreeMap.replay -- ignoring, already have import map" << endl;
} else {
- dout(10) << "EImportMap.replay -- reconstructing (auth) subtree spanning tree" << endl;
+ dout(10) << "ESubtreeMap.replay -- reconstructing (auth) subtree spanning tree" << endl;
// first, stick the spanning tree in my cache
+ //metablob.print(cout);
metablob.replay(mds);
// restore import/export maps
- for (set<dirfrag_t>::iterator p = imports.begin();
- p != imports.end();
+ for (map<dirfrag_t, list<dirfrag_t> >::iterator p = subtrees.begin();
+ p != subtrees.end();
++p) {
- CDir *dir = mds->mdcache->get_dirfrag(*p);
- mds->mdcache->adjust_subtree_auth(dir, mds->get_nodeid());
+ CDir *dir = mds->mdcache->get_dirfrag(p->first);
+ mds->mdcache->adjust_bounded_subtree_auth(dir, p->second, mds->get_nodeid());
}
}
mds->mdcache->show_subtrees();
void EImportFinish::replay(MDS *mds)
{
- dout(10) << "EImportFinish.replay " << base << " success=" << success << endl;
- if (success)
- mds->mdcache->finish_ambiguous_import(base);
- else
- mds->mdcache->cancel_ambiguous_import(base);
+ if (mds->mdcache->have_ambiguous_import(base)) {
+ dout(10) << "EImportFinish.replay " << base << " success=" << success << endl;
+ if (success)
+ mds->mdcache->finish_ambiguous_import(base);
+ else
+ mds->mdcache->cancel_ambiguous_import(base);
+ } else {
+ dout(10) << "EImportFinish.replay " << base << " success=" << success
+ << ", predates my subtree_map start point, ignoring"
+ << endl;
+ // verify that?
+ }
}
#define MDS_INO_ANCHORTABLE 3
#define MDS_INO_LOG_OFFSET 0x100
#define MDS_INO_IDS_OFFSET 0x200
-#define MDS_INO_STRAY_OFFSET 0x300
+#define MDS_INO_CLIENTMAP_OFFSET 0x300
+#define MDS_INO_STRAY_OFFSET 0x400
#define MDS_INO_BASE 0x1000
#define MDS_INO_STRAY(x) (MDS_INO_STRAY_OFFSET+((unsigned)x))
}
+// inode caps info for client reconnect
+struct inode_caps_reconnect_t {
+ int32_t wanted;
+ int32_t issued;
+ off_t size;
+ utime_t mtime, atime;
+
+ inode_caps_reconnect_t() {}
+ inode_caps_reconnect_t(int w, int i) :
+ wanted(w), issued(i), size(0) {}
+ inode_caps_reconnect_t(int w, int i, off_t sz, utime_t mt, utime_t at) :
+ wanted(w), issued(i), size(sz), mtime(mt), atime(at) {}
+};
// ================================================================
#define CDIR_AUTH_UNKNOWN -2
#define CDIR_AUTH_DEFAULT pair<int,int>(-1, -2)
#define CDIR_AUTH_UNDEF pair<int,int>(-2, -2)
-#define CDIR_AUTH_ROOTINODE pair<int,int>( 0, -2)
+//#define CDIR_AUTH_ROOTINODE pair<int,int>( 0, -2)
// printer
ostream& operator<<(ostream& out, MDSCacheObject &o);
+class MDSCacheObjectInfo {
+public:
+ inodeno_t ino;
+ dirfrag_t dirfrag;
+ string dname;
+
+ void _encode(bufferlist& bl) const {
+ ::_encode(ino, bl);
+ ::_encode(dirfrag, bl);
+ ::_encode(dname, bl);
+ }
+ void _decode(bufferlist& bl, int& off) {
+ ::_decode(ino, bl, off);
+ ::_decode(dirfrag, bl, off);
+ ::_decode(dname, bl, off);
+ }
+};
+
class MDSCacheObject {
public:
// -- pins --
const static int PIN_REPLICATED = 1000;
const static int PIN_DIRTY = 1001;
- const static int PIN_RDLOCK = -1002;
- const static int PIN_XLOCK = 1003;
- const static int PIN_REQUEST = -1004;
- const static int PIN_WAITER = 1005;
+ const static int PIN_LOCK = -1002;
+ const static int PIN_REQUEST = -1003;
+ const static int PIN_WAITER = 1004;
+ const static int PIN_DIRTYSCATTERED = 1005;
const char *generic_pin_name(int p) {
switch (p) {
- case PIN_REPLICATED: return "replicated";
- case PIN_DIRTY: return "dirty";
- case PIN_RDLOCK: return "rdlock";
- case PIN_XLOCK: return "xlock";
+ case PIN_REPLICATED: return "replicated";
+ case PIN_DIRTY: return "dirty";
+ case PIN_LOCK: return "lock";
case PIN_REQUEST: return "request";
- case PIN_WAITER: return "waiter";
- default: assert(0);
- }
+ case PIN_WAITER: return "waiter";
+ case PIN_DIRTYSCATTERED: return "dirtyscattered";
+ default: assert(0);
+ }
}
// -- state --
const static int STATE_REJOINING = (1<<28); // replica has not joined w/ primary copy
// -- wait --
- const static int WAIT_SINGLEAUTH = (1<<30);
+ const static int WAIT_SINGLEAUTH = (1<<30);
+ const static int WAIT_AUTHPINNABLE = (1<<29);
// ============================================
// cons
public:
MDSCacheObject() :
- state(0),
- ref(0),
- replica_nonce(0) {}
+ state(0),
+ ref(0),
+ replica_nonce(0) {}
virtual ~MDSCacheObject() {}
// printing
virtual void print(ostream& out) = 0;
virtual ostream& print_db_line_prefix(ostream& out) {
- return out << "mdscacheobject(" << this << ") ";
+ return out << "mdscacheobject(" << this << ") ";
}
// --------------------------------------------
// authority
virtual pair<int,int> authority() = 0;
bool is_ambiguous_auth() {
- return authority().second != CDIR_AUTH_UNKNOWN;
+ return authority().second != CDIR_AUTH_UNKNOWN;
}
// --------------------------------------------
virtual void last_put() {}
virtual void bad_put(int by) {
- assert(ref_set.count(by) > 0);
- assert(ref > 0);
+ assert(ref_set.count(by) > 0);
+ assert(ref > 0);
}
void put(int by) {
if (ref == 0 || ref_set.count(by) == 0) {
- bad_put(by);
+ bad_put(by);
} else {
- ref--;
- ref_set.erase(ref_set.find(by));
- assert(ref == (int)ref_set.size());
- if (ref == 0)
- last_put();
- }
+ ref--;
+ ref_set.erase(ref_set.find(by));
+ assert(ref == (int)ref_set.size());
+ if (ref == 0)
+ last_put();
+ }
}
virtual void first_get() {}
virtual void bad_get(int by) {
- assert(by < 0 || ref_set.count(by) == 0);
- assert(0);
+ assert(by < 0 || ref_set.count(by) == 0);
+ assert(0);
}
void get(int by) {
if (by >= 0 && ref_set.count(by)) {
- bad_get(by);
+ bad_get(by);
} else {
- if (ref == 0)
- first_get();
- ref++;
- ref_set.insert(by);
- assert(ref == (int)ref_set.size());
- }
+ if (ref == 0)
+ first_get();
+ ref++;
+ ref_set.insert(by);
+ assert(ref == (int)ref_set.size());
+ }
}
void print_pin_set(ostream& out) {
int last = *it;
int c = 1;
do {
- it++;
- if (it == ref_set.end()) break;
+ it++;
+ if (it == ref_set.end()) break;
} while (*it == last);
if (c > 1)
- out << "*" << c;
+ out << "*" << c;
}
}
+ // --------------------------------------------
+ // auth pins
+ virtual bool can_auth_pin() = 0;
+ virtual void auth_pin() = 0;
+ virtual void auth_unpin() = 0;
+
+
// --------------------------------------------
// replication
protected:
bool is_replica(int mds) { return replicas.count(mds); }
int num_replicas() { return replicas.size(); }
int add_replica(int mds) {
- if (replicas.count(mds))
- return ++replicas[mds]; // inc nonce
- if (replicas.empty())
- get(PIN_REPLICATED);
- return replicas[mds] = 1;
+ if (replicas.count(mds))
+ return ++replicas[mds]; // inc nonce
+ if (replicas.empty())
+ get(PIN_REPLICATED);
+ return replicas[mds] = 1;
}
void add_replica(int mds, int nonce) {
- if (replicas.empty())
- get(PIN_REPLICATED);
- replicas[mds] = nonce;
+ if (replicas.empty())
+ get(PIN_REPLICATED);
+ replicas[mds] = nonce;
}
int get_replica_nonce(int mds) {
- assert(replicas.count(mds));
- return replicas[mds];
+ assert(replicas.count(mds));
+ return replicas[mds];
}
void remove_replica(int mds) {
- assert(replicas.count(mds));
- replicas.erase(mds);
- if (replicas.empty())
- put(PIN_REPLICATED);
+ assert(replicas.count(mds));
+ replicas.erase(mds);
+ if (replicas.empty())
+ put(PIN_REPLICATED);
}
void clear_replicas() {
- if (!replicas.empty())
- put(PIN_REPLICATED);
- replicas.clear();
+ if (!replicas.empty())
+ put(PIN_REPLICATED);
+ replicas.clear();
}
map<int,int>::iterator replicas_begin() { return replicas.begin(); }
map<int,int>::iterator replicas_end() { return replicas.end(); }
const map<int,int>& get_replicas() { return replicas; }
+ void list_replicas(set<int>& ls) {
+ for (map<int,int>::const_iterator p = replicas.begin();
+ p != replicas.end();
+ ++p)
+ ls.insert(p->first);
+ }
int get_replica_nonce() { return replica_nonce;}
void set_replica_nonce(int n) { replica_nonce = n; }
public:
bool is_waiter_for(int mask) {
- return waiting.count(mask) > 0; // FIXME: not quite right.
+ return waiting.count(mask) > 0; // FIXME: not quite right.
}
- void add_waiter(int mask, Context *c) {
- if (waiting.empty())
- get(PIN_WAITER);
- waiting.insert(pair<int,Context*>(mask, c));
- pdout(10,g_conf.debug_mds) << (mdsco_db_line_prefix(this))
- << "add_waiter " << mask << " " << c
- << " on " << *this
- << endl;
-
+ virtual void add_waiter(int mask, Context *c) {
+ if (waiting.empty())
+ get(PIN_WAITER);
+ waiting.insert(pair<int,Context*>(mask, c));
+ pdout(10,g_conf.debug_mds) << (mdsco_db_line_prefix(this))
+ << "add_waiter " << hex << mask << dec << " " << c
+ << " on " << *this
+ << endl;
+
}
- void take_waiting(int mask, list<Context*>& ls) {
- if (waiting.empty()) return;
- multimap<int,Context*>::iterator it = waiting.begin();
- while (it != waiting.end()) {
- if (it->first & mask) {
- ls.push_back(it->second);
- pdout(10,g_conf.debug_mds) << (mdsco_db_line_prefix(this))
- << "take_waiting mask " << mask << " took " << it->second
- << " tag " << it->first
- << " on " << *this
- << endl;
- waiting.erase(it++);
- } else {
- pdout(10,g_conf.debug_mds) << "take_waiting mask " << mask << " SKIPPING " << it->second
- << " tag " << it->first
- << " on " << *this
- << endl;
- it++;
- }
- }
- if (waiting.empty())
- put(PIN_WAITER);
+ virtual void take_waiting(int mask, list<Context*>& ls) {
+ if (waiting.empty()) return;
+ multimap<int,Context*>::iterator it = waiting.begin();
+ while (it != waiting.end()) {
+ if (it->first & mask) {
+ ls.push_back(it->second);
+ pdout(10,g_conf.debug_mds) << (mdsco_db_line_prefix(this))
+ << "take_waiting mask " << hex << mask << dec << " took " << it->second
+ << " tag " << it->first
+ << " on " << *this
+ << endl;
+ waiting.erase(it++);
+ } else {
+ pdout(10,g_conf.debug_mds) << "take_waiting mask " << hex << mask << dec << " SKIPPING " << it->second
+ << " tag " << it->first
+ << " on " << *this
+ << endl;
+ it++;
+ }
+ }
+ if (waiting.empty())
+ put(PIN_WAITER);
}
void finish_waiting(int mask, int result = 0) {
- list<Context*> finished;
- take_waiting(mask, finished);
- finish_contexts(finished, result);
+ list<Context*> finished;
+ take_waiting(mask, finished);
+ finish_contexts(finished, result);
}
// locking
// noop unless overloaded.
virtual SimpleLock* get_lock(int type) { assert(0); return 0; }
- virtual void set_mlock_info(MLock *m) { assert(0); }
+ virtual void set_object_info(MDSCacheObjectInfo &info) { assert(0); }
virtual void encode_lock_state(int type, bufferlist& bl) { assert(0); }
virtual void decode_lock_state(int type, bufferlist& bl) { assert(0); }
virtual void finish_lock_waiters(int type, int mask, int r=0) { assert(0); }
return out;
}
+inline ostream& operator<<(ostream& out, const MDSCacheObjectInfo &info) {
+ if (info.ino) return out << info.ino;
+ if (info.dname.length()) return out << info.dirfrag << "/" << info.dname;
+ return out << info.dirfrag;
+}
+
inline ostream& operator<<(ostream& out, mdsco_db_line_prefix o) {
o.object->print_db_line_prefix(out);
return out;
#define __MCLIENTFILECAPS_H
#include "msg/Message.h"
+#include "mds/Capability.h"
class MClientFileCaps : public Message {
public:
- static const int OP_ACK = 0; // mds->client or client->mds update. FIXME?
- static const int OP_RELEASE = 1; // mds closed the cap
- static const int OP_STALE = 2; // mds has exported the cap
- static const int OP_REAP = 3; // mds has imported the cap from get_mds()
+ static const int OP_GRANT = 0; // mds->client grant.
+ static const int OP_ACK = 1; // client->mds ack (if prior grant was a recall)
+ static const int OP_RELEASE = 2; // mds closed the cap
+ static const int OP_STALE = 3; // mds has exported the cap
+ static const int OP_REAP = 4; // mds has imported the cap from get_mds()
+ static const char* get_opname(int op) {
+ switch (op) {
+ case OP_GRANT: return "grant";
+ case OP_ACK: return "ack";
+ case OP_RELEASE: return "release";
+ case OP_STALE: return "stale";
+ case OP_REAP: return "reap";
+ default: assert(0); return 0;
+ }
+ }
private:
- inode_t inode;
- int caps;
- long seq;
- int wanted;
+ int32_t op;
+ inode_t inode;
+ capseq_t seq;
+ int32_t caps;
+ int32_t wanted;
- int special; // stale || reap; in conjunction w/ mds value
- int mds;
+ int32_t mds;
public:
inodeno_t get_ino() { return inode.ino; }
// for cap migration
int get_mds() { return mds; }
- int get_special() { return special; }
+ int get_op() { return op; }
void set_caps(int c) { caps = c; }
void set_wanted(int w) { wanted = w; }
void set_mds(int m) { mds = m; }
- void set_special(int s) { special = s; }
+ void set_op(int s) { op = s; }
MClientFileCaps() {}
- MClientFileCaps(inode_t& inode,
- long seq,
- int caps,
- int wanted,
- int special = OP_ACK,
- int mds=0) :
- Message(MSG_CLIENT_FILECAPS) {
- this->inode = inode;
- this->seq = seq;
- this->caps = caps;
- this->wanted = wanted;
- this->special = special;
- this->mds = mds;
- }
+ MClientFileCaps(int op_,
+ inode_t& inode_,
+ long seq_,
+ int caps_,
+ int wanted_,
+ int mds_=0) :
+ Message(MSG_CLIENT_FILECAPS),
+ op(op_),
+ inode(inode_),
+ seq(seq_),
+ caps(caps_),
+ wanted(wanted_),
+ mds(mds_) { }
char *get_type_name() { return "Cfcap";}
void print(ostream& out) {
- out << "client_file_caps(" << inode.ino
+ out << "client_file_caps(" << get_opname(op)
+ << " " << inode.ino
<< " seq " << seq
<< " caps " << cap_string(caps)
<< " wanted" << cap_string(wanted)
void decode_payload() {
int off = 0;
+ ::_decode(op, payload, off);
::_decode(seq, payload, off);
::_decode(inode, payload, off);
::_decode(caps, payload, off);
::_decode(wanted, payload, off);
::_decode(mds, payload, off);
- ::_decode(special, payload, off);
}
void encode_payload() {
+ ::_encode(op, payload);
::_encode(seq, payload);
::_encode(inode, payload);
::_encode(caps, payload);
::_encode(wanted, payload);
::_encode(mds, payload);
- ::_encode(special, payload);
}
};
class MClientReconnect : public Message {
public:
- struct inode_caps_t {
- __int32_t caps;
- __int32_t seq;
- __int32_t wanted;
- off_t size;
- utime_t mtime, atime;
- inode_caps_t() {}
- inode_caps_t(int c, int s, int w) :
- caps(c), seq(s), wanted(w), size(0) {}
- inode_caps_t(int c, int s, int w, off_t sz, utime_t mt, utime_t at) :
- caps(c), seq(s), wanted(w), size(sz), mtime(mt), atime(at) {}
- };
-
- map<inodeno_t, inode_caps_t> inode_caps;
- map<inodeno_t, string> inode_path;
- bool closed;
+ map<inodeno_t, inode_caps_reconnect_t> inode_caps;
+ map<inodeno_t, string> inode_path;
+ bool closed; // true if this session was closed by the client.
MClientReconnect() : Message(MSG_CLIENT_RECONNECT),
closed(false) { }
}
void add_inode_caps(inodeno_t ino,
- int havecaps, long seq, int wanted,
+ int wanted, int issued,
off_t sz, utime_t mt, utime_t at) {
- inode_caps[ino] = inode_caps_t(havecaps, seq, wanted, sz, mt, at);
+ inode_caps[ino] = inode_caps_reconnect_t(wanted, issued, sz, mt, at);
}
void add_inode_path(inodeno_t ino, const string& path) {
inode_path[ino] = path;
int op;
int caller_uid, caller_gid;
+ inodeno_t cwd_ino;
} st;
// path arguments
inodeno_t get_mds_wants_replica_in_dirino() {
return st.mds_wants_replica_in_dirino; }
+ inodeno_t get_cwd_ino() { return st.cwd_ino ? st.cwd_ino:inodeno_t(MDS_INO_ROOT); }
+
void decode_payload() {
int off = 0;
payload.copy(off, sizeof(st), (char*)&st);
void print(ostream& out) {
out << "clientreq(client" << get_client()
<< "." << get_tid()
- << ":";
+ << " ";
switch(get_op()) {
case MDS_OP_STATFS:
out << "statfs"; break;
assert(0);
}
if (get_path().length())
- out << "=" << get_path();
+ out << " " << get_path();
if (get_sarg().length())
out << " " << get_sarg();
if (st.retry_attempt)
class MClientSession : public Message {
public:
- const static int OP_OPEN = 1;
- const static int OP_OPEN_ACK = 2;
- const static int OP_CLOSE = 3;
- const static int OP_CLOSE_ACK = 4;
+ const static int OP_REQUEST_OPEN = 1;
+ const static int OP_OPEN = 2;
+ const static int OP_REQUEST_CLOSE = 3;
+ const static int OP_CLOSE = 4;
static const char *get_opname(int o) {
switch (o) {
+ case OP_REQUEST_OPEN: return "request_open";
case OP_OPEN: return "open";
- case OP_OPEN_ACK: return "open_ack";
+ case OP_REQUEST_CLOSE: return "request_close";
case OP_CLOSE: return "close";
- case OP_CLOSE_ACK: return "close_ack";
default: assert(0);
}
}
- __int32_t op;
+ int32_t op;
+ version_t seq;
MClientSession() : Message(MSG_CLIENT_SESSION) { }
- MClientSession(int o) : Message(MSG_CLIENT_SESSION),
- op(o) { }
+ MClientSession(int o, version_t s=0) :
+ Message(MSG_CLIENT_SESSION),
+ op(o), seq(s) { }
char *get_type_name() { return "client_session"; }
void print(ostream& out) {
- out << "client_session " << get_opname(op);
+ out << "client_session(" << get_opname(op);
+ if (seq) out << " seq " << seq;
+ out << ")";
}
void decode_payload() {
int off = 0;
::_decode(op, payload, off);
+ ::_decode(seq, payload, off);
}
void encode_payload() {
::_encode(op, payload);
+ ::_encode(seq, payload);
}
};
CDirDiscover *straydir;
CDentryDiscover *straydn;
- MDentryUnlink() {}
+ MDentryUnlink() :
+ Message(MSG_MDS_DENTRYUNLINK),
+ strayin(0), straydir(0), straydn(0) { }
MDentryUnlink(dirfrag_t df, string& n) :
Message(MSG_MDS_DENTRYUNLINK),
dirfrag(df),
int asker;
inodeno_t base_ino; // 1 -> root
frag_t base_dir_frag;
- bool want_base_dir;
filepath want; // ... [/]need/this/stuff
inodeno_t want_ino;
+ bool want_base_dir;
+ bool want_xlocked;
+
public:
int get_asker() { return asker; }
inodeno_t get_base_ino() { return base_ino; }
filepath& get_want() { return want; }
inodeno_t get_want_ino() { return want_ino; }
const string& get_dentry(int n) { return want[n]; }
- bool wants_base_dir() { return want_base_dir; }
+
+ bool wants_base_dir() { return want_base_dir; }
+ bool wants_xlocked() { return want_xlocked; }
void set_base_dir_frag(frag_t f) { base_dir_frag = f; }
MDiscover() { }
- MDiscover(int asker,
- inodeno_t base_ino,
- filepath& want,
- bool want_base_dir = true) :
- Message(MSG_MDS_DISCOVER) {
- this->asker = asker;
- this->base_ino = base_ino;
- this->want = want;
- want_ino = 0;
- this->want_base_dir = want_base_dir;
- }
- MDiscover(int asker,
+ MDiscover(int asker_,
+ inodeno_t base_ino_,
+ filepath& want_,
+ bool want_base_dir_ = true,
+ bool discover_xlocks_ = false) :
+ Message(MSG_MDS_DISCOVER),
+ asker(asker_),
+ base_ino(base_ino_),
+ want(want_),
+ want_ino(0),
+ want_base_dir(want_base_dir_),
+ want_xlocked(discover_xlocks_) { }
+ MDiscover(int asker_,
dirfrag_t base_dirfrag,
- inodeno_t want_ino,
- bool want_base_dir = true) :
- Message(MSG_MDS_DISCOVER) {
- this->asker = asker;
- this->base_ino = base_dirfrag.ino;
- this->base_dir_frag = base_dirfrag.frag;
- this->want_ino = want_ino;
- this->want_base_dir = want_base_dir;
- }
+ inodeno_t want_ino_,
+ bool want_base_dir_ = true) :
+ Message(MSG_MDS_DISCOVER),
+ asker(asker_),
+ base_ino(base_dirfrag.ino),
+ base_dir_frag(base_dirfrag.frag),
+ want_ino(want_ino_),
+ want_base_dir(want_base_dir_),
+ want_xlocked(false) { }
char *get_type_name() { return "Dis"; }
void print(ostream &out) {
::_decode(asker, payload, off);
::_decode(base_ino, payload, off);
::_decode(base_dir_frag, payload, off);
- ::_decode(want_base_dir, payload, off);
want._decode(payload, off);
::_decode(want_ino, payload, off);
+ ::_decode(want_base_dir, payload, off);
+ ::_decode(want_xlocked, payload, off);
}
void encode_payload() {
- payload.append((char*)&asker, sizeof(asker));
- payload.append((char*)&base_ino, sizeof(base_ino));
- payload.append((char*)&base_dir_frag, sizeof(base_dir_frag));
- payload.append((char*)&want_base_dir, sizeof(want_base_dir));
+ ::_encode(asker, payload);
+ ::_encode(base_ino, payload);
+ ::_encode(base_dir_frag, payload);
want._encode(payload);
::_encode(want_ino, payload);
+ ::_encode(want_base_dir, payload);
+ ::_encode(want_xlocked, payload);
}
};
bool flag_error_ino;
bool flag_error_dir;
string error_dentry; // dentry that was not found (to trigger waiters on asker)
- int dir_auth_hint;
+ int dir_auth_hint;
+ bool wanted_xlocks_hint;
vector<CDirDiscover*> dirs; // not inode-aligned if no_base_dir = true.
vector<CDentryDiscover*> dentries; // not inode-aligned if no_base_dentry = true
dirs.size() + no_base_dir )); // dn/inode + dirs
}
- bool has_base_dir() { return !no_base_dir && dirs.size(); }
- bool has_base_dentry() { return !no_base_dentry && dentries.size(); }
- bool has_root() {
- return (base_ino == MDS_INO_ROOT && no_base_dir && no_base_dentry);
- }
+ bool has_base_dir() { return !no_base_dir && dirs.size(); }
+ bool has_base_dentry() { return !no_base_dentry && dentries.size(); }
+ bool has_base_inode() { return no_base_dir && no_base_dentry; }
const string& get_path() { return path; }
bool is_flag_error_dir() { return flag_error_dir; }
string& get_error_dentry() { return error_dentry; }
int get_dir_auth_hint() { return dir_auth_hint; }
+ bool get_wanted_xlocks_hint() { return wanted_xlocks_hint; }
+
+ void set_wanted_xlocks_hint(bool w) { wanted_xlocks_hint = w; }
// these index _arguments_ are aligned to each ([[dir, ] dentry, ] inode) set.
CInodeDiscover& get_inode(int n) { return *(inodes[n]); }
::_decode(flag_error_dir, payload, off);
::_decode(error_dentry, payload, off);
::_decode(dir_auth_hint, payload, off);
+ ::_decode(wanted_xlocks_hint, payload, off);
// dirs
int n;
::_encode(flag_error_dir, payload);
::_encode(error_dentry, payload);
::_encode(dir_auth_hint, payload);
+ ::_encode(wanted_xlocks_hint, payload);
// dirs
int n = dirs.size();
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MHASHDIR_H
-#define __MHASHDIR_H
-
-#include "msg/Message.h"
-
-class MHashDir : public Message {
- inodeno_t ino;
- bufferlist state;
- int nden;
-
- public:
- MHashDir() {}
- MHashDir(inodeno_t ino) :
- Message(MSG_MDS_HASHDIR) {
- this->ino = ino;
- nden = 0;
- }
- virtual char *get_type_name() { return "Ha"; }
-
- inodeno_t get_ino() { return ino; }
- bufferlist& get_state() { return state; }
- bufferlist* get_state_ptr() { return &state; }
- int get_nden() { return nden; }
-
- void set_nden(int n) { nden = n; }
- void inc_nden() { nden++; }
-
- void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(ino), (char*)&ino);
- off += sizeof(ino);
- payload.copy(off, sizeof(nden), (char*)&nden);
- off += sizeof(nden);
-
- size_t len;
- payload.copy(off, sizeof(len), (char*)&len);
- off += sizeof(len);
- state.substr_of(payload, off, len);
- }
- void encode_payload() {
- payload.append((char*)&ino, sizeof(ino));
- payload.append((char*)&nden, sizeof(nden));
- size_t size = state.length();
- payload.append((char*)&size, sizeof(size));
- payload.claim_append(state);
- }
-
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MHASHDIRACK_H
-#define __MHASHDIRACK_H
-
-#include "MHashDir.h"
-
-class MHashDirAck : public Message {
- inodeno_t ino;
-
- public:
- inodeno_t get_ino() { return ino; }
-
- MHashDirAck() {}
- MHashDirAck(inodeno_t ino) :
- Message(MSG_MDS_HASHDIRACK) {
- this->ino = ino;
- }
- virtual char *get_type_name() { return "HAck"; }
-
- virtual void decode_payload() {
- payload.copy(0, sizeof(ino), (char*)&ino);
- }
- virtual void encode_payload() {
- payload.append((char*)&ino, sizeof(ino));
- }
-
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MHASHDIRDISCOVER_H
-#define __MHASHDIRDISCOVER_H
-
-#include "msg/Message.h"
-#include "mds/CInode.h"
-#include "include/types.h"
-
-class MHashDirDiscover : public Message {
- inodeno_t ino;
- string path;
-
- public:
- inodeno_t get_ino() { return ino; }
- string& get_path() { return path; }
-
- MHashDirDiscover() {}
- MHashDirDiscover(CInode *in) :
- Message(MSG_MDS_HASHDIRDISCOVER) {
- in->make_path(path);
- ino = in->ino();
- }
- virtual char *get_type_name() { return "HDis"; }
-
-
- void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(ino), (char*)&ino);
- off += sizeof(ino);
- _decode(path, payload, off);
- }
-
- void encode_payload() {
- payload.append((char*)&ino, sizeof(ino));
- _encode(path, payload);
- }
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MHASHDIRDISCOVERACK_H
-#define __MHASHDIRDISCOVERACK_H
-
-#include "msg/Message.h"
-#include "mds/CInode.h"
-#include "include/types.h"
-
-class MHashDirDiscoverAck : public Message {
- inodeno_t ino;
- bool success;
-
- public:
- inodeno_t get_ino() { return ino; }
- bool is_success() { return success; }
-
- MHashDirDiscoverAck() {}
- MHashDirDiscoverAck(inodeno_t ino, bool success=true) :
- Message(MSG_MDS_HASHDIRDISCOVERACK) {
- this->ino = ino;
- this->success = false;
- }
- virtual char *get_type_name() { return "HDisA"; }
-
-
- void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(ino), (char*)&ino);
- off += sizeof(ino);
- payload.copy(off, sizeof(success), (char*)&success);
- off += sizeof(success);
- }
-
- void encode_payload() {
- payload.append((char*)&ino, sizeof(ino));
- payload.append((char*)&success, sizeof(success));
- }
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MHASHDIRNOTIFY_H
-#define __MHASHDIRNOTIFY_H
-
-#include "msg/Message.h"
-
-class MHashDirNotify : public Message {
- inodeno_t ino;
- int from;
-
- public:
- inodeno_t get_ino() { return ino; }
- int get_from() { return from; }
-
- MHashDirNotify() {}
- MHashDirNotify(inodeno_t ino, int from) :
- Message(MSG_MDS_HASHDIRNOTIFY) {
- this->ino = ino;
- this->from = from;
- }
- virtual char *get_type_name() { return "HN"; }
-
- virtual void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(ino), (char*)&ino);
- off += sizeof(ino);
- payload.copy(off, sizeof(from), (char*)&from);
- off += sizeof(from);
- }
- virtual void encode_payload() {
- payload.append((char*)&ino, sizeof(ino));
- payload.append((char*)&from, sizeof(from));
- }
-
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MHASHDIRPREP_H
-#define __MHASHDIRPREP_H
-
-#include "msg/Message.h"
-#include "mds/CInode.h"
-#include "include/types.h"
-
-class MHashDirPrep : public Message {
- inodeno_t ino;
- bool assim;
-
- // subdir dentry names + inodes
- map<string,CInodeDiscover*> inodes;
-
- public:
- inodeno_t get_ino() { return ino; }
- map<string,CInodeDiscover*>& get_inodes() { return inodes; }
-
- bool did_assim() { return assim; }
- void mark_assim() { assert(!assim); assim = true; }
-
- MHashDirPrep() : assim(false) { }
- MHashDirPrep(inodeno_t ino) :
- Message(MSG_MDS_HASHDIRPREP),
- assim(false) {
- this->ino = ino;
- }
- ~MHashDirPrep() {
- for (map<string,CInodeDiscover*>::iterator it = inodes.begin();
- it != inodes.end();
- it++)
- delete it->second;
- }
-
-
- virtual char *get_type_name() { return "HP"; }
-
- void add_inode(const string& dentry, CInodeDiscover *in) {
- inodes[dentry] = in;
- }
-
- void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(ino), (char*)&ino);
- off += sizeof(ino);
-
- // inodes
- int ni;
- payload.copy(off, sizeof(int), (char*)&ni);
- off += sizeof(int);
- for (int i=0; i<ni; i++) {
- // dentry
- string dname;
- _decode(dname, payload, off);
-
- // inode
- CInodeDiscover *in = new CInodeDiscover;
- in->_decode(payload, off);
-
- inodes[dname] = in;
- }
- }
-
- virtual void encode_payload() {
- payload.append((char*)&ino, sizeof(ino));
-
- // inodes
- int ni = inodes.size();
- payload.append((char*)&ni, sizeof(int));
- for (map<string,CInodeDiscover*>::iterator iit = inodes.begin();
- iit != inodes.end();
- iit++) {
- _encode(iit->first, payload); // dentry
- iit->second->_encode(payload); // inode
- }
- }
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MHASHDIRPREPACK_H
-#define __MHASHDIRPREPACK_H
-
-#include "msg/Message.h"
-#include "include/types.h"
-
-class MHashDirPrepAck : public Message {
- inodeno_t ino;
-
- public:
- inodeno_t get_ino() { return ino; }
-
- MHashDirPrepAck() {}
- MHashDirPrepAck(inodeno_t ino) :
- Message(MSG_MDS_HASHDIRPREPACK) {
- this->ino = ino;
- }
-
- virtual char *get_type_name() { return "HPAck"; }
-
- void decode_payload() {
- payload.copy(0, sizeof(ino), (char*)&ino);
- }
- void encode_payload() {
- payload.append((char*)&ino, sizeof(ino));
- }
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MHASHREADDIR_H
-#define __MHASHREADDIR_H
-
-#include "include/types.h"
-#include "msg/Message.h"
-
-class MHashReaddir : public Message {
- inodeno_t ino;
-
- public:
- MHashReaddir() { }
- MHashReaddir(inodeno_t ino) :
- Message(MSG_MDS_HASHREADDIR) {
- this->ino = ino;
- }
-
- inodeno_t get_ino() { return ino; }
-
- virtual char *get_type_name() { return "Hls"; }
-
- virtual void decode_payload() {
- payload.copy(0, sizeof(ino), (char*)&ino);
- }
- virtual void encode_payload() {
- payload.append((char*)&ino, sizeof(ino));
- }
-
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MHASHREADDIRREPLY_H
-#define __MHASHREADDIRREPLY_H
-
-#include "MClientReply.h"
-
-class MHashReaddirReply : public Message {
- inodeno_t ino;
-
- list<InodeStat*> dir_in;
- list<string> dir_dn;
-
- int num;
-
- public:
- MHashReaddirReply() { }
- MHashReaddirReply(inodeno_t _ino, list<InodeStat*>& inls, list<string>& dnls, int n) :
- Message(MSG_MDS_HASHREADDIRREPLY),
- ino(_ino),
- num(n) {
- dir_in.swap(inls);
- dir_dn.swap(dnls);
- }
- ~MHashReaddirReply() {
- for (list<InodeStat*>::iterator it = dir_in.begin(); it != dir_in.end(); it++)
- delete *it;
- }
-
- inodeno_t get_ino() { return ino; }
- list<InodeStat*>& get_in() { return dir_in; }
- list<string>& get_dn() { return dir_dn; }
-
- virtual char *get_type_name() { return "Hls"; }
-
- virtual void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(ino), (char*)&ino);
- off += sizeof(ino);
- int n;
- payload.copy(n, sizeof(n), (char*)&n);
- off += sizeof(n);
- for (int i=0; i<n; i++) {
- string dn;
- ::_decode(dn, payload, off);
- dir_dn.push_back(dn);
-
- InodeStat *ci = new InodeStat;
- ci->_decode(payload, off);
- dir_in.push_back(ci);
- }
- }
- virtual void encode_payload() {
- payload.append((char*)&ino, sizeof(ino));
- int n = dir_in.size(); // FIXME?
- payload.append((char*)&n, sizeof(n));
- list<string>::iterator pdn = dir_dn.begin();
- for (list<InodeStat*>::iterator pin = dir_in.begin();
- pin != dir_in.end();
- ++pin, ++pdn) {
- ::_encode(*pdn, payload);
- (*pin)->_encode(payload);
- }
- }
-
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MINODELINK_H
-#define __MINODELINK_H
-
-class MInodeLink : public Message {
-public:
- static const int OP_PREPARE = 1;
- static const int OP_AGREE = 2;
- static const int OP_COMMIT = 3;
- static const int OP_ACK = 4;
- static const int OP_ROLLBACK = 5;
-
- const char *get_opname(int o) {
- switch (o) {
- case OP_PREPARE: return "prepare";
- case OP_AGREE: return "agree";
- case OP_COMMIT: return "commit";
- case OP_ACK: return "ack";
- case OP_ROLLBACK: return "rollback";
- default: assert(0);
- }
- }
-
-private:
- struct _st {
- inodeno_t ino; // inode to nlink++
- metareqid_t reqid; // relevant request
- int op; // see above
- bool inc; // true == ++, false == --
-
- utime_t ctime;
- } st;
-
-public:
- inodeno_t get_ino() { return st.ino; }
- metareqid_t get_reqid() { return st.reqid; }
- int get_op() { return st.op; }
- bool get_inc() { return st.inc; }
-
- utime_t get_ctime() { return st.ctime; }
- void set_ctime(utime_t ct) { st.ctime = ct; }
-
- MInodeLink() {}
- MInodeLink(int op, inodeno_t ino, bool inc, metareqid_t ri) :
- Message(MSG_MDS_INODELINK) {
- st.op = op;
- st.ino = ino;
- st.inc = inc;
- st.reqid = ri;
- }
-
- virtual char *get_type_name() { return "inode_link"; }
- void print(ostream& o) {
- o << "inode_link(" << get_opname(st.op)
- << " " << st.ino
- << " nlink" << (st.inc ? "++":"--")
- << " " << st.reqid << ")";
- }
-
- virtual void decode_payload() {
- int off = 0;
- _decoderaw(st, payload, off);
- }
- virtual void encode_payload() {
- _encode(st, payload);
- }
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MINODELINKACK_H
-#define __MINODELINKACK_H
-
-typedef struct {
- inodeno_t ino;
- bool success;
-} MInodeLinkAck_st;
-
-class MInodeLinkAck : public Message {
- MInodeLinkAck_st st;
-
- public:
- inodeno_t get_ino() { return st.ino; }
- bool is_success() { return st.success; }
-
- MInodeLinkAck() {}
- MInodeLinkAck(inodeno_t ino, bool success) :
- Message(MSG_MDS_INODELINKACK) {
- st.ino = ino;
- st.success = success;
- }
- virtual char *get_type_name() { return "InLA";}
-
- virtual void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(st), (char*)&st);
- off += sizeof(st);
- }
- virtual void encode_payload() {
- payload.append((char*)&st,sizeof(st));
- }
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MINODEUNLINK_H
-#define __MINODEUNLINK_H
-
-typedef struct {
- inodeno_t ino;
- int from;
-} MInodeUnlink_st;
-
-class MInodeUnlink : public Message {
- MInodeUnlink_st st;
-
- public:
- inodeno_t get_ino() { return st.ino; }
- int get_from() { return st.from; }
-
- MInodeUnlink() {}
- MInodeUnlink(inodeno_t ino, int from) :
- Message(MSG_MDS_INODEUNLINK) {
- st.ino = ino;
- st.from = from;
- }
- virtual char *get_type_name() { return "InUl";}
-
- virtual void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(st), (char*)&st);
- off += sizeof(st);
- }
- virtual void encode_payload() {
- payload.append((char*)&st,sizeof(st));
- }
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MINODEUNLINKACK_H
-#define __MINODEUNLINKACK_H
-
-typedef struct {
- inodeno_t ino;
-} MInodeUnlinkAck_st;
-
-class MInodeUnlinkAck : public Message {
- MInodeUnlinkAck_st st;
-
- public:
- inodeno_t get_ino() { return st.ino; }
-
- MInodeUnlinkAck() {}
- MInodeUnlinkAck(inodeno_t ino) :
- Message(MSG_MDS_INODEUNLINKACK) {
- st.ino = ino;
- }
- virtual char *get_type_name() { return "InUlA";}
-
- virtual void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(st), (char*)&st);
- off += sizeof(st);
- }
- virtual void encode_payload() {
- payload.append((char*)&st,sizeof(st));
- }
-};
-
-#endif
#define LOCK_AC_MIXED -2
#define LOCK_AC_LOCK -3
-#define LOCK_AC_REQXLOCKACK -4 // req dentry xlock
-#define LOCK_AC_REQXLOCKNAK -5 // req dentry xlock
-
#define LOCK_AC_SCATTER -6
// for auth
#define LOCK_AC_MIXEDACK 2
#define LOCK_AC_LOCKACK 3
-#define LOCK_AC_REQREAD 4
-#define LOCK_AC_REQWRITE 5
-
-#define LOCK_AC_REQXLOCK 6
-#define LOCK_AC_UNXLOCK 7
-#define LOCK_AC_FINISH 8
-
+#define LOCK_AC_REQSCATTER 7
#define LOCK_AC_FOR_REPLICA(a) ((a) < 0)
#define LOCK_AC_FOR_AUTH(a) ((a) > 0)
+static const char *get_lock_action_name(int a) {
+ switch (a) {
+ case LOCK_AC_SYNC: return "sync";
+ case LOCK_AC_MIXED: return "mixed";
+ case LOCK_AC_LOCK: return "lock";
+ case LOCK_AC_SCATTER: return "scatter";
+ case LOCK_AC_SYNCACK: return "syncack";
+ case LOCK_AC_MIXEDACK: return "mixedack";
+ case LOCK_AC_LOCKACK: return "lockack";
+ case LOCK_AC_REQSCATTER: return "reqscatter";
+ default: assert(0);
+ }
+}
+
+
class MLock : public Message {
int asker; // who is initiating this request
int action; // action type
-
- char otype; // lock object type
- inodeno_t ino; // ino ref, or possibly
- dirfrag_t dirfrag;
- string dn; // dentry name
-
metareqid_t reqid; // for remote lock requests
+
+ char lock_type; // lock object type
+ MDSCacheObjectInfo object_info;
bufferlist data; // and possibly some data
public:
- inodeno_t get_ino() { return ino; }
- dirfrag_t get_dirfrag() { return dirfrag; }
- string& get_dn() { return dn; }
bufferlist& get_data() { return data; }
int get_asker() { return asker; }
int get_action() { return action; }
- int get_otype() { return otype; }
metareqid_t get_reqid() { return reqid; }
+ int get_lock_type() { return lock_type; }
+ MDSCacheObjectInfo &get_object_info() { return object_info; }
+
MLock() {}
MLock(int action, int asker) :
Message(MSG_MDS_LOCK) {
}
MLock(SimpleLock *lock, int action, int asker) :
Message(MSG_MDS_LOCK) {
- this->otype = lock->get_type();
- lock->get_parent()->set_mlock_info(this);
+ this->lock_type = lock->get_type();
+ lock->get_parent()->set_object_info(object_info);
this->action = action;
this->asker = asker;
}
MLock(SimpleLock *lock, int action, int asker, bufferlist& bl) :
Message(MSG_MDS_LOCK) {
- this->otype = lock->get_type();
- lock->get_parent()->set_mlock_info(this);
+ this->lock_type = lock->get_type();
+ lock->get_parent()->set_object_info(object_info);
this->action = action;
this->asker = asker;
data.claim(bl);
}
virtual char *get_type_name() { return "ILock"; }
void print(ostream& out) {
- out << "lock(a=" << action
- << " " << ino
- << " " << get_lock_type_name(otype)
+ out << "lock(a=" << get_lock_action_name(action)
+ << " " << get_lock_type_name(lock_type)
+ << " " << object_info
<< ")";
}
- void set_ino(inodeno_t ino, char ot) {
- otype = ot;
- this->ino = ino;
- }
- void set_ino(inodeno_t ino) {
- this->ino = ino;
- }
- /*
- void set_dirfrag(dirfrag_t df) {
- otype = LOCK_OTYPE_DIR;
- this->dirfrag = df;
- }
- */
- void set_dn(dirfrag_t df, const string& dn) {
- otype = LOCK_OTYPE_DN;
- this->dirfrag = df;
- this->dn = dn;
- }
void set_reqid(metareqid_t ri) { reqid = ri; }
void set_data(const bufferlist& data) {
this->data = data;
void decode_payload() {
int off = 0;
- ::_decode(action, payload, off);
::_decode(asker, payload, off);
- ::_decode(otype, payload, off);
- ::_decode(ino, payload, off);
- ::_decode(dirfrag, payload, off);
+ ::_decode(action, payload, off);
::_decode(reqid, payload, off);
- ::_decode(dn, payload, off);
+ ::_decode(lock_type, payload, off);
+ object_info._decode(payload, off);
::_decode(data, payload, off);
}
virtual void encode_payload() {
- ::_encode(action, payload);
::_encode(asker, payload);
- ::_encode(otype, payload);
- ::_encode(ino, payload);
- ::_encode(dirfrag, payload);
+ ::_encode(action, payload);
::_encode(reqid, payload);
- ::_encode(dn, payload);
+ ::_encode(lock_type, payload);
+ object_info._encode(payload);
::_encode(data, payload);
}
#include "msg/Message.h"
#include "include/types.h"
+#include "include/encodable.h"
// sent from replica to auth
class MMDSCacheRejoin : public Message {
public:
- static const int OP_REJOIN = 1; // replica -> auth, i exist. and maybe my lock state.
+ static const int OP_WEAK = 1; // replica -> auth, i exist, + maybe open files.
+ static const int OP_STRONG = 2; // replica -> auth, i exist, + open files and lock state.
static const int OP_ACK = 3; // auth -> replica, here is your lock state.
- static const int OP_MISSING = 4; // auth -> replica, i am missing these items
- static const int OP_FULL = 5; // replica -> auth, here is the full object.
+ static const int OP_PURGE = 4; // auth -> replica, remove these items, they are old/obsolete.
+ static const int OP_MISSING = 5; // auth -> replica, i am missing these items
+ static const int OP_FULL = 6; // replica -> auth, here is the full object.
static const char *get_opname(int op) {
switch (op) {
- case OP_REJOIN: return "rejoin";
+ case OP_WEAK: return "weak";
+ case OP_STRONG: return "strong";
case OP_ACK: return "ack";
case OP_MISSING: return "missing";
case OP_FULL: return "full";
inode_full() {}
inode_full(const inode_t& i, const string& s, const fragtree_t& f) :
inode(i), symlink(s), dirfragtree(f) {}
- inode_full(bufferlist& bl, int& off) {
+
+ void _decode(bufferlist& bl, int& off) {
::_decode(inode, bl, off);
::_decode(symlink, bl, off);
::_decode(dirfragtree, bl, off);
}
- void _encode(bufferlist& bl) {
+ void _encode(bufferlist& bl) const {
::_encode(inode, bl);
::_encode(symlink, bl);
::_encode(dirfragtree, bl);
dirfrag_strong(int n) : nonce(n) {}
};
struct dn_strong {
+ inodeno_t ino;
+ inodeno_t remote_ino;
int32_t nonce;
int32_t lock;
- dn_strong() {}
- dn_strong(int n, int l) : nonce(n), lock(l) {}
+ dn_strong() : ino(0), remote_ino(0), nonce(0), lock(0) {}
+ dn_strong(inodeno_t pi, inodeno_t ri, int n, int l) :
+ ino(pi), remote_ino(ri), nonce(n), lock(l) {}
+ bool is_primary() { return ino > 0; }
+ bool is_remote() { return remote_ino > 0; }
+ bool is_null() { return ino == 0 && remote_ino == 0; }
+ };
+
+ struct dn_weak {
+ inodeno_t ino;
+ inodeno_t remote_ino;
+ dn_weak() : ino(0), remote_ino(0) {}
+ dn_weak(inodeno_t pi, inodeno_t ri) : ino(pi), remote_ino(ri) {}
+ bool is_primary() { return ino > 0; }
+ bool is_remote() { return remote_ino > 0; }
+ bool is_null() { return ino == 0 && remote_ino == 0; }
};
// -- data --
int32_t op;
+ // weak
+ map<dirfrag_t, map<string, dn_weak> > weak;
set<inodeno_t> weak_inodes;
- map<inodeno_t, inode_strong> strong_inodes;
- list<inode_full> full_inodes;
- map<inodeno_t, map<int, metareqid_t> > xlocked_inodes;
- set<dirfrag_t> weak_dirfrags;
+ // strong
map<dirfrag_t, dirfrag_strong> strong_dirfrags;
-
- map<dirfrag_t, set<string> > weak_dentries;
map<dirfrag_t, map<string, dn_strong> > strong_dentries;
+ map<inodeno_t, inode_strong> strong_inodes;
+
+ // open
+ bufferlist cap_export_bl;
+ map<inodeno_t,map<int, inode_caps_reconnect_t> > cap_exports;
+ map<inodeno_t,string> cap_export_paths;
+
+ // full
+ list<inode_full> full_inodes;
+
+ // authpins, xlocks
+ map<inodeno_t, metareqid_t> authpinned_inodes;
+ map<inodeno_t, map<int, metareqid_t> > xlocked_inodes;
+ map<dirfrag_t, map<string, metareqid_t> > authpinned_dentries;
map<dirfrag_t, map<string, metareqid_t> > xlocked_dentries;
MMDSCacheRejoin() : Message(MSG_MDS_CACHEREJOIN) {}
// -- builders --
// inodes
- void add_weak_inode(inodeno_t ino) {
- weak_inodes.insert(ino);
+ void add_weak_inode(inodeno_t i) {
+ weak_inodes.insert(i);
}
void add_strong_inode(inodeno_t i, int n, int cw, int a, int l, int dft, int f, int dl) {
strong_inodes[i] = inode_strong(n, cw, a, l, dft, f, dl);
void add_full_inode(inode_t &i, const string& s, const fragtree_t &f) {
full_inodes.push_back(inode_full(i, s, f));
}
+ void add_inode_authpin(inodeno_t ino, const metareqid_t& ri) {
+ authpinned_inodes[ino] = ri;
+ }
void add_inode_xlock(inodeno_t ino, int lt, const metareqid_t& ri) {
xlocked_inodes[ino][lt] = ri;
}
+
+ void copy_cap_exports(bufferlist &bl) {
+ cap_export_bl = bl;
+ }
// dirfrags
void add_weak_dirfrag(dirfrag_t df) {
- weak_dirfrags.insert(df);
+ weak[df];
+ }
+ void add_weak_dirfrag(dirfrag_t df, map<string,dn_weak>& dnmap) {
+ weak[df] = dnmap;
}
void add_strong_dirfrag(dirfrag_t df, int n) {
strong_dirfrags[df] = dirfrag_strong(n);
}
// dentries
- void add_weak_dentry(dirfrag_t df, const string& dname) {
- weak_dentries[df].insert(dname);
+ void add_weak_dentry(dirfrag_t df, const string& dname, dn_weak& dnw) {
+ weak[df][dname] = dnw;
+ }
+ void add_weak_null_dentry(dirfrag_t df, const string& dname) {
+ weak[df][dname] = dn_weak(0, 0);
+ }
+ void add_weak_primary_dentry(dirfrag_t df, const string& dname, inodeno_t ino) {
+ weak[df][dname] = dn_weak(ino, 0);
}
- void add_strong_dentry(dirfrag_t df, const string& dname, int n, int ls) {
- strong_dentries[df][dname] = dn_strong(n, ls);
+ void add_weak_remote_dentry(dirfrag_t df, const string& dname, inodeno_t ino) {
+ weak[df][dname] = dn_weak(0, ino);
+ }
+ void add_strong_dentry(dirfrag_t df, const string& dname, inodeno_t pi, inodeno_t ri, int n, int ls) {
+ strong_dentries[df][dname] = dn_strong(pi, ri, n, ls);
+ }
+ void add_dentry_authpin(dirfrag_t df, const string& dname, const metareqid_t& ri) {
+ authpinned_dentries[df][dname] = ri;
}
void add_dentry_xlock(dirfrag_t df, const string& dname, const metareqid_t& ri) {
xlocked_dentries[df][dname] = ri;
// -- encoding --
void encode_payload() {
::_encode(op, payload);
- ::_encode(weak_inodes, payload);
::_encode(strong_inodes, payload);
-
- uint32_t nfull = full_inodes.size();
- ::_encode(nfull, payload);
- for (list<inode_full>::iterator p = full_inodes.begin(); p != full_inodes.end(); ++p)
- p->_encode(payload);
-
+ ::_encode_complex(full_inodes, payload);
+ ::_encode(authpinned_inodes, payload);
::_encode(xlocked_inodes, payload);
- ::_encode(weak_dirfrags, payload);
+ ::_encode(cap_export_bl, payload);
::_encode(strong_dirfrags, payload);
- ::_encode(weak_dentries, payload);
+ ::_encode(weak, payload);
+ ::_encode(weak_inodes, payload);
::_encode(strong_dentries, payload);
+ ::_encode(authpinned_dentries, payload);
::_encode(xlocked_dentries, payload);
}
void decode_payload() {
int off = 0;
::_decode(op, payload, off);
- ::_decode(weak_inodes, payload, off);
::_decode(strong_inodes, payload, off);
-
- uint32_t nfull;
- ::_decode(nfull, payload, off);
- for (unsigned i=0; i<nfull; i++)
- full_inodes.push_back(inode_full(payload, off));
-
+ ::_decode_complex(full_inodes, payload, off);
+ ::_decode(authpinned_inodes, payload, off);
::_decode(xlocked_inodes, payload, off);
- ::_decode(weak_dirfrags, payload, off);
+ ::_decode(cap_export_bl, payload, off);
+ if (cap_export_bl.length()) {
+ int off = 0;
+ ::_decode(cap_exports, cap_export_bl, off);
+ ::_decode(cap_export_paths, cap_export_bl, off);
+ }
::_decode(strong_dirfrags, payload, off);
- ::_decode(weak_dentries, payload, off);
+ ::_decode(weak, payload, off);
+ ::_decode(weak_inodes, payload, off);
::_decode(strong_dentries, payload, off);
+ ::_decode(authpinned_dentries, payload, off);
::_decode(xlocked_dentries, payload, off);
}
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-#ifndef __MMDSCACHEREJOINACK_H
-#define __MMDSCACHEREJOINACK_H
-
-#include "msg/Message.h"
-
-#include "include/types.h"
-
-// sent from auth back to replica
-
-class MMDSCacheRejoinAck : public Message {
- public:
- struct inodeinfo {
- inodeno_t ino;
- int authlock;
- int linklock;
- int dirfragtreelock;
- int filelock;
- int nonce;
- inodeinfo() {}
- inodeinfo(inodeno_t i, int a, int l, int dft, int f, int n) :
- ino(i),
- authlock(a), linklock(l), dirfragtreelock(dft), filelock(f),
- nonce(n) {}
- };
- struct dninfo {
- int lock;
- int nonce;
- dninfo() {}
- dninfo(int l, int n) : lock(l), nonce(n) {}
- };
- struct dirinfo {
- dirfrag_t dirfrag;
- int nonce;
- dirinfo() {}
- dirinfo(dirfrag_t df, int n) : dirfrag(df), nonce(n) {}
- };
- list<inodeinfo> inodes;
- map<dirfrag_t, map<string,dninfo> > dentries;
- list<dirinfo> dirfrags;
-
- MMDSCacheRejoinAck() : Message(MSG_MDS_CACHEREJOINACK) {}
-
- char *get_type_name() { return "cache_rejoin_ack"; }
-
- void add_dirfrag(dirfrag_t dirfrag, int nonce) {
- dirfrags.push_back(dirinfo(dirfrag,nonce));
- }
- void add_dentry(dirfrag_t dirfrag, const string& dn, int ls, int nonce) {
- dentries[dirfrag][dn] = dninfo(ls, nonce);
- }
- void add_inode(inodeno_t ino, int authl, int linkl, int dftl, int fl, int nonce) {
- inodes.push_back(inodeinfo(ino, authl, linkl, dftl, fl, nonce));
- }
-
- void encode_payload() {
- ::_encode(inodes, payload);
- ::_encode(dirfrags, payload);
- for (list<dirinfo>::iterator p = dirfrags.begin(); p != dirfrags.end(); ++p)
- ::_encode(dentries[p->dirfrag], payload);
- }
- void decode_payload() {
- int off = 0;
- ::_decode(inodes, payload, off);
- ::_decode(dirfrags, payload, off);
- for (list<dirinfo>::iterator p = dirfrags.begin(); p != dirfrags.end(); ++p)
- ::_decode(dentries[p->dirfrag], payload, off);
- }
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-#ifndef __MMDSIMPORTMAP_H
-#define __MMDSIMPORTMAP_H
-
-#include "msg/Message.h"
-
-#include "include/types.h"
-
-
-class MMDSImportMap : public Message {
- public:
- map<dirfrag_t, list<dirfrag_t> > imap;
- map<dirfrag_t, list<dirfrag_t> > ambiguous_imap;
-
- MMDSImportMap() : Message(MSG_MDS_IMPORTMAP) {}
-
- char *get_type_name() { return "mdsimportmap"; }
-
- void print(ostream& out) {
- out << "mdsimportmap(" << imap.size()
- << "+" << ambiguous_imap.size()
- << " imports)";
- }
-
- void add_import(dirfrag_t im) {
- imap[im].clear();
- }
- void add_import_export(dirfrag_t im, dirfrag_t ex) {
- imap[im].push_back(ex);
- }
-
- void add_ambiguous_import(dirfrag_t im, const list<dirfrag_t>& m) {
- ambiguous_imap[im] = m;
- }
-
- void encode_payload() {
- ::_encode(imap, payload);
- ::_encode(ambiguous_imap, payload);
- }
- void decode_payload() {
- int off = 0;
- ::_decode(imap, payload, off);
- ::_decode(ambiguous_imap, payload, off);
- }
-};
-
-#endif
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation. See file COPYING.
+ *
+ */
+
+#ifndef __MMDSRESOLVE_H
+#define __MMDSRESOLVE_H
+
+#include "msg/Message.h"
+
+#include "include/types.h"
+
+class MMDSResolve : public Message {
+ public:
+ map<dirfrag_t, list<dirfrag_t> > subtrees;
+ map<dirfrag_t, list<dirfrag_t> > ambiguous_imports;
+ list<metareqid_t> slave_requests;
+
+ MMDSResolve() : Message(MSG_MDS_RESOLVE) {}
+
+ char *get_type_name() { return "mds_resolve"; }
+
+ void print(ostream& out) {
+ out << "mds_resolve(" << subtrees.size()
+ << "+" << ambiguous_imports.size()
+ << " subtrees +" << slave_requests.size() << " slave requests)";
+ }
+
+ void add_subtree(dirfrag_t im) {
+ subtrees[im].clear();
+ }
+ void add_subtree_bound(dirfrag_t im, dirfrag_t ex) {
+ subtrees[im].push_back(ex);
+ }
+
+ void add_ambiguous_import(dirfrag_t im, const list<dirfrag_t>& m) {
+ ambiguous_imports[im] = m;
+ }
+
+ void add_slave_request(metareqid_t reqid) {
+ slave_requests.push_back(reqid);
+ }
+
+ void encode_payload() {
+ ::_encode(subtrees, payload);
+ ::_encode(ambiguous_imports, payload);
+ ::_encode(slave_requests, payload);
+ }
+ void decode_payload() {
+ int off = 0;
+ ::_decode(subtrees, payload, off);
+ ::_decode(ambiguous_imports, payload, off);
+ ::_decode(slave_requests, payload, off);
+ }
+};
+
+#endif
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation. See file COPYING.
+ *
+ */
+
+#ifndef __MMDSRESOLVEACK_H
+#define __MMDSRESOLVEACK_H
+
+#include "msg/Message.h"
+
+#include "include/types.h"
+
+
+class MMDSResolveAck : public Message {
+ public:
+ list<metareqid_t> commit;
+ list<metareqid_t> abort;
+
+ MMDSResolveAck() : Message(MSG_MDS_RESOLVEACK) {}
+
+ char *get_type_name() { return "resolve_ack"; }
+ /*void print(ostream& out) {
+ out << "resolve_ack.size()
+ << "+" << ambiguous_imap.size()
+ << " imports +" << slave_requests.size() << " slave requests)";
+ }
+ */
+
+ void add_commit(metareqid_t r) {
+ commit.push_back(r);
+ }
+ void add_abort(metareqid_t r) {
+ abort.push_back(r);
+ }
+
+ void encode_payload() {
+ ::_encode(commit, payload);
+ ::_encode(abort, payload);
+ }
+ void decode_payload() {
+ int off = 0;
+ ::_decode(commit, payload, off);
+ ::_decode(abort, payload, off);
+ }
+};
+
+#endif
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
+ *
+ * This is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License version 2.1, as published by the Free Software
+ * Foundation. See file COPYING.
+ *
+ */
+
+
+#ifndef __MMDSSLAVEREQUEST_H
+#define __MMDSSLAVEREQUEST_H
+
+#include "msg/Message.h"
+#include "mds/mdstypes.h"
+#include "include/encodable.h"
+
+class MMDSSlaveRequest : public Message {
+ public:
+ static const int OP_XLOCK = 1;
+ static const int OP_XLOCKACK = -1;
+ static const int OP_UNXLOCK = 2;
+ static const int OP_AUTHPIN = 3;
+ static const int OP_AUTHPINACK = -3;
+
+ static const int OP_LINKPREP = 4;
+ static const int OP_UNLINKPREP = 5;
+ static const int OP_LINKPREPACK = -4;
+
+ static const int OP_RENAMEPREP = 7;
+ static const int OP_RENAMEPREPACK = -7;
+
+ static const int OP_RENAMEGETINODE = 8;
+ static const int OP_RENAMEGETINODEACK = -8;
+
+ static const int OP_FINISH = 17;
+
+ static const int OP_ABORT = 20; // used for recovery only
+ //static const int OP_COMMIT = 21; // used for recovery only
+
+
+ const static char *get_opname(int o) {
+ switch (o) {
+ case OP_XLOCK: return "xlock";
+ case OP_XLOCKACK: return "xlock_ack";
+ case OP_UNXLOCK: return "unxlock";
+ case OP_AUTHPIN: return "authpin";
+ case OP_AUTHPINACK: return "authpin_ack";
+
+ case OP_LINKPREP: return "link_prep";
+ case OP_LINKPREPACK: return "link_prep_ack";
+ case OP_UNLINKPREP: return "unlink_prep";
+
+ case OP_RENAMEPREP: return "rename_prep";
+ case OP_RENAMEPREPACK: return "rename_prep_ack";
+ case OP_RENAMEGETINODE: return "rename_get_inode";
+ case OP_RENAMEGETINODEACK: return "rename_get_inode_ack";
+
+ case OP_FINISH: return "finish"; // commit
+ case OP_ABORT: return "abort";
+ //case OP_COMMIT: return "commit";
+
+ default: assert(0); return 0;
+ }
+ }
+
+ private:
+ metareqid_t reqid;
+ char op;
+
+ // for locking
+ char lock_type; // lock object type
+ MDSCacheObjectInfo object_info;
+
+ // for authpins
+ list<MDSCacheObjectInfo> authpins;
+
+ public:
+ // for rename prep
+ string srcdnpath;
+ string destdnpath;
+ set<int> srcdn_replicas;
+ bufferlist inode_export;
+ version_t inode_export_v;
+ utime_t now;
+
+ bufferlist stray; // stray dir + dentry
+
+public:
+ metareqid_t get_reqid() { return reqid; }
+ int get_op() { return op; }
+ bool is_reply() { return op < 0; }
+
+ int get_lock_type() { return lock_type; }
+ MDSCacheObjectInfo &get_object_info() { return object_info; }
+
+ list<MDSCacheObjectInfo>& get_authpins() { return authpins; }
+
+ void set_lock_type(int t) { lock_type = t; }
+
+ // ----
+ MMDSSlaveRequest() : Message(MSG_MDS_SLAVE_REQUEST) { }
+ MMDSSlaveRequest(metareqid_t ri, int o) :
+ Message(MSG_MDS_SLAVE_REQUEST),
+ reqid(ri), op(o) { }
+ void encode_payload() {
+ ::_encode(reqid, payload);
+ ::_encode(op, payload);
+ ::_encode(lock_type, payload);
+ object_info._encode(payload);
+ ::_encode_complex(authpins, payload);
+ ::_encode(srcdnpath, payload);
+ ::_encode(destdnpath, payload);
+ ::_encode(srcdn_replicas, payload);
+ ::_encode(now, payload);
+ ::_encode(inode_export, payload);
+ ::_encode(inode_export_v, payload);
+ ::_encode(stray, payload);
+ }
+ void decode_payload() {
+ int off = 0;
+ ::_decode(reqid, payload, off);
+ ::_decode(op, payload, off);
+ ::_decode(lock_type, payload, off);
+ object_info._decode(payload, off);
+ ::_decode_complex(authpins, payload, off);
+ ::_decode(srcdnpath, payload, off);
+ ::_decode(destdnpath, payload, off);
+ ::_decode(srcdn_replicas, payload, off);
+ ::_decode(now, payload, off);
+ ::_decode(inode_export, payload, off);
+ ::_decode(inode_export_v, payload, off);
+ ::_decode(stray, payload, off);
+ }
+
+ char *get_type_name() { return "slave_request"; }
+ void print(ostream& out) {
+ out << "slave_request(" << reqid
+ << " " << get_opname(op)
+ << ")";
+ }
+
+};
+
+#endif
}
- MOSDMap() :
- Message(MSG_OSD_MAP) {}
- MOSDMap(OSDMap *oc) :
- Message(MSG_OSD_MAP) {
+ MOSDMap() : Message(MSG_OSD_MAP) { }
+ MOSDMap(OSDMap *oc) : Message(MSG_OSD_MAP) {
oc->encode(maps[oc->get_epoch()]);
}
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MRENAME_H
-#define __MRENAME_H
-
-class MRename : public Message {
- inodeno_t srcdirino;
- string srcname;
- inodeno_t destdirino;
- string destname;
- int initiator;
-
- bufferlist inode_state;
-
- public:
- int get_initiator() { return initiator; }
- inodeno_t get_srcdirino() { return srcdirino; }
- string& get_srcname() { return srcname; }
- inodeno_t get_destdirino() { return destdirino; }
- string& get_destname() { return destname; }
- bufferlist& get_inode_state() { return inode_state; }
-
- MRename() {}
- MRename(int initiator,
- inodeno_t srcdirino,
- const string& srcname,
- inodeno_t destdirino,
- const string& destname,
- bufferlist& inode_state) :
- Message(MSG_MDS_RENAME) {
- this->initiator = initiator;
- this->srcdirino = srcdirino;
- this->srcname = srcname;
- this->destdirino = destdirino;
- this->destname = destname;
- this->inode_state.claim( inode_state );
- }
- virtual char *get_type_name() { return "Rn";}
-
- virtual void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(initiator), (char*)&initiator);
- off += sizeof(initiator);
- payload.copy(off, sizeof(srcdirino), (char*)&srcdirino);
- off += sizeof(srcdirino);
- payload.copy(off, sizeof(destdirino), (char*)&destdirino);
- off += sizeof(destdirino);
- _decode(srcname, payload, off);
- _decode(destname, payload, off);
- size_t len;
- payload.copy(off, sizeof(len), (char*)&len);
- off += sizeof(len);
- inode_state.substr_of(payload, off, len);
- off += len;
- }
- virtual void encode_payload() {
- payload.append((char*)&initiator,sizeof(initiator));
- payload.append((char*)&srcdirino,sizeof(srcdirino));
- payload.append((char*)&destdirino,sizeof(destdirino));
- _encode(srcname, payload);
- _encode(destname, payload);
- size_t len = inode_state.length();
- payload.append((char*)&len, sizeof(len));
- payload.claim_append(inode_state);
- }
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MRENAMEACK_H
-#define __MRENAMEACK_H
-
-/* FIXME: relateive to dn, not inode */
-
-class MRenameAck : public Message {
- inodeno_t ino;
-
- public:
- inodeno_t get_ino() { return ino; }
-
- MRenameAck() {}
- MRenameAck(inodeno_t ino) :
- Message(MSG_MDS_RENAMEACK) {
- this->ino = ino;
- }
- virtual char *get_type_name() { return "RnAck";}
-
- virtual void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(ino), (char*)&ino);
- off += sizeof(ino);
- }
- virtual void encode_payload() {
- payload.append((char*)&ino,sizeof(ino));
- }
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MRENAMENOTIFY_H
-#define __MRENAMENOTIFY_H
-
-class MRenameNotify : public Message {
- inodeno_t ino;
- inodeno_t srcdirino;
- string srcname;
- inodeno_t destdirino;
- string destname;
- string destdirpath;
- int srcauth;
-
- public:
- inodeno_t get_ino() { return ino; }
- inodeno_t get_srcdirino() { return srcdirino; }
- string& get_srcname() { return srcname; }
- inodeno_t get_destdirino() { return destdirino; }
- string& get_destname() { return destname; }
- string& get_destdirpath() { return destdirpath; }
- int get_srcauth() { return srcauth; }
-
- MRenameNotify() {}
- MRenameNotify(inodeno_t ino,
- inodeno_t srcdirino,
- const string& srcname,
- inodeno_t destdirino,
- const string& destdirpath,
- const string& destname,
- int srcauth
- ) :
- Message(MSG_MDS_RENAMENOTIFY) {
- this->ino = ino;
- this->srcdirino = srcdirino;
- this->srcname = srcname;
- this->destdirino = destdirino;
- this->destname = destname;
- this->destdirpath = destdirpath;
- this->srcauth = srcauth;
- }
- virtual char *get_type_name() { return "Rnot";}
-
- virtual void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(ino), (char*)&ino);
- off += sizeof(ino);
- payload.copy(off, sizeof(srcdirino), (char*)&srcdirino);
- off += sizeof(srcdirino);
- payload.copy(off, sizeof(destdirino), (char*)&destdirino);
- off += sizeof(destdirino);
- ::_decode(srcname, payload, off);
- ::_decode(destname, payload, off);
- ::_decode(destdirpath, payload, off);
- payload.copy(off, sizeof(srcauth), (char*)&srcauth);
- off += sizeof(srcauth);
- }
- virtual void encode_payload() {
- payload.append((char*)&ino,sizeof(ino));
- payload.append((char*)&srcdirino,sizeof(srcdirino));
- payload.append((char*)&destdirino,sizeof(destdirino));
- ::_encode(srcname, payload);
- ::_encode(destname, payload);
- ::_encode(destdirpath, payload);
- payload.append((char*)&srcauth, sizeof(srcauth));
- }
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MRENAMENOTIFYACK_H
-#define __MRENAMENOTIFYACK_H
-
-class MRenameNotifyAck : public Message {
- inodeno_t ino;
-
- public:
- inodeno_t get_ino() { return ino; }
-
- MRenameNotifyAck() {}
- MRenameNotifyAck(inodeno_t ino) :
- Message(MSG_MDS_RENAMENOTIFYACK) {
- this->ino = ino;
- }
- virtual char *get_type_name() { return "RnotA";}
-
- virtual void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(ino), (char*)&ino);
- off += sizeof(ino);
- }
- virtual void encode_payload() {
- payload.append((char*)&ino,sizeof(ino));
- }
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MRENAMEPREP_H
-#define __MRENAMEPREP_H
-
-class MRenamePrep : public Message {
- inodeno_t srcdirino;
- string srcname;
- string srcpath;
- inodeno_t destdirino;
- string destname;
- string destpath;
- int initiator;
- int srcauth;
-
- public:
- int get_initiator() { return initiator; }
- inodeno_t get_srcdirino() { return srcdirino; }
- string& get_srcname() { return srcname; }
- string& get_srcpath() { return srcpath; }
- int get_srcauth() { return srcauth; }
- inodeno_t get_destdirino() { return destdirino; }
- string& get_destname() { return destname; }
- string& get_destpath() { return destpath; }
-
- MRenamePrep() {}
- MRenamePrep(int initiator,
- inodeno_t srcdirino,
- const string& srcname,
- const string& srcpath,
- inodeno_t destdirino,
- const string& destname,
- const string& destpath,
- int srcauth) :
- Message(MSG_MDS_RENAMEPREP) {
- this->initiator = initiator;
- this->srcdirino = srcdirino;
- this->srcname = srcname;
- this->srcpath = srcpath;
- this->destdirino = destdirino;
- this->destname = destname;
- this->destpath = destpath;
- this->srcauth = srcauth;
- }
- virtual char *get_type_name() { return "RnP";}
-
- virtual void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(initiator), (char*)&initiator);
- off += sizeof(initiator);
- payload.copy(off, sizeof(srcdirino), (char*)&srcdirino);
- off += sizeof(srcdirino);
- payload.copy(off, sizeof(destdirino), (char*)&destdirino);
- off += sizeof(destdirino);
- ::_decode(srcname, payload, off);
- ::_decode(srcpath, payload, off);
- ::_decode(destname, payload, off);
- ::_decode(destpath, payload, off);
- payload.copy(off, sizeof(srcauth), (char*)&srcauth);
- off += sizeof(srcauth);
- }
- virtual void encode_payload() {
- payload.append((char*)&initiator,sizeof(initiator));
- payload.append((char*)&srcdirino,sizeof(srcdirino));
- payload.append((char*)&destdirino,sizeof(destdirino));
- ::_encode(srcname, payload);
- ::_encode(srcpath, payload);
- ::_encode(destname, payload);
- ::_encode(destpath, payload);
- payload.append((char*)&srcauth, sizeof(srcauth));
- }
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MRENAMEREQ_H
-#define __MRENAMEREQ_H
-
-class MRenameReq : public Message {
- int initiator;
- inodeno_t srcdirino;
- string srcname;
- inodeno_t destdirino;
- string destname;
- string destpath;
- int destauth;
-
- public:
- int get_initiator() { return initiator; }
- inodeno_t get_srcdirino() { return srcdirino; }
- string& get_srcname() { return srcname; }
- inodeno_t get_destdirino() { return destdirino; }
- string& get_destname() { return destname; }
- string& get_destpath() { return destpath; }
- int get_destauth() { return destauth; }
-
- MRenameReq() {}
- MRenameReq(int initiator,
- inodeno_t srcdirino,
- const string& srcname,
- inodeno_t destdirino,
- const string& destname,
- const string& destpath,
- int destauth) :
- Message(MSG_MDS_RENAMEREQ) {
- this->initiator = initiator;
- this->srcdirino = srcdirino;
- this->srcname = srcname;
- this->destdirino = destdirino;
- this->destname = destname;
- this->destpath = destpath;
- this->destauth = destauth;
- }
- virtual char *get_type_name() { return "RnReq";}
-
- virtual void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(initiator), (char*)&initiator);
- off += sizeof(initiator);
- payload.copy(off, sizeof(srcdirino), (char*)&srcdirino);
- off += sizeof(srcdirino);
- payload.copy(off, sizeof(destdirino), (char*)&destdirino);
- off += sizeof(destdirino);
- ::_decode(srcname, payload, off);
- ::_decode(destname, payload, off);
- ::_decode(destpath, payload, off);
- payload.copy(off, sizeof(destauth), (char*)&destauth);
- off += sizeof(destauth);
- }
- virtual void encode_payload() {
- payload.append((char*)&initiator,sizeof(initiator));
- payload.append((char*)&srcdirino,sizeof(srcdirino));
- payload.append((char*)&destdirino,sizeof(destdirino));
- ::_encode(srcname, payload);
- ::_encode(destname, payload);
- ::_encode(destpath, payload);
- payload.append((char*)&destauth, sizeof(destauth));
- }
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MRENAMEWARNING_H
-#define __MRENAMEWARNING_H
-
-class MRenameWarning : public Message {
- inodeno_t ino;
-
- public:
- inodeno_t get_ino() { return ino; }
-
- MRenameWarning() {}
- MRenameWarning(inodeno_t ino) :
- Message(MSG_MDS_RENAMEWARNING) {
- this->ino = ino;
- }
- virtual char *get_type_name() { return "RnW";}
-
- virtual void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(ino), (char*)&ino);
- off += sizeof(ino);
- }
- virtual void encode_payload() {
- payload.append((char*)&ino,sizeof(ino));
- }
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MUNHASHDIR_H
-#define __MUNHASHDIR_H
-
-#include "msg/Message.h"
-
-class MUnhashDir : public Message {
- inodeno_t ino;
-
- public:
- inodeno_t get_ino() { return ino; }
-
- MUnhashDir() {}
- MUnhashDir(inodeno_t ino) :
- Message(MSG_MDS_UNHASHDIR) {
- this->ino = ino;
- }
- virtual char *get_type_name() { return "UH"; }
-
- virtual void decode_payload() {
- payload.copy(0, sizeof(ino), (char*)&ino);
- }
- virtual void encode_payload() {
- payload.append((char*)&ino, sizeof(ino));
- }
-
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MUNHASHDIRACK_H
-#define __MUNHASHDIRACK_H
-
-#include "msg/Message.h"
-
-class MUnhashDirAck : public Message {
- inodeno_t ino;
- bufferlist state;
- int nden;
-
- public:
- MUnhashDirAck() {}
- MUnhashDirAck(inodeno_t ino, bufferlist& bl, int nden) :
- Message(MSG_MDS_UNHASHDIRACK) {
- this->ino = ino;
- state.claim(bl);
- this->nden = nden;
- }
- virtual char *get_type_name() { return "UHaA"; }
-
- inodeno_t get_ino() { return ino; }
- bufferlist& get_state() { return state; }
- bufferlist* get_state_ptr() { return &state; }
- int get_nden() { return nden; }
-
- //void set_nden(int n) { nden = n; }
- //void inc_nden() { nden++; }
-
- void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(ino), (char*)&ino);
- off += sizeof(ino);
- payload.copy(off, sizeof(nden), (char*)&nden);
- off += sizeof(nden);
-
- size_t len;
- payload.copy(off, sizeof(len), (char*)&len);
- off += sizeof(len);
- state.substr_of(payload, off, len);
- }
- void encode_payload() {
- payload.append((char*)&ino, sizeof(ino));
- payload.append((char*)&nden, sizeof(nden));
- size_t size = state.length();
- payload.append((char*)&size, sizeof(size));
- payload.claim_append(state);
- }
-
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MUNHASHDIRNOTIFY_H
-#define __MUNHASHDIRNOTIFY_H
-
-#include "msg/Message.h"
-
-class MUnhashDirNotify : public Message {
- inodeno_t ino;
- //int peer;
-
- public:
- inodeno_t get_ino() { return ino; }
- //int get_peer() { return peer; }
-
- MUnhashDirNotify() {}
- MUnhashDirNotify(inodeno_t ino/*, int peer*/) :
- Message(MSG_MDS_UNHASHDIRNOTIFY) {
- this->ino = ino;
- //this->peer = peer;
- }
- virtual char *get_type_name() { return "UHN"; }
-
- virtual void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(ino), (char*)&ino);
- off += sizeof(ino);
- //payload.copy(off, sizeof(peer), (char*)&peer);
- //off += sizeof(peer);
- }
- virtual void encode_payload() {
- payload.append((char*)&ino, sizeof(ino));
- //payload.append((char*)&peer, sizeof(peer));
- }
-
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MUNHASHDIRNOTIFYACK_H
-#define __MUNHASHDIRNOTIFYACK_H
-
-#include "msg/Message.h"
-
-class MUnhashDirNotifyAck : public Message {
- inodeno_t ino;
-
- public:
- inodeno_t get_ino() { return ino; }
-
- MUnhashDirNotifyAck() {}
- MUnhashDirNotifyAck(inodeno_t ino) :
- Message(MSG_MDS_UNHASHDIRNOTIFYACK) {
- this->ino = ino;
- }
- virtual char *get_type_name() { return "UHNa"; }
-
- virtual void decode_payload() {
- payload.copy(0, sizeof(ino), (char*)&ino);
- }
- virtual void encode_payload() {
- payload.append((char*)&ino, sizeof(ino));
- }
-
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MUNHASHDIRPREP_H
-#define __MUNHASHDIRPREP_H
-
-#include "msg/Message.h"
-
-class MUnhashDirPrep : public Message {
- inodeno_t ino;
-
- public:
- inodeno_t get_ino() { return ino; }
-
- MUnhashDirPrep() {}
- MUnhashDirPrep(inodeno_t ino) :
- Message(MSG_MDS_UNHASHDIRPREP) {
- this->ino = ino;
- }
- virtual char *get_type_name() { return "UHP"; }
-
- virtual void decode_payload() {
- payload.copy(0, sizeof(ino), (char*)&ino);
- }
- virtual void encode_payload() {
- payload.append((char*)&ino, sizeof(ino));
- }
-
-};
-
-#endif
+++ /dev/null
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2004-2006 Sage Weil <sage@newdream.net>
- *
- * This is free software; you can redistribute it and/or
- * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software
- * Foundation. See file COPYING.
- *
- */
-
-
-#ifndef __MUNHASHDIRPREPACK_H
-#define __MUNHASHDIRPREPACK_H
-
-#include "msg/Message.h"
-#include "mds/CInode.h"
-#include "include/types.h"
-
-class MUnhashDirPrepAck : public Message {
- inodeno_t ino;
- bool assim;
-
- // subdir dentry names + inodes
- map<string,CInodeDiscover*> inodes;
-
- public:
- inodeno_t get_ino() { return ino; }
- map<string,CInodeDiscover*>& get_inodes() { return inodes; }
-
- bool did_assim() { return assim; }
- void mark_assim() { assert(!assim); assim = true; }
-
- MUnhashDirPrepAck() : assim(false) { }
- MUnhashDirPrepAck(inodeno_t ino) :
- Message(MSG_MDS_UNHASHDIRPREPACK),
- assim(false) {
- this->ino = ino;
- }
- ~MUnhashDirPrepAck() {
- for (map<string,CInodeDiscover*>::iterator it = inodes.begin();
- it != inodes.end();
- it++)
- delete it->second;
- }
-
-
- virtual char *get_type_name() { return "HP"; }
-
- void add_inode(const string& dentry, CInodeDiscover *in) {
- inodes[dentry] = in;
- }
-
- void decode_payload() {
- int off = 0;
- payload.copy(off, sizeof(ino), (char*)&ino);
- off += sizeof(ino);
-
- // inodes
- int ni;
- payload.copy(off, sizeof(int), (char*)&ni);
- off += sizeof(int);
- for (int i=0; i<ni; i++) {
- // dentry
- string dname;
- _decode(dname, payload, off);
-
- // inode
- CInodeDiscover *in = new CInodeDiscover;
- in->_decode(payload, off);
-
- inodes[dname] = in;
- }
- }
-
- virtual void encode_payload() {
- payload.append((char*)&ino, sizeof(ino));
-
- // inodes
- int ni = inodes.size();
- payload.append((char*)&ni, sizeof(int));
- for (map<string,CInodeDiscover*>::iterator iit = inodes.begin();
- iit != inodes.end();
- iit++) {
- _encode(iit->first, payload); // dentry
- iit->second->_encode(payload); // inode
- }
- }
-};
-
-#endif
// reply with latest mds, osd maps
mon->mdsmon->send_latest(to);
- mon->osdmon->send_latest(0, to);
+ mon->osdmon->send_latest(to);
delete m;
}
void MDSMonitor::print_map(MDSMap &m)
{
- dout(7) << "print_map epoch " << m.get_epoch() << " num_mds " << g_conf.num_mds << endl;
+ dout(7) << "print_map epoch " << m.get_epoch() << " target_num " << m.target_num << endl;
entity_inst_t blank;
set<int> all;
m.get_mds_set(all);
void MDSMonitor::create_initial()
{
dout(10) << "create_initial" << endl;
+ pending_mdsmap.target_num = g_conf.num_mds;
pending_mdsmap.created = g_clock.now();
print_map(pending_mdsmap);
}
if (mon->is_leader() &&
g_conf.mon_stop_with_last_mds &&
mdsmap.get_epoch() > 1 &&
- mdsmap.get_num_up_or_failed_mds() == 0)
+ mdsmap.is_stopped())
mon->messenger->send_message(new MGenericMessage(MSG_SHUTDOWN),
mon->monmap->get_inst(mon->whoami));
}
// reply to beacon?
- if (state != MDSMap::STATE_OUT) {
+ if (state != MDSMap::STATE_STOPPED) {
last_beacon[from] = g_clock.now(); // note time
mon->messenger->send_message(new MMDSBeacon(m->get_mds_inst(), state, seq),
m->get_mds_inst());
int state = m->get_state();
version_t seq = m->get_seq();
+ assert(state != mdsmap.get_state(from));
+
// boot?
if (state == MDSMap::STATE_BOOT) {
// assign a name.
if (mdsmap.is_failed(from)) {
dout(10) << "mds_beacon boot: mds" << from << " was failed, replaying" << endl;
state = MDSMap::STATE_REPLAY;
- } else if (mdsmap.is_out(from)) {
- dout(10) << "mds_beacon boot: mds" << from << " was out, starting" << endl;
+ } else if (mdsmap.is_stopped(from)) {
+ dout(10) << "mds_beacon boot: mds" << from << " was stopped, starting" << endl;
state = MDSMap::STATE_STARTING;
} else if (!mdsmap.have_inst(from) || mdsmap.get_inst(from) != m->get_mds_inst()) {
dout(10) << "mds_beacon boot: mds" << from << " is someone else" << endl;
dout(10) << "mds_beacon boot: assigned new mds" << from << endl;
state = MDSMap::STATE_CREATING;
break;
- } else if (pending_mdsmap.is_out(from)) {
- dout(10) << "mds_beacon boot: assigned out mds" << from << endl;
+ } else if (pending_mdsmap.is_stopped(from)) {
+ dout(10) << "mds_beacon boot: assigned stopped mds" << from << endl;
state = MDSMap::STATE_STARTING;
break;
}
}
}
- assert(state != MDSMap::STATE_BOOT);
-
+ assert(state == MDSMap::STATE_CREATING ||
+ state == MDSMap::STATE_STARTING ||
+ state == MDSMap::STATE_REPLAY);
+
// put it in the map.
pending_mdsmap.mds_inst[from].addr = m->get_mds_inst().addr;
pending_mdsmap.mds_inst[from].name = MSG_ADDR_MDS(from);
// someone (new) has joined the cluster.
pending_mdsmap.same_inst_since = pending_mdsmap.epoch;
- // if degraded, starting -> standby
- if (pending_mdsmap.is_degraded() &&
- state == MDSMap::STATE_STARTING) {
- dout(10) << "mds_beacon boot: cluster degraded, mds" << from << " will be standby" << endl;
- state = MDSMap::STATE_STANDBY;
- }
+ // reset the beacon timer
+ last_beacon[from] = g_clock.now();
}
- // if creating -> active, go to standby instead
+ // created?
if (state == MDSMap::STATE_ACTIVE &&
mdsmap.is_creating(from)) {
pending_mdsmap.mds_created.insert(from);
dout(10) << "mds_beacon created mds" << from << endl;
-
- if (mdsmap.is_degraded()) {
- dout(10) << "mds_beacon cluster degraded, marking mds" << from << " as standby" << endl;
- state = MDSMap::STATE_STANDBY;
- }
+ }
+
+ // if starting|creating and degraded|full, go to standby
+ if ((state == MDSMap::STATE_STARTING ||
+ state == MDSMap::STATE_CREATING ||
+ mdsmap.is_starting(from) ||
+ mdsmap.is_creating(from)) &&
+ (pending_mdsmap.is_degraded() ||
+ pending_mdsmap.is_full())) {
+ dout(10) << "mds_beacon cluster degraded|full, mds" << from << " will be standby" << endl;
+ state = MDSMap::STATE_STANDBY;
}
// update the map
<< endl;
// did someone leave the cluster?
- if (state == MDSMap::STATE_OUT &&
- mdsmap.mds_state[from] != MDSMap::STATE_OUT)
+ if (state == MDSMap::STATE_STOPPED &&
+ !mdsmap.is_stopped(from))
pending_mdsmap.same_inst_since = pending_mdsmap.epoch;
-
+
// change the state
pending_mdsmap.mds_state[from] = state;
if (pending_mdsmap.is_up(from))
pending_mdsmap.mds_state_seq[from] = seq;
else
pending_mdsmap.mds_state_seq.erase(from);
-
-
+
dout(7) << "pending map now:" << endl;
print_map(pending_mdsmap);
{
if (m->get_state() == MDSMap::STATE_BOOT) {
dout(10) << "_updated (booted) mds" << from << " " << *m << endl;
- mon->osdmon->send_latest(0, mdsmap.get_inst(from));
+ mon->osdmon->send_latest(mdsmap.get_inst(from));
} else {
dout(10) << "_updated mds" << from << " " << *m << endl;
}
+ if (m->get_state() == MDSMap::STATE_STOPPED) {
+ // send the map manually (they're out of the map, so they won't get it automatic)
+ send_latest(m->get_mds_inst());
+ }
delete m;
}
bool MDSMonitor::handle_command(MMonCommand *m)
{
- int r = -1;
- string rs = "unrecognized command";
+ int r = -EINVAL;
stringstream ss;
if (m->cmd.size() > 1) {
if (mdsmap.is_active(who)) {
r = 0;
ss << "telling mds" << who << " to stop";
- getline(ss,rs);
-
pending_mdsmap.mds_state[who] = MDSMap::STATE_STOPPING;
-
} else {
+ r = -EEXIST;
ss << "mds" << who << " not active (" << mdsmap.get_state_name(mdsmap.get_state(who)) << ")";
- getline(ss,rs);
}
}
- /*
- else if (m->cmd[1] == "setnum" && m->cmd.size() > 2) {
- g_conf.num_mds = atoi(m->cmd[2].c_str());
- ss << "g_conf.num_mds = " << g_conf.num_mds << endl;
- getline(ss,rs);
- print_map();
+ else if (m->cmd[1] == "set_target_num" && m->cmd.size() > 2) {
+ pending_mdsmap.target_num = atoi(m->cmd[2].c_str());
+ r = 0;
+ ss << "target_num = " << pending_mdsmap.target_num << endl;
}
- */
}
-
+ if (r == -EINVAL) {
+ ss << "unrecognized command";
+ }
+
// reply
+ string rs;
+ getline(ss,rs);
mon->messenger->send_message(new MMonCommandAck(r, rs), m->get_source_inst());
delete m;
return r >= 0;
// failure!
int newstate;
switch (mdsmap.get_state(*p)) {
+ case MDSMap::STATE_STANDBY:
+ if (mdsmap.has_created(*p))
+ newstate = MDSMap::STATE_STOPPED;
+ else
+ newstate = MDSMap::STATE_DNE;
+ break;
+
case MDSMap::STATE_CREATING:
// didn't finish creating
newstate = MDSMap::STATE_DNE;
break;
- case MDSMap::STATE_STANDBY:
- if (mdsmap.has_created(*p))
- newstate = MDSMap::STATE_OUT;
- else
- newstate = MDSMap::STATE_DNE;
+ case MDSMap::STATE_STARTING:
+ newstate = MDSMap::STATE_STOPPED;
break;
case MDSMap::STATE_REPLAY:
newstate = MDSMap::STATE_FAILED;
break;
- case MDSMap::STATE_STARTING:
- case MDSMap::STATE_STOPPED:
- newstate = MDSMap::STATE_OUT;
- break;
-
default:
assert(0);
}
// tell a random osd
int osd = rand() % g_conf.num_osd;
- send_latest(0, osdmap.get_inst(osd));
+ send_latest(osdmap.get_inst(osd));
}
}
propose_pending();
- send_latest(0, osdmap.get_inst(r)); // after
+ send_latest(osdmap.get_inst(r)); // after
}
pending_inc.mon_epoch = mon->mon_epoch;
// tell me about it
- for (map<int,entity_inst_t>::iterator i = pending_inc.new_up.begin();
- i != pending_inc.new_up.end();
- i++) {
- dout(0) << " osd" << i->first << " UP " << i->second << endl;
- derr(0) << " osd" << i->first << " UP " << i->second << endl;
- }
for (map<int,entity_inst_t>::iterator i = pending_inc.new_down.begin();
i != pending_inc.new_down.end();
i++) {
derr(0) << " osd" << i->first << " DOWN " << i->second << endl;
mon->messenger->mark_down(i->second.addr);
}
- for (list<int>::iterator i = pending_inc.new_in.begin();
- i != pending_inc.new_in.end();
- i++) {
- dout(0) << " osd" << *i << " IN" << endl;
- derr(0) << " osd" << *i << " IN" << endl;
+ for (map<int,entity_inst_t>::iterator i = pending_inc.new_up.begin();
+ i != pending_inc.new_up.end();
+ i++) {
+ dout(0) << " osd" << i->first << " UP " << i->second << endl;
+ derr(0) << " osd" << i->first << " UP " << i->second << endl;
}
for (list<int>::iterator i = pending_inc.new_out.begin();
i != pending_inc.new_out.end();
dout(0) << " osd" << *i << " OUT" << endl;
derr(0) << " osd" << *i << " OUT" << endl;
}
+ for (list<int>::iterator i = pending_inc.new_in.begin();
+ i != pending_inc.new_in.end();
+ i++) {
+ dout(0) << " osd" << *i << " IN" << endl;
+ derr(0) << " osd" << *i << " IN" << endl;
+ }
// encode
assert(paxos->get_version() + 1 == pending_inc.epoch);
void OSDMonitor::_reported_failure(MOSDFailure *m)
{
dout(7) << "_reported_failure on " << m->get_failed() << ", telling " << m->get_from() << endl;
- send_latest(m->get_epoch(), m->get_from());
+ send_latest(m->get_from(), m->get_epoch());
}
void OSDMonitor::_booted(MOSDBoot *m)
{
dout(7) << "_booted " << m->inst << endl;
- send_latest(m->sb.current_epoch, m->inst);
+ send_latest(m->inst, m->sb.current_epoch);
delete m;
}
}
}
-void OSDMonitor::send_latest(epoch_t since, entity_inst_t who)
+void OSDMonitor::send_latest(entity_inst_t who, epoch_t since)
{
if (paxos->is_readable()) {
dout(5) << "send_latest to " << who << " now" << endl;
- if (since)
- send_incremental(since, who);
- else
+ if (since == (epoch_t)(-1))
send_full(who);
+ else
+ send_incremental(since, who);
} else {
dout(5) << "send_latest to " << who << " later" << endl;
awaiting_map[who.name].first = who;
e--) {
bufferlist bl;
if (mon->store->get_bl_sn(bl, "osdmap", e) > 0) {
- dout(20) << "send_incremental inc " << e << endl;
+ dout(20) << "send_incremental inc " << e << " " << bl.length() << " bytes" << endl;
m->incremental_maps[e] = bl;
}
else if (mon->store->get_bl_sn(bl, "osdmap_full", e) > 0) {
void mark_all_down();
- void send_latest(epoch_t since, entity_inst_t i);
+ void send_latest(entity_inst_t i, epoch_t since=(epoch_t)(-1));
void fake_osd_failure(int osd, bool down);
void fake_osdmap_update();
#include "config.h"
#undef dout
-#define derr(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cerr << g_clock.now() << " mon" << whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".paxos(" << machine_name << " " << get_statename(state) << " lc " << last_committed << ") "
-#define dout(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cout << g_clock.now() << " mon" << whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".paxos(" << machine_name << " " << get_statename(state) << " lc " << last_committed << ") "
+#define derr(l) if (l<=g_conf.debug || l<=g_conf.debug_paxos) cerr << g_clock.now() << " mon" << whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".paxos(" << machine_name << " " << get_statename(state) << " lc " << last_committed << ") "
+#define dout(l) if (l<=g_conf.debug || l<=g_conf.debug_paxos) cout << g_clock.now() << " mon" << whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".paxos(" << machine_name << " " << get_statename(state) << " lc " << last_committed << ") "
void Paxos::init()
#include "config.h"
#undef dout
-#define dout(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cout << g_clock.now() << " mon" << mon->whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".paxosservice(" << get_paxos_name(paxos->machine_id) << ") "
+#define dout(l) if (l<=g_conf.debug || l<=g_conf.debug_paxos) cout << g_clock.now() << " mon" << mon->whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << ".paxosservice(" << get_paxos_name(paxos->machine_id) << ") "
//#define derr(l) if (l<=g_conf.debug || l<=g_conf.debug_mon) cerr << g_clock.now() << " mon" << mon->whoami << (mon->is_starting() ? (const char*)"(starting)":(mon->is_leader() ? (const char*)"(leader)":(mon->is_peon() ? (const char*)"(peon)":(const char*)"(?\?)"))) << "." << get_paxos_name(paxos->machine_id) << " "
bool fm_shutdown = false;
pthread_t thread_id;
-
-
+extern std::map<entity_name_t,float> g_fake_kill_after; // in config.cc
+utime_t start_time;
+map<utime_t,entity_name_t> fail_queue;
+list<Message*> sent_to_failed_queue;
void *fakemessenger_thread(void *ptr)
{
+ start_time = g_clock.now();
+
lock.Lock();
while (1) {
if (fm_shutdown) break;
}
+// fake failure
+
// lame main looper
dout(18) << "do_loop top" << endl;
+ // fail_queue
+ while (!fail_queue.empty() &&
+ fail_queue.begin()->first < g_clock.now()) {
+ entity_name_t nm = fail_queue.begin()->second;
+ fail_queue.erase(fail_queue.begin());
+
+ dout(0) << "MUST FAKE KILL " << nm << endl;
+
+ for (map<entity_addr_t, FakeMessenger*>::iterator p = directory.begin();
+ p != directory.end();
+ ++p) {
+ if (p->second->get_myname() == nm) {
+ dout(0) << "FAKING FAILURE of " << nm << " at " << p->first << endl;
+ directory.erase(p);
+ p->second->failed = true;
+ break;
+ }
+ }
+ }
+
+ list<Message*> ls;
+ ls.swap(sent_to_failed_queue);
+ for (list<Message*>::iterator p = ls.begin();
+ p != ls.end();
+ ++p) {
+ Message *m = *p;
+ FakeMessenger *mgr = directory[m->get_source_addr()];
+ Dispatcher *dis = 0;
+ if (mgr) dis = mgr->get_dispatcher();
+ if (dis) {
+ dout(1) << "fail on " << *m
+ << " to " << m->get_dest() << " from " << m->get_source()
+ << ", passing back to sender." << endl;
+ dis->ms_handle_failure(m, m->get_dest_inst());
+ } else {
+ dout(1) << "fail on " << *m
+ << " to " << m->get_dest() << " from " << m->get_source()
+ << ", sender gone, dropping." << endl;
+ delete m;
+ }
+ }
+
// messages
map<entity_addr_t, FakeMessenger*>::iterator it = directory.begin();
while (it != directory.end()) {
FakeMessenger::FakeMessenger(entity_name_t me) : Messenger(me)
{
+ failed = false;
+
lock.Lock();
{
// assign rank
// add to directory
directory[ _myinst.addr ] = this;
+
+ // put myself in the fail queue?
+ if (g_fake_kill_after.count(me)) {
+ utime_t w = start_time;
+ w += g_fake_kill_after[me];
+ dout(0) << "will fake failure of " << me << " at " << w << endl;
+ fail_queue[w] = me;
+ }
}
lock.Unlock();
_myinst.name = m;
directory[_myinst.addr] = this;
+ // put myself in the fail queue?
+ if (g_fake_kill_after.count(m)) {
+ utime_t w = start_time;
+ w += g_fake_kill_after[m];
+ dout(0) << "will fake failure of " << m << " at " << w << endl;
+ fail_queue[w] = m;
+ }
+
}
for (map<entity_addr_t, FakeMessenger*>::iterator p = directory.begin();
p != directory.end();
++p) {
- dout(0) << "** have " << p->first << " to " << p->second << endl;
+ dout(20) << "** have " << p->first << " to " << p->second << endl;
}
- //assert(dm);
- delete m;
+
+ // do the failure callback
+ sent_to_failed_queue.push_back(m);
}
// wake up loop?
entity_inst_t _myinst;
public:
+ bool failed;
+
FakeMessenger(entity_name_t me);
~FakeMessenger();
return qlen;
}
+ void suicide() {
+ if (!failed) {
+ failed = true;
+ }
+ }
+
};
int fakemessenger_do_loop();
#include "messages/MClientReply.h"
#include "messages/MClientFileCaps.h"
+#include "messages/MMDSSlaveRequest.h"
+
#include "messages/MMDSGetMap.h"
#include "messages/MMDSMap.h"
#include "messages/MMDSBeacon.h"
-#include "messages/MMDSImportMap.h"
+#include "messages/MMDSResolve.h"
+#include "messages/MMDSResolveAck.h"
#include "messages/MMDSCacheRejoin.h"
//#include "messages/MMDSCacheRejoinAck.h"
#include "messages/MExportDirNotifyAck.h"
#include "messages/MExportDirFinish.h"
-#include "messages/MRenameWarning.h"
-#include "messages/MRenameNotify.h"
-#include "messages/MRenameNotifyAck.h"
-#include "messages/MRename.h"
-#include "messages/MRenamePrep.h"
-#include "messages/MRenameReq.h"
-#include "messages/MRenameAck.h"
#include "messages/MDentryUnlink.h"
#include "messages/MHeartbeat.h"
#include "messages/MAnchor.h"
-#include "messages/MInodeLink.h"
-#include "messages/MInodeLinkAck.h"
//#include "messages/MInodeUpdate.h"
#include "messages/MCacheExpire.h"
break;
// mds
+ case MSG_MDS_SLAVE_REQUEST:
+ m = new MMDSSlaveRequest;
+ break;
+
case MSG_MDS_GETMAP:
m = new MMDSGetMap();
break;
case MSG_MDS_BEACON:
m = new MMDSBeacon;
break;
- case MSG_MDS_IMPORTMAP:
- m = new MMDSImportMap;
+ case MSG_MDS_RESOLVE:
+ m = new MMDSResolve;
+ break;
+ case MSG_MDS_RESOLVEACK:
+ m = new MMDSResolveAck;
break;
case MSG_MDS_CACHEREJOIN:
m = new MMDSCacheRejoin;
break;
- case MSG_MDS_RENAMEWARNING:
- m = new MRenameWarning();
- break;
- case MSG_MDS_RENAMENOTIFY:
- m = new MRenameNotify();
- break;
- case MSG_MDS_RENAMENOTIFYACK:
- m = new MRenameNotifyAck();
- break;
- case MSG_MDS_RENAME:
- m = new MRename();
- break;
- case MSG_MDS_RENAMEPREP:
- m = new MRenamePrep();
- break;
- case MSG_MDS_RENAMEREQ:
- m = new MRenameReq();
- break;
- case MSG_MDS_RENAMEACK:
- m = new MRenameAck();
- break;
case MSG_MDS_DENTRYUNLINK:
m = new MDentryUnlink();
case MSG_MDS_ANCHOR:
m = new MAnchor();
- break;
-
- case MSG_MDS_INODELINK:
- m = new MInodeLink();
- break;
- case MSG_MDS_INODELINKACK:
- m = new MInodeLinkAck();
break;
/* case MSG_MDS_INODEUPDATE:
#define MSG_MDS_HEARTBEAT 104 // for mds load balancer
#define MSG_MDS_BEACON 105 // to monitor
-#define MSG_MDS_IMPORTMAP 106
-#define MSG_MDS_CACHEREJOIN 107
+#define MSG_MDS_RESOLVE 106
+#define MSG_MDS_RESOLVEACK 107
+
+#define MSG_MDS_CACHEREJOIN 108
#define MSG_MDS_DISCOVER 110
#define MSG_MDS_DISCOVERREPLY 111
#define MSG_MDS_ANCHOR 130
-#define MSG_MDS_INODELINK 140
-#define MSG_MDS_INODELINKACK 141
-#define MSG_MDS_INODEUNLINK 142
-#define MSG_MDS_INODEUNLINKACK 143
-
#define MSG_MDS_EXPORTDIRDISCOVER 149
#define MSG_MDS_EXPORTDIRDISCOVERACK 150
#define MSG_MDS_EXPORTDIRCANCEL 151
#define MSG_MDS_EXPORTDIRNOTIFYACK 159
#define MSG_MDS_EXPORTDIRFINISH 160
-
-#define MSG_MDS_HASHDIRDISCOVER 170
-#define MSG_MDS_HASHDIRDISCOVERACK 171
-#define MSG_MDS_HASHDIRPREP 172
-#define MSG_MDS_HASHDIRPREPACK 173
-#define MSG_MDS_HASHDIR 174
-#define MSG_MDS_HASHDIRACK 175
-#define MSG_MDS_HASHDIRNOTIFY 176
-
-#define MSG_MDS_HASHREADDIR 178
-#define MSG_MDS_HASHREADDIRREPLY 179
-
-#define MSG_MDS_UNHASHDIRPREP 180
-#define MSG_MDS_UNHASHDIRPREPACK 181
-#define MSG_MDS_UNHASHDIR 182
-#define MSG_MDS_UNHASHDIRACK 183
-#define MSG_MDS_UNHASHDIRNOTIFY 184
-#define MSG_MDS_UNHASHDIRNOTIFYACK 185
+#define MSG_MDS_SLAVE_REQUEST 170
#define MSG_MDS_DENTRYUNLINK 200
-#define MSG_MDS_RENAMEWARNING 300 // sent from src to bystanders
-#define MSG_MDS_RENAMENOTIFY 301 // sent from dest to bystanders
-#define MSG_MDS_RENAMENOTIFYACK 302 // sent back to src
-#define MSG_MDS_RENAMEACK 303 // sent from src to initiator, to xlock_finish
-
-#define MSG_MDS_RENAMEPREP 304 // sent from initiator to dest auth (if dir)
-#define MSG_MDS_RENAMEREQ 305 // sent from initiator (or dest if dir) to src auth
-#define MSG_MDS_RENAME 306 // sent from src to dest, includes inode
-
#define MSG_MDS_LOCK 500
#define MSG_MDS_SHUTDOWNSTART 900
// shutdown
virtual int shutdown() = 0;
+ virtual void suicide() = 0;
// send message
virtual void prepare_dest(const entity_addr_t& addr) {}
}
else {
// oh well. sending entity musta just shut down?
- assert(0);
delete q.front();
}
q.pop_front();
return 0;
}
+void Rank::EntityMessenger::suicide()
+{
+ dout(10) << "suicide " << get_myaddr() << endl;
+ shutdown();
+ // hmm, or exit(0)?
+}
void Rank::EntityMessenger::prepare_dest(const entity_addr_t& addr)
{
void reset_myname(entity_name_t m);
int shutdown();
+ void suicide();
void prepare_dest(const entity_addr_t& addr);
int send_message(Message *m, entity_inst_t dest,
int port=0, int fromport=0);
*
*/
+#define intabs(x) ((x) >= 0 ? (x):(-(x)))
#include <sys/stat.h>
#include <iostream>
#include "common/Timer.h"
-#define NUMMDS g_conf.num_mds
-#define NUMOSD g_conf.num_osd
-#define NUMCLIENT g_conf.num_client
-
class C_Test : public Context {
public:
void finish(int r) {
args.swap(nargs);
}
+ // stop on our own (by default)
+ g_conf.mon_stop_on_last_unmount = true;
+ g_conf.mon_stop_with_last_mds = true;
+
parse_config_options(args);
parse_syn_options(args);
+
+ //int start_mon = g_conf.num_mon > 0 ? g_conf.num_mon:0;
+ int start_mds = g_conf.num_mds > 0 ? g_conf.num_mds:0;
+ int start_osd = g_conf.num_osd > 0 ? g_conf.num_osd:0;
+ int start_client = g_conf.num_client > 0 ? g_conf.num_client:0;
+
+ //g_conf.num_mon = intabs(g_conf.num_mon);
+ g_conf.num_mds = intabs(g_conf.num_mds);
+ g_conf.num_client = intabs(g_conf.num_client);
+ g_conf.num_osd = intabs(g_conf.num_osd);
+
+
if (g_conf.kill_after)
g_timer.add_event_after(g_conf.kill_after, new C_Die);
if (g_conf.debug_after)
int need = 0;
if (g_conf.ms_skip_rank0) need++;
- need += NUMMDS;
+ need += start_mds;
if (g_conf.ms_stripe_osds)
need++;
else
- need += NUMOSD;
- if (NUMCLIENT) {
+ need += start_osd;
+ if (start_client) {
if (!g_conf.ms_overlay_clients)
need += 1;
}
assert(need <= world);
if (myrank == 0)
- cerr << "nummds " << NUMMDS << " numosd " << NUMOSD << " numclient " << NUMCLIENT << " .. need " << need << ", have " << world << endl;
+ cerr << "nummds " << start_mds << " numosd " << start_osd << " numclient " << start_client << " .. need " << need << ", have " << world << endl;
char hostname[100];
// create mds
map<int,MDS*> mds;
map<int,OSD*> mdsosd;
- for (int i=0; i<NUMMDS; i++) {
+ for (int i=0; i<start_mds; i++) {
if (myrank != g_conf.ms_skip_rank0+i) continue;
Messenger *m = rank.register_entity(MSG_ADDR_MDS(i));
cerr << "mds" << i << " at " << rank.my_addr << " " << hostname << "." << pid << endl;
// create osd
map<int,OSD*> osd;
- int max_osd_nodes = world - NUMMDS - g_conf.ms_skip_rank0; // assumes 0 clients, if we stripe.
- int osds_per_node = (NUMOSD-1)/max_osd_nodes + 1;
- for (int i=0; i<NUMOSD; i++) {
+ int max_osd_nodes = world - start_mds - g_conf.ms_skip_rank0; // assumes 0 clients, if we stripe.
+ int osds_per_node = (start_osd-1)/max_osd_nodes + 1;
+ for (int i=0; i<start_osd; i++) {
if (g_conf.ms_stripe_osds) {
- if (myrank != g_conf.ms_skip_rank0+NUMMDS + i / osds_per_node) continue;
+ if (myrank != g_conf.ms_skip_rank0+start_mds + i / osds_per_node) continue;
} else {
- if (myrank != g_conf.ms_skip_rank0+NUMMDS + i) continue;
+ if (myrank != g_conf.ms_skip_rank0+start_mds + i) continue;
}
if (kill_osd_after.count(i))
if (g_conf.ms_overlay_clients) sleep(5);
// create client
- int skip_osd = NUMOSD;
+ int skip_osd = start_osd;
if (g_conf.ms_overlay_clients)
skip_osd = 0; // put clients with osds too!
- int client_nodes = world - NUMMDS - skip_osd - g_conf.ms_skip_rank0;
+ int client_nodes = world - start_mds - skip_osd - g_conf.ms_skip_rank0;
int clients_per_node = 1;
- if (NUMCLIENT && client_nodes > 0) clients_per_node = (NUMCLIENT-1) / client_nodes + 1;
+ if (start_client && client_nodes > 0) clients_per_node = (start_client-1) / client_nodes + 1;
set<int> clientlist;
- map<int,Client *> client;//[NUMCLIENT];
- map<int,SyntheticClient *> syn;//[NUMCLIENT];
+ map<int,Client *> client;//[start_client];
+ map<int,SyntheticClient *> syn;//[start_client];
int nclients = 0;
- for (int i=0; i<NUMCLIENT; i++) {
- //if (myrank != NUMMDS + NUMOSD + i % client_nodes) continue;
- if (myrank != g_conf.ms_skip_rank0+NUMMDS + skip_osd + i / clients_per_node) continue;
+ for (int i=0; i<start_client; i++) {
+ //if (myrank != start_mds + start_osd + i % client_nodes) continue;
+ if (myrank != g_conf.ms_skip_rank0+start_mds + skip_osd + i / clients_per_node) continue;
clientlist.insert(i);
client[i] = new Client(rank.register_entity(MSG_ADDR_CLIENT_NEW), monmap);
delete i->second;
*/
/*
- for (int i=0; i<NUMMDS; i++) {
+ for (int i=0; i<start_mds; i++) {
if (myrank != MPI_DEST_TO_RANK(MSG_ADDR_MDS(i),world)) continue;
delete mds[i];
}
- for (int i=0; i<NUMOSD; i++) {
+ for (int i=0; i<start_osd; i++) {
if (myrank != MPI_DEST_TO_RANK(MSG_ADDR_OSD(i),world)) continue;
delete osd[i];
}
- for (int i=0; i<NUMCLIENT; i++) {
+ for (int i=0; i<start_client; i++) {
if (myrank != MPI_DEST_TO_RANK(MSG_ADDR_CLIENT(i),world)) continue;
delete client[i];
}
int FakeStore::write(object_t oid,
off_t offset, size_t len,
- bufferlist& bl,
+ const bufferlist& bl,
Context *onsafe)
{
dout(20) << "write " << oid << " len " << len << " off " << offset << endl;
int remove(object_t oid, Context *onsafe);
int truncate(object_t oid, off_t size, Context *onsafe);
int read(object_t oid, off_t offset, size_t len, bufferlist& bl);
- int write(object_t oid, off_t offset, size_t len, bufferlist& bl, Context *onsafe);
+ int write(object_t oid, off_t offset, size_t len, const bufferlist& bl, Context *onsafe);
void sync();
void sync(Context *onsafe);
dout(10) << "handle_osd_map decoding inc map epoch " << cur+1 << dendl;
bufferlist bl;
- if (m->incremental_maps.count(cur+1))
+ if (m->incremental_maps.count(cur+1)) {
+ dout(10) << " using provided inc map" << endl;
bl = m->incremental_maps[cur+1];
- else
+ } else {
+ dout(10) << " using my locally stored inc map" << endl;
get_inc_map_bl(cur+1, bl);
+ }
OSDMap::Incremental inc;
int off = 0;
list<int> old_overload; // no longer overload
void encode(bufferlist& bl) {
- bl.append((char*)&epoch, sizeof(epoch));
- bl.append((char*)&mon_epoch, sizeof(mon_epoch));
- bl.append((char*)&ctime, sizeof(ctime));
+ ::_encode(epoch, bl);
+ ::_encode(mon_epoch, bl);
+ ::_encode(ctime, bl);
::_encode(new_up, bl);
::_encode(new_down, bl);
::_encode(new_in, bl);
::_encode(fullmap, bl);
}
void decode(bufferlist& bl, int& off) {
- bl.copy(off, sizeof(epoch), (char*)&epoch);
- off += sizeof(epoch);
- bl.copy(off, sizeof(mon_epoch), (char*)&mon_epoch);
- off += sizeof(mon_epoch);
- bl.copy(off, sizeof(ctime), (char*)&ctime);
- off += sizeof(ctime);
+ ::_decode(epoch, bl, off);
+ ::_decode(mon_epoch, bl, off);
+ ::_decode(ctime, bl, off);
::_decode(new_up, bl, off);
::_decode(new_down, bl, off);
::_decode(new_in, bl, off);
pattrsets.push_back(&aset);
}
- void write(object_t oid, off_t off, size_t len, bufferlist& bl) {
+ void write(object_t oid, off_t off, size_t len, const bufferlist& bl) {
int op = OP_WRITE;
ops.push_back(op);
oids.push_back(oid);
virtual int read(object_t oid,
off_t offset, size_t len,
bufferlist& bl) = 0;
-
- /*virtual int write(object_t oid,
- off_t offset, size_t len,
- bufferlist& bl,
- bool fsync=true) = 0;
- */
virtual int write(object_t oid,
off_t offset, size_t len,
- bufferlist& bl,
+ const bufferlist& bl,
Context *onsafe) = 0;//{ return -1; }
virtual void trim_from_cache(object_t oid,
off_t offset, size_t len) { }
void Journaler::flush(Context *onsync)
{
- if (write_pos == flush_pos) {
+ // all flushed and acked?
+ if (write_pos == ack_pos) {
assert(write_buf.length() == 0);
dout(10) << "flush nothing to flush, write pointers at " << write_pos << "/" << flush_pos << "/" << ack_pos << endl;
-
if (onsync) {
onsync->finish(0);
delete onsync;
return;
}
- unsigned len = write_pos - flush_pos;
- assert(len == write_buf.length());
- dout(10) << "flush flushing " << flush_pos << "~" << len << endl;
-
- // submit write for anything pending
- // flush _start_ pos to _finish_flush
- filer.write(inode, flush_pos, len, write_buf, 0,
- g_conf.journaler_safe ? 0:new C_Flush(this, flush_pos), // on ACK
- g_conf.journaler_safe ? new C_Flush(this, flush_pos):0); // on COMMIT
- pending_flush[flush_pos] = g_clock.now();
-
- // adjust pointers
- flush_pos = write_pos;
- write_buf.clear();
-
- dout(10) << "flush write pointers now at " << write_pos << "/" << flush_pos << "/" << ack_pos << endl;
+ if (write_pos == flush_pos) {
+ assert(write_buf.length() == 0);
+ dout(10) << "flush nothing to flush, write pointers at " << write_pos << "/" << flush_pos << "/" << ack_pos << endl;
+ } else {
+ // flush
+ unsigned len = write_pos - flush_pos;
+ assert(len == write_buf.length());
+ dout(10) << "flush flushing " << flush_pos << "~" << len << endl;
+
+ // submit write for anything pending
+ // flush _start_ pos to _finish_flush
+ filer.write(inode, flush_pos, len, write_buf, 0,
+ g_conf.journaler_safe ? 0:new C_Flush(this, flush_pos), // on ACK
+ g_conf.journaler_safe ? new C_Flush(this, flush_pos):0); // on COMMIT
+ pending_flush[flush_pos] = g_clock.now();
+
+ // adjust pointers
+ flush_pos = write_pos;
+ write_buf.clear();
+
+ dout(10) << "flush write pointers now at " << write_pos << "/" << flush_pos << "/" << ack_pos << endl;
+ }
// queue waiter (at _new_ write_pos; will go when reached by ack_pos)
if (onsync)
waitfor_flush[write_pos].push_back(onsync);
// write head?
- if (last_wrote_head.sec() + 30 < g_clock.now().sec()) {
+ if (last_wrote_head.sec() + g_conf.journaler_write_head_interval < g_clock.now().sec()) {
write_head();
}
}
#cdir:adjust_nested_auth_pins on [dir 163 /foo/ rep@13 | child] count now 0 + 1
if (/adjust_nested_auth_pins/) {
- my ($what) = /\[(\w+ \d+) /;
+ my ($what) = / (\w+)\]/;
+ $what =~ s/ 0x/ /;
$hist{$what} .= "$l: $_"
if defined $pin{$what};
}
# cinode:auth_pin on inode [1000000002625 /gnu/blah_client_created. 0x89b7700] count now 1 + 0
- if (/auth_pin /) {
- my ($what) = /\[(\w+ \d+) /;
-# print "add_waiter $c $what\n";
+ elsif (/auth_pin / && !/waiting/) {
+ #my ($what) = /\[(\w+ \w+) /;
+ my ($what) = / (\w+)\]/;
+ $what =~ s/ 0x/ /;
+ #print "$_ add_waiter $c $what\n";
$pin{$what}++;
$hist{$what} .= "$l: $_";
push( @pins, $what ) unless grep {$_ eq $what} @pins;
# cinode:auth_unpin on inode [1000000002625 (dangling) 0x89b7700] count now 0 + 0
- if (/auth_unpin/) {
- my ($what) = /\[(\w+ \d+) /;# / on (.*\])/;
+ elsif (/auth_unpin/) {
+ #my ($what) = /\[(\w+ \w+) /;# / on (.*\])/;
+ my ($what) = / (\w+)\]/;
+ $what =~ s/ 0x/ /;
$pin{$what}--;
$hist{$what} .= "$l: $_";
unless ($pin{$what}) {