summaryrefslogtreecommitdiff
path: root/s
diff options
context:
space:
mode:
Diffstat (limited to 's')
-rw-r--r--s/balance.cpp197
-rw-r--r--s/balance.h69
-rw-r--r--s/balancer_policy.cpp227
-rw-r--r--s/balancer_policy.h28
-rw-r--r--s/chunk.cpp1097
-rw-r--r--s/chunk.h279
-rw-r--r--s/client.cpp292
-rw-r--r--s/client.h120
-rw-r--r--s/commands_admin.cpp568
-rw-r--r--s/commands_public.cpp541
-rw-r--r--s/config.cpp409
-rw-r--r--s/config.h120
-rw-r--r--s/config_migrate.cpp76
-rw-r--r--s/cursors.cpp153
-rw-r--r--s/cursors.h37
-rw-r--r--s/d_chunk_manager.cpp328
-rw-r--r--s/d_chunk_manager.h150
-rw-r--r--s/d_logic.cpp37
-rw-r--r--s/d_logic.h213
-rw-r--r--s/d_migrate.cpp1197
-rw-r--r--s/d_split.cpp722
-rw-r--r--s/d_state.cpp694
-rw-r--r--s/d_writeback.cpp97
-rw-r--r--s/d_writeback.h75
-rw-r--r--s/dbgrid.vcproj1048
-rw-r--r--s/dbgrid.vcxproj18
-rwxr-xr-xs/dbgrid.vcxproj.filters36
-rw-r--r--s/grid.cpp257
-rw-r--r--s/grid.h33
-rw-r--r--s/request.cpp168
-rw-r--r--s/request.h63
-rw-r--r--s/s_only.cpp63
-rw-r--r--s/server.cpp193
-rw-r--r--s/server.h4
-rw-r--r--s/shard.cpp271
-rw-r--r--s/shard.h97
-rw-r--r--s/shard_version.cpp151
-rw-r--r--s/shard_version.h (renamed from s/d_util.cpp)32
-rw-r--r--s/shardconnection.cpp173
-rw-r--r--s/shardkey.cpp152
-rw-r--r--s/shardkey.h40
-rw-r--r--s/stats.cpp2
-rw-r--r--s/stats.h2
-rw-r--r--s/strategy.cpp308
-rw-r--r--s/strategy.h13
-rw-r--r--s/strategy_shard.cpp196
-rw-r--r--s/strategy_single.cpp184
-rw-r--r--s/util.h68
-rw-r--r--s/writeback_listener.cpp254
-rw-r--r--s/writeback_listener.h67
50 files changed, 6763 insertions, 4856 deletions
diff --git a/s/balance.cpp b/s/balance.cpp
index 33cafdf..ee0c992 100644
--- a/s/balance.cpp
+++ b/s/balance.cpp
@@ -1,4 +1,4 @@
-// balance.cpp
+//@file balance.cpp
/**
* Copyright (C) 2008 10gen Inc.
@@ -31,10 +31,10 @@
#include "grid.h"
namespace mongo {
-
+
Balancer balancer;
- Balancer::Balancer() : _balancedLastTime(0), _policy( new BalancerPolicy ){}
+ Balancer::Balancer() : _balancedLastTime(0), _policy( new BalancerPolicy ) {}
Balancer::~Balancer() {
delete _policy;
@@ -43,15 +43,15 @@ namespace mongo {
int Balancer::_moveChunks( const vector<CandidateChunkPtr>* candidateChunks ) {
int movedCount = 0;
- for ( vector<CandidateChunkPtr>::const_iterator it = candidateChunks->begin(); it != candidateChunks->end(); ++it ){
+ for ( vector<CandidateChunkPtr>::const_iterator it = candidateChunks->begin(); it != candidateChunks->end(); ++it ) {
const CandidateChunk& chunkInfo = *it->get();
DBConfigPtr cfg = grid.getDBConfig( chunkInfo.ns );
assert( cfg );
-
+
ChunkManagerPtr cm = cfg->getChunkManager( chunkInfo.ns );
assert( cm );
-
+
const BSONObj& chunkToMove = chunkInfo.chunk;
ChunkPtr c = cm->findChunk( chunkToMove["min"].Obj() );
if ( c->getMin().woCompare( chunkToMove["min"].Obj() ) || c->getMax().woCompare( chunkToMove["max"].Obj() ) ) {
@@ -61,62 +61,65 @@ namespace mongo {
c = cm->findChunk( chunkToMove["min"].Obj() );
if ( c->getMin().woCompare( chunkToMove["min"].Obj() ) || c->getMax().woCompare( chunkToMove["max"].Obj() ) ) {
- log() << "chunk mismatch after reload, ignoring will retry issue cm: "
+ log() << "chunk mismatch after reload, ignoring will retry issue cm: "
<< c->getMin() << " min: " << chunkToMove["min"].Obj() << endl;
continue;
}
}
-
- string errmsg;
- if ( c->moveAndCommit( Shard::make( chunkInfo.to ) , errmsg ) ){
+
+ BSONObj res;
+ if ( c->moveAndCommit( Shard::make( chunkInfo.to ) , Chunk::MaxChunkSize , res ) ) {
movedCount++;
continue;
}
- log() << "MOVE FAILED **** " << errmsg << "\n"
- << " from: " << chunkInfo.from << " to: " << chunkInfo.to << " chunk: " << chunkToMove << endl;
+ // the move requires acquiring the collection metadata's lock, which can fail
+ log() << "balacer move failed: " << res << " from: " << chunkInfo.from << " to: " << chunkInfo.to
+ << " chunk: " << chunkToMove << endl;
+
+ if ( res["chunkTooBig"].trueValue() ) {
+ // reload just to be safe
+ cm = cfg->getChunkManager( chunkInfo.ns );
+ assert( cm );
+ c = cm->findChunk( chunkToMove["min"].Obj() );
+
+ log() << "forcing a split because migrate failed for size reasons" << endl;
+
+ res = BSONObj();
+ c->singleSplit( true , res );
+ log() << "forced split results: " << res << endl;
+
+ // TODO: if the split fails, mark as jumbo SERVER-2571
+ }
}
return movedCount;
}
-
- void Balancer::_ping(){
- assert( _myid.size() && _started );
- try {
- ScopedDbConnection conn( configServer.getPrimary() );
- _ping( conn.conn() );
- conn.done();
- }
- catch ( std::exception& e ){
- log() << "bare ping failed: " << e.what() << endl;
- }
-
- }
- void Balancer::_ping( DBClientBase& conn ){
+ void Balancer::_ping( DBClientBase& conn ) {
WriteConcern w = conn.getWriteConcern();
conn.setWriteConcern( W_NONE );
- conn.update( ShardNS::mongos ,
- BSON( "_id" << _myid ) ,
- BSON( "$set" << BSON( "ping" << DATENOW << "up" << (int)(time(0)-_started) ) ) ,
- true );
+ conn.update( ShardNS::mongos ,
+ BSON( "_id" << _myid ) ,
+ BSON( "$set" << BSON( "ping" << DATENOW << "up" << (int)(time(0)-_started) ) ) ,
+ true );
conn.setWriteConcern( w);
}
-
- bool Balancer::_checkOIDs(){
+
+ bool Balancer::_checkOIDs() {
vector<Shard> all;
Shard::getAllShards( all );
-
+
map<int,Shard> oids;
-
- for ( vector<Shard>::iterator i=all.begin(); i!=all.end(); ++i ){
+
+ for ( vector<Shard>::iterator i=all.begin(); i!=all.end(); ++i ) {
Shard s = *i;
BSONObj f = s.runCommand( "admin" , "features" );
- if ( f["oidMachine"].isNumber() ){
+ if ( f["oidMachine"].isNumber() ) {
int x = f["oidMachine"].numberInt();
- if ( oids.count(x) == 0 ){
+ if ( oids.count(x) == 0 ) {
oids[x] = s;
}
else {
@@ -133,7 +136,7 @@ namespace mongo {
return true;
}
- void Balancer::_doBalanceRound( DBClientBase& conn, vector<CandidateChunkPtr>* candidateChunks ){
+ void Balancer::_doBalanceRound( DBClientBase& conn, vector<CandidateChunkPtr>* candidateChunks ) {
assert( candidateChunks );
//
@@ -143,8 +146,8 @@ namespace mongo {
auto_ptr<DBClientCursor> cursor = conn.query( ShardNS::collection , BSONObj() );
vector< string > collections;
- while ( cursor->more() ){
- BSONObj col = cursor->next();
+ while ( cursor->more() ) {
+ BSONObj col = cursor->nextSafe();
// sharded collections will have a shard "key".
if ( ! col["key"].eoo() )
@@ -164,7 +167,7 @@ namespace mongo {
//
// TODO: skip unresponsive shards and mark information as stale.
//
-
+
vector<Shard> allShards;
Shard::getAllShards( allShards );
if ( allShards.size() < 2) {
@@ -172,14 +175,16 @@ namespace mongo {
return;
}
- map< string, BSONObj > shardLimitsMap;
- for ( vector<Shard>::const_iterator it = allShards.begin(); it != allShards.end(); ++it ){
+ map< string, BSONObj > shardLimitsMap;
+ for ( vector<Shard>::const_iterator it = allShards.begin(); it != allShards.end(); ++it ) {
const Shard& s = *it;
ShardStatus status = s.getStatus();
- BSONObj limitsObj = BSON( ShardFields::maxSize( s.getMaxSize() ) <<
- ShardFields::currSize( status.mapped() ) <<
- ShardFields::draining( s.isDraining()) );
+ BSONObj limitsObj = BSON( ShardFields::maxSize( s.getMaxSize() ) <<
+ LimitsFields::currSize( status.mapped() ) <<
+ ShardFields::draining( s.isDraining() ) <<
+ LimitsFields::hasOpsQueued( status.hasOpsQueued() )
+ );
shardLimitsMap[ s.getName() ] = limitsObj;
}
@@ -193,8 +198,8 @@ namespace mongo {
map< string,vector<BSONObj> > shardToChunksMap;
cursor = conn.query( ShardNS::chunk , QUERY( "ns" << ns ).sort( "min" ) );
- while ( cursor->more() ){
- BSONObj chunk = cursor->next();
+ while ( cursor->more() ) {
+ BSONObj chunk = cursor->nextSafe();
vector<BSONObj>& chunks = shardToChunksMap[chunk["shard"].String()];
chunks.push_back( chunk.getOwned() );
}
@@ -204,8 +209,8 @@ namespace mongo {
log(1) << "skipping empty collection (" << ns << ")";
continue;
}
-
- for ( vector<Shard>::iterator i=allShards.begin(); i!=allShards.end(); ++i ){
+
+ for ( vector<Shard>::iterator i=allShards.begin(); i!=allShards.end(); ++i ) {
// this just makes sure there is an entry in shardToChunksMap for every shard
Shard s = *i;
shardToChunksMap[s.getName()].size();
@@ -216,75 +221,109 @@ namespace mongo {
}
}
- void Balancer::run(){
+ bool Balancer::_init() {
+ try {
+
+ log() << "about to contact config servers and shards" << endl;
+
+ // contact the config server and refresh shard information
+ // checks that each shard is indeed a different process (no hostname mixup)
+ // these checks are redundant in that they're redone at every new round but we want to do them initially here
+ // so to catch any problem soon
+ Shard::reloadShardInfo();
+ _checkOIDs();
+
+ log() << "config servers and shards contacted successfully" << endl;
- { // init stuff, don't want to do at static init
StringBuilder buf;
buf << getHostNameCached() << ":" << cmdLine.port;
_myid = buf.str();
- log(1) << "balancer myid: " << _myid << endl;
-
_started = time(0);
- Shard::reloadShardInfo();
+ log() << "balancer id: " << _myid << " started at " << time_t_to_String_short(_started) << endl;
+
+ return true;
+
}
-
- _ping();
- _checkOIDs();
+ catch ( std::exception& ) {
+ log( LL_WARNING ) << "could not initialize balancer, please check that all shards and config servers are up" << endl;
+ return false;
+
+ }
+ }
+
+ void Balancer::run() {
+
+ // this is the body of a BackgroundJob so if we throw here we're basically ending the balancer thread prematurely
+ while ( ! inShutdown() ) {
+
+ if ( ! _init() ) {
+ log() << "will retry to initialize balancer in one minute" << endl;
+ sleepsecs( 60 );
+ continue;
+ }
+
+ break;
+ }
+
+ // getConnectioString and the constructor of a DistributedLock do not throw, which is what we expect on while
+ // on the balancer thread
ConnectionString config = configServer.getConnectionString();
DistributedLock balanceLock( config , "balancer" );
- while ( ! inShutdown() ){
-
+ while ( ! inShutdown() ) {
+
try {
+
+ // first make sure we should even be running
+ if ( ! grid.shouldBalance() ) {
+ log(1) << "skipping balancing round because balancing is disabled" << endl;
+ sleepsecs( 30 );
+ continue;
+ }
+
+
ScopedDbConnection conn( config );
- _ping( conn.conn() );
- if ( ! _checkOIDs() ){
+ _ping( conn.conn() );
+ if ( ! _checkOIDs() ) {
uassert( 13258 , "oids broken after resetting!" , _checkOIDs() );
}
-
+
// use fresh shard state
- Shard::reloadShardInfo();
+ Shard::reloadShardInfo();
dist_lock_try lk( &balanceLock , "doing balance round" );
- if ( ! lk.got() ){
- log(1) << "skipping balancing round during ongoing split or move activity." << endl;
+ if ( ! lk.got() ) {
+ log(1) << "skipping balancing round because another balancer is active" << endl;
conn.done();
sleepsecs( 30 ); // no need to wake up soon
continue;
}
-
- if ( ! grid.shouldBalance() ) {
- log(1) << "skipping balancing round because balancing is disabled" << endl;;
- conn.done();
-
- sleepsecs( 30 );
- continue;
- }
- log(1) << "*** start balancing round" << endl;
+ log(1) << "*** start balancing round" << endl;
vector<CandidateChunkPtr> candidateChunks;
_doBalanceRound( conn.conn() , &candidateChunks );
if ( candidateChunks.size() == 0 ) {
log(1) << "no need to move any chunk" << endl;
- } else {
+ }
+ else {
_balancedLastTime = _moveChunks( &candidateChunks );
}
- log(1) << "*** end of balancing round" << endl;
+ log(1) << "*** end of balancing round" << endl;
conn.done();
sleepsecs( _balancedLastTime ? 5 : 10 );
}
- catch ( std::exception& e ){
+ catch ( std::exception& e ) {
log() << "caught exception while doing balance: " << e.what() << endl;
// Just to match the opening statement if in log level 1
- log(1) << "*** End of balancing round" << endl;
+ log(1) << "*** End of balancing round" << endl;
sleepsecs( 30 ); // sleep a fair amount b/c of error
continue;
diff --git a/s/balance.h b/s/balance.h
index cafae11..0ad2647 100644
--- a/s/balance.h
+++ b/s/balance.h
@@ -1,4 +1,4 @@
-// balance.h
+//@file balance.h
/**
* Copyright (C) 2008 10gen Inc.
@@ -24,7 +24,16 @@
#include "balancer_policy.h"
namespace mongo {
-
+
+ /**
+ * The balancer is a background task that tries to keep the number of chunks across all servers of the cluster even. Although
+ * every mongos will have one balancer running, only one of them will be active at the any given point in time. The balancer
+ * uses a 'DistributedLock' for that coordination.
+ *
+ * The balancer does act continuously but in "rounds". At a given round, it would decide if there is an imbalance by
+ * checking the difference in chunks between the most and least loaded shards. It would issue a request for a chunk
+ * migration per round, if it found so.
+ */
class Balancer : public BackgroundJob {
public:
Balancer();
@@ -34,47 +43,63 @@ namespace mongo {
virtual void run();
- virtual string name() { return "Balancer"; }
+ virtual string name() const { return "Balancer"; }
private:
typedef BalancerPolicy::ChunkInfo CandidateChunk;
typedef shared_ptr<CandidateChunk> CandidateChunkPtr;
+ // hostname:port of my mongos
+ string _myid;
+
+ // time the Balancer started running
+ time_t _started;
+
+ // number of moved chunks in last round
+ int _balancedLastTime;
+
+ // decide which chunks to move; owned here.
+ BalancerPolicy* _policy;
+
+ /**
+ * Checks that the balancer can connect to all servers it needs to do its job.
+ *
+ * @return true if balancing can be started
+ *
+ * This method throws on a network exception
+ */
+ bool _init();
+
/**
- * Gathers all the necessary information about shards and chunks, and
- * decides whether there are candidate chunks to be moved.
+ * Gathers all the necessary information about shards and chunks, and decides whether there are candidate chunks to
+ * be moved.
+ *
+ * @param conn is the connection with the config server(s)
+ * @param candidateChunks (IN/OUT) filled with candidate chunks, one per collection, that could possibly be moved
*/
void _doBalanceRound( DBClientBase& conn, vector<CandidateChunkPtr>* candidateChunks );
/**
- * Execute the chunk migrations described in 'candidateChunks' and
- * returns the number of chunks effectively moved.
+ * Issues chunk migration request, one at a time.
+ *
+ * @param candidateChunks possible chunks to move
+ * @return number of chunks effectively moved
*/
int _moveChunks( const vector<CandidateChunkPtr>* candidateChunks );
/**
- * Check the health of the master configuration server
+ * Marks this balancer as being live on the config server(s).
+ *
+ * @param conn is the connection with the config server(s)
*/
- void _ping();
void _ping( DBClientBase& conn );
/**
- * @return true if everything is ok
+ * @return true if all the servers listed in configdb as being shards are reachable and are distinct processes
*/
bool _checkOIDs();
- // internal state
-
- string _myid; // hostname:port of my mongos
- time_t _started; // time Balancer starte running
- int _balancedLastTime; // number of moved chunks in last round
- BalancerPolicy* _policy; // decide which chunks to move; owned here.
-
- // non-copyable, non-assignable
-
- Balancer(const Balancer&);
- Balancer operator=(const Balancer&);
};
-
+
extern Balancer balancer;
}
diff --git a/s/balancer_policy.cpp b/s/balancer_policy.cpp
index 98619c0..2098a1f 100644
--- a/s/balancer_policy.cpp
+++ b/s/balancer_policy.cpp
@@ -28,54 +28,62 @@
namespace mongo {
- BalancerPolicy::ChunkInfo* BalancerPolicy::balance( const string& ns,
- const ShardToLimitsMap& shardToLimitsMap,
- const ShardToChunksMap& shardToChunksMap,
- int balancedLastTime ){
+ // limits map fields
+ BSONField<long long> LimitsFields::currSize( "currSize" );
+ BSONField<bool> LimitsFields::hasOpsQueued( "hasOpsQueued" );
+
+ BalancerPolicy::ChunkInfo* BalancerPolicy::balance( const string& ns,
+ const ShardToLimitsMap& shardToLimitsMap,
+ const ShardToChunksMap& shardToChunksMap,
+ int balancedLastTime ) {
pair<string,unsigned> min("",numeric_limits<unsigned>::max());
pair<string,unsigned> max("",0);
vector<string> drainingShards;
-
- for (ShardToChunksIter i = shardToChunksMap.begin(); i!=shardToChunksMap.end(); ++i ){
- // Find whether this shard has reached its size cap or whether it is being removed.
+ for (ShardToChunksIter i = shardToChunksMap.begin(); i!=shardToChunksMap.end(); ++i ) {
+
+ // Find whether this shard's capacity or availability are exhausted
const string& shard = i->first;
BSONObj shardLimits;
ShardToLimitsIter it = shardToLimitsMap.find( shard );
if ( it != shardToLimitsMap.end() ) shardLimits = it->second;
const bool maxedOut = isSizeMaxed( shardLimits );
const bool draining = isDraining( shardLimits );
+ const bool opsQueued = hasOpsQueued( shardLimits );
- // Check whether this shard is a better chunk receiver then the current one.
- // Maxed out shards or draining shards cannot be considered receivers.
+ // Is this shard a better chunk receiver then the current one?
+ // Shards that would be bad receiver candidates:
+ // + maxed out shards
+ // + draining shards
+ // + shards with operations queued for writeback
const unsigned size = i->second.size();
- if ( ! maxedOut && ! draining ){
- if ( size < min.second ){
+ if ( ! maxedOut && ! draining && ! opsQueued ) {
+ if ( size < min.second ) {
min = make_pair( shard , size );
}
}
// Check whether this shard is a better chunk donor then the current one.
// Draining shards take a lower priority than overloaded shards.
- if ( size > max.second ){
- max = make_pair( shard , size );
+ if ( size > max.second ) {
+ max = make_pair( shard , size );
}
- if ( draining && (size > 0)){
+ if ( draining && (size > 0)) {
drainingShards.push_back( shard );
}
}
- // If there is no candidate chunk receiver -- they may have all been maxed out,
- // draining, ... -- there's not much that the policy can do.
- if ( min.second == numeric_limits<unsigned>::max() ){
+ // If there is no candidate chunk receiver -- they may have all been maxed out,
+ // draining, ... -- there's not much that the policy can do.
+ if ( min.second == numeric_limits<unsigned>::max() ) {
log() << "no availalable shards to take chunks" << endl;
return NULL;
}
-
+
log(1) << "collection : " << ns << endl;
log(1) << "donor : " << max.second << " chunks on " << max.first << endl;
log(1) << "receiver : " << min.second << " chunks on " << min.first << endl;
- if ( ! drainingShards.empty() ){
+ if ( ! drainingShards.empty() ) {
string drainingStr;
joinStringDelim( drainingShards, &drainingStr, ',' );
log(1) << "draining : " << ! drainingShards.empty() << "(" << drainingShards.size() << ")" << endl;
@@ -86,34 +94,36 @@ namespace mongo {
const int imbalance = max.second - min.second;
const int threshold = balancedLastTime ? 2 : 8;
string from, to;
- if ( imbalance >= threshold ){
+ if ( imbalance >= threshold ) {
from = max.first;
to = min.first;
- } else if ( ! drainingShards.empty() ){
+ }
+ else if ( ! drainingShards.empty() ) {
from = drainingShards[ rand() % drainingShards.size() ];
to = min.first;
- } else {
- // Everything is balanced here!
+ }
+ else {
+ // Everything is balanced here!
return NULL;
}
const vector<BSONObj>& chunksFrom = shardToChunksMap.find( from )->second;
const vector<BSONObj>& chunksTo = shardToChunksMap.find( to )->second;
BSONObj chunkToMove = pickChunk( chunksFrom , chunksTo );
- log() << "chose [" << from << "] to [" << to << "] " << chunkToMove << endl;
+ log() << "chose [" << from << "] to [" << to << "] " << chunkToMove << endl;
return new ChunkInfo( ns, to, from, chunkToMove );
}
- BSONObj BalancerPolicy::pickChunk( const vector<BSONObj>& from, const vector<BSONObj>& to ){
+ BSONObj BalancerPolicy::pickChunk( const vector<BSONObj>& from, const vector<BSONObj>& to ) {
// It is possible for a donor ('from') shard to have less chunks than a recevier one ('to')
- // if the donor is in draining mode.
-
+ // if the donor is in draining mode.
+
if ( to.size() == 0 )
return from[0];
-
+
if ( from[0]["min"].Obj().woCompare( to[to.size()-1]["max"].Obj() , BSONObj() , false ) == 0 )
return from[0];
@@ -123,174 +133,41 @@ namespace mongo {
return from[0];
}
- bool BalancerPolicy::isSizeMaxed( BSONObj limits ){
- // If there's no limit information for the shard, assume it can be a chunk receiver
+ bool BalancerPolicy::isSizeMaxed( BSONObj limits ) {
+ // If there's no limit information for the shard, assume it can be a chunk receiver
// (i.e., there's not bound on space utilization)
- if ( limits.isEmpty() ){
+ if ( limits.isEmpty() ) {
return false;
}
long long maxUsage = limits[ ShardFields::maxSize.name() ].Long();
- if ( maxUsage == 0 ){
+ if ( maxUsage == 0 ) {
return false;
}
- long long currUsage = limits[ ShardFields::currSize.name() ].Long();
- if ( currUsage < maxUsage ){
+ long long currUsage = limits[ LimitsFields::currSize.name() ].Long();
+ if ( currUsage < maxUsage ) {
return false;
}
return true;
}
- bool BalancerPolicy::isDraining( BSONObj limits ){
+ bool BalancerPolicy::isDraining( BSONObj limits ) {
BSONElement draining = limits[ ShardFields::draining.name() ];
- if ( draining.eoo() || ! draining.Bool() ){
+ if ( draining.eoo() || ! draining.trueValue() ) {
return false;
}
return true;
}
- class PolicyObjUnitTest : public UnitTest {
- public:
-
- typedef ShardFields sf; // convenience alias
-
- void caseSizeMaxedShard(){
- BSONObj shard0 = BSON( sf::maxSize(0LL) << sf::currSize(0LL) );
- assert( ! BalancerPolicy::isSizeMaxed( shard0 ) );
-
- BSONObj shard1 = BSON( sf::maxSize(100LL) << sf::currSize(80LL) );
- assert( ! BalancerPolicy::isSizeMaxed( shard1 ) );
-
- BSONObj shard2 = BSON( sf::maxSize(100LL) << sf::currSize(110LL) );
- assert( BalancerPolicy::isSizeMaxed( shard2 ) );
-
- BSONObj empty;
- assert( ! BalancerPolicy::isSizeMaxed( empty ) );
- }
-
- void caseDrainingShard(){
- BSONObj shard0 = BSON( sf::draining(true) );
- assert( BalancerPolicy::isDraining( shard0 ) );
-
- BSONObj shard1 = BSON( sf::draining(false) );
- assert( ! BalancerPolicy::isDraining( shard1 ) );
-
- BSONObj empty;
- assert( ! BalancerPolicy::isDraining( empty ) );
- }
-
- void caseBalanceNormal(){
- // 2 chunks and 0 chunk shards
- BalancerPolicy::ShardToChunksMap chunkMap;
- vector<BSONObj> chunks;
- chunks.push_back(BSON( "min" << BSON( "x" << BSON( "$minKey"<<1) ) <<
- "max" << BSON( "x" << 49 )));
- chunks.push_back(BSON( "min" << BSON( "x" << 49 ) <<
- "max" << BSON( "x" << BSON( "$maxkey"<<1 ))));
- chunkMap["shard0"] = chunks;
- chunks.clear();
- chunkMap["shard1"] = chunks;
-
- // no limits
- BalancerPolicy::ShardToLimitsMap limitsMap;
- BSONObj limits0 = BSON( sf::maxSize(0LL) << sf::currSize(2LL) << sf::draining(false) );
- BSONObj limits1 = BSON( sf::maxSize(0LL) << sf::currSize(0LL) << sf::draining(false) );
- limitsMap["shard0"] = limits0;
- limitsMap["shard1"] = limits1;
-
- BalancerPolicy::ChunkInfo* c = NULL;
- c = BalancerPolicy::balance( "ns", limitsMap, chunkMap, 1 );
- assert( c != NULL );
- }
-
- void caseBalanceDraining(){
- // one normal, one draining
- // 2 chunks and 0 chunk shards
- BalancerPolicy::ShardToChunksMap chunkMap;
- vector<BSONObj> chunks;
- chunks.push_back(BSON( "min" << BSON( "x" << BSON( "$minKey"<<1) ) <<
- "max" << BSON( "x" << 49 )));
- chunkMap["shard0"] = chunks;
- chunks.clear();
- chunks.push_back(BSON( "min" << BSON( "x" << 49 ) <<
- "max" << BSON( "x" << BSON( "$maxkey"<<1 ))));
- chunkMap["shard1"] = chunks;
-
- // shard0 is draining
- BalancerPolicy::ShardToLimitsMap limitsMap;
- BSONObj limits0 = BSON( sf::maxSize(0LL) << sf::currSize(2LL) << sf::draining(true) );
- BSONObj limits1 = BSON( sf::maxSize(0LL) << sf::currSize(0LL) << sf::draining(false) );
- limitsMap["shard0"] = limits0;
- limitsMap["shard1"] = limits1;
-
- BalancerPolicy::ChunkInfo* c = NULL;
- c = BalancerPolicy::balance( "ns", limitsMap, chunkMap, 0 );
- assert( c != NULL );
- assert( c->to == "shard1" );
- assert( c->from == "shard0" );
- assert( ! c->chunk.isEmpty() );
- }
-
- void caseBalanceEndedDraining(){
- // 2 chunks and 0 chunk (drain completed) shards
- BalancerPolicy::ShardToChunksMap chunkMap;
- vector<BSONObj> chunks;
- chunks.push_back(BSON( "min" << BSON( "x" << BSON( "$minKey"<<1) ) <<
- "max" << BSON( "x" << 49 )));
- chunks.push_back(BSON( "min" << BSON( "x" << 49 ) <<
- "max" << BSON( "x" << BSON( "$maxkey"<<1 ))));
- chunkMap["shard0"] = chunks;
- chunks.clear();
- chunkMap["shard1"] = chunks;
-
- // no limits
- BalancerPolicy::ShardToLimitsMap limitsMap;
- BSONObj limits0 = BSON( sf::maxSize(0LL) << sf::currSize(2LL) << sf::draining(false) );
- BSONObj limits1 = BSON( sf::maxSize(0LL) << sf::currSize(0LL) << sf::draining(true) );
- limitsMap["shard0"] = limits0;
- limitsMap["shard1"] = limits1;
-
- BalancerPolicy::ChunkInfo* c = NULL;
- c = BalancerPolicy::balance( "ns", limitsMap, chunkMap, 0 );
- assert( c == NULL );
- }
-
- void caseBalanceImpasse(){
- // one maxed out, one draining
- // 2 chunks and 0 chunk shards
- BalancerPolicy::ShardToChunksMap chunkMap;
- vector<BSONObj> chunks;
- chunks.push_back(BSON( "min" << BSON( "x" << BSON( "$minKey"<<1) ) <<
- "max" << BSON( "x" << 49 )));
- chunkMap["shard0"] = chunks;
- chunks.clear();
- chunks.push_back(BSON( "min" << BSON( "x" << 49 ) <<
- "max" << BSON( "x" << BSON( "$maxkey"<<1 ))));
- chunkMap["shard1"] = chunks;
-
- // shard0 is draining, shard1 is maxed out
- BalancerPolicy::ShardToLimitsMap limitsMap;
- BSONObj limits0 = BSON( sf::maxSize(0LL) << sf::currSize(2LL) << sf::draining(true) );
- BSONObj limits1 = BSON( sf::maxSize(1LL) << sf::currSize(1LL) << sf::draining(false) );
- limitsMap["shard0"] = limits0;
- limitsMap["shard1"] = limits1;
-
- BalancerPolicy::ChunkInfo* c = NULL;
- c = BalancerPolicy::balance( "ns", limitsMap, chunkMap, 0 );
- assert( c == NULL );
- }
-
- void run(){
- caseSizeMaxedShard();
- caseDrainingShard();
- caseBalanceNormal();
- caseBalanceDraining();
- caseBalanceImpasse();
- log(1) << "policyObjUnitTest passed" << endl;
+ bool BalancerPolicy::hasOpsQueued( BSONObj limits ) {
+ BSONElement opsQueued = limits[ LimitsFields::hasOpsQueued.name() ];
+ if ( opsQueued.eoo() || ! opsQueued.trueValue() ) {
+ return false;
}
- } policyObjUnitTest;
+ return true;
+ }
} // namespace mongo
diff --git a/s/balancer_policy.h b/s/balancer_policy.h
index 3622edc..cef5aa6 100644
--- a/s/balancer_policy.h
+++ b/s/balancer_policy.h
@@ -1,4 +1,4 @@
-// balancer_policy.h
+// @file balancer_policy.h
/**
* Copyright (C) 2010 10gen Inc.
@@ -29,20 +29,20 @@ namespace mongo {
/**
* Returns a suggested chunk to move whithin a collection's shards, given information about
- * space usage and number of chunks for that collection. If the policy doesn't recommend
+ * space usage and number of chunks for that collection. If the policy doesn't recommend
* moving, it returns NULL.
*
* @param ns is the collections namepace.
- * @param shardLimitMap is a map from shardId to an object that describes (for now) space
+ * @param shardLimitMap is a map from shardId to an object that describes (for now) space
* cap and usage. E.g.: { "maxSize" : <size_in_MB> , "usedSize" : <size_in_MB> }.
* @param shardToChunksMap is a map from shardId to chunks that live there. A chunk's format
- * is { }.
+ * is { }.
* @param balancedLastTime is the number of chunks effectively moved in the last round.
* @returns NULL or ChunkInfo of the best move to make towards balacing the collection.
*/
typedef map< string,BSONObj > ShardToLimitsMap;
typedef map< string,vector<BSONObj> > ShardToChunksMap;
- static ChunkInfo* balance( const string& ns, const ShardToLimitsMap& shardToLimitsMap,
+ static ChunkInfo* balance( const string& ns, const ShardToLimitsMap& shardToLimitsMap,
const ShardToChunksMap& shardToChunksMap, int balancedLastTime );
// below exposed for testing purposes only -- treat it as private --
@@ -57,11 +57,16 @@ namespace mongo {
static bool isSizeMaxed( BSONObj shardLimits );
/**
- * Returns true if 'shardLimist' contains a field "draining". Expects the optional field
+ * Returns true if 'shardLimist' contains a field "draining". Expects the optional field
* "isDraining" on 'shrdLimits'.
*/
static bool isDraining( BSONObj shardLimits );
+ /**
+ * Returns true if a shard currently has operations in any of its writeback queues
+ */
+ static bool hasOpsQueued( BSONObj shardLimits );
+
private:
// Convenience types
typedef ShardToChunksMap::const_iterator ShardToChunksIter;
@@ -76,7 +81,16 @@ namespace mongo {
const BSONObj chunk;
ChunkInfo( const string& a_ns , const string& a_to , const string& a_from , const BSONObj& a_chunk )
- : ns( a_ns ) , to( a_to ) , from( a_from ), chunk( a_chunk ){}
+ : ns( a_ns ) , to( a_to ) , from( a_from ), chunk( a_chunk ) {}
+ };
+
+ /**
+ * Field names used in the 'limits' map.
+ */
+ struct LimitsFields {
+ // we use 'draining' and 'maxSize' from the 'shards' collection plus the following
+ static BSONField<long long> currSize; // currently used disk space in bytes
+ static BSONField<bool> hasOpsQueued; // writeback queue is not empty?
};
} // namespace mongo
diff --git a/s/chunk.cpp b/s/chunk.cpp
index 87d7747..1c72535 100644
--- a/s/chunk.cpp
+++ b/s/chunk.cpp
@@ -1,4 +1,4 @@
-// shard.cpp
+// @file chunk.cpp
/**
* Copyright (C) 2008 10gen Inc.
@@ -17,63 +17,62 @@
*/
#include "pch.h"
-#include "chunk.h"
-#include "config.h"
-#include "grid.h"
-#include "../util/unittest.h"
+
#include "../client/connpool.h"
-#include "../client/distlock.h"
#include "../db/queryutil.h"
+#include "../util/unittest.h"
+
+#include "chunk.h"
+#include "config.h"
#include "cursors.h"
+#include "grid.h"
#include "strategy.h"
+#include "client.h"
namespace mongo {
- inline bool allOfType(BSONType type, const BSONObj& o){
+ inline bool allOfType(BSONType type, const BSONObj& o) {
BSONObjIterator it(o);
- while(it.more()){
+ while(it.more()) {
if (it.next().type() != type)
return false;
}
return true;
}
- RWLock chunkSplitLock("rw:chunkSplitLock");
-
// ------- Shard --------
- int Chunk::MaxChunkSize = 1024 * 1024 * 200;
-
- Chunk::Chunk( ChunkManager * manager )
- : _manager(manager),
- _lastmod(0), _modified(false), _dataWritten(0)
- {}
+ string Chunk::chunkMetadataNS = "config.chunks";
+
+ int Chunk::MaxChunkSize = 1024 * 1024 * 64;
+
+ Chunk::Chunk( ChunkManager * manager ) : _manager(manager), _lastmod(0) {
+ _setDataWritten();
+ }
Chunk::Chunk(ChunkManager * info , const BSONObj& min, const BSONObj& max, const Shard& shard)
- : _manager(info), _min(min), _max(max), _shard(shard),
- _lastmod(0), _modified(false), _dataWritten(0)
- {}
+ : _manager(info), _min(min), _max(max), _shard(shard), _lastmod(0) {
+ _setDataWritten();
+ }
+
+ void Chunk::_setDataWritten() {
+ _dataWritten = rand() % ( MaxChunkSize / 5 );
+ }
string Chunk::getns() const {
assert( _manager );
- return _manager->getns();
+ return _manager->getns();
}
- void Chunk::setShard( const Shard& s ){
- _shard = s;
- _manager->_migrationNotification(this);
- _modified = true;
- }
-
- bool Chunk::contains( const BSONObj& obj ) const{
+ bool Chunk::contains( const BSONObj& obj ) const {
return
_manager->getShardKey().compare( getMin() , obj ) <= 0 &&
_manager->getShardKey().compare( obj , getMax() ) < 0;
}
bool ChunkRange::contains(const BSONObj& obj) const {
- // same as Chunk method
- return
+ // same as Chunk method
+ return
_manager->getShardKey().compare( getMin() , obj ) <= 0 &&
_manager->getShardKey().compare( obj , getMax() ) < 0;
}
@@ -85,324 +84,288 @@ namespace mongo {
bool Chunk::maxIsInf() const {
return _manager->getShardKey().globalMax().woCompare( getMax() ) == 0;
}
-
- BSONObj Chunk::pickSplitPoint() const{
- int sort = 0;
-
- if ( minIsInf() ){
- sort = 1;
- }
- else if ( maxIsInf() ){
- sort = -1;
- }
-
- if ( sort ){
- ShardConnection conn( getShard().getConnString() , _manager->getns() );
- Query q;
- if ( sort == 1 )
- q.sort( _manager->getShardKey().key() );
- else {
- BSONObj k = _manager->getShardKey().key();
- BSONObjBuilder r;
-
- BSONObjIterator i(k);
- while( i.more() ) {
- BSONElement e = i.next();
- uassert( 10163 , "can only handle numbers here - which i think is correct" , e.isNumber() );
- r.append( e.fieldName() , -1 * e.number() );
- }
-
- q.sort( r.obj() );
- }
- BSONObj end = conn->findOne( _manager->getns() , q );
- conn.done();
- if ( ! end.isEmpty() )
- return _manager->getShardKey().extractKey( end );
+ BSONObj Chunk::_getExtremeKey( int sort ) const {
+ ShardConnection conn( getShard().getConnString() , _manager->getns() );
+ Query q;
+ if ( sort == 1 ) {
+ q.sort( _manager->getShardKey().key() );
}
-
- BSONObj cmd = BSON( "medianKey" << _manager->getns()
- << "keyPattern" << _manager->getShardKey().key()
- << "min" << getMin()
- << "max" << getMax() );
+ else {
+ // need to invert shard key pattern to sort backwards
+ // TODO: make a helper in ShardKeyPattern?
- ScopedDbConnection conn( getShard().getConnString() );
- BSONObj result;
- if ( ! conn->runCommand( "admin" , cmd , result ) ){
- stringstream ss;
- ss << "medianKey command failed: " << result;
- uassert( 10164 , ss.str() , 0 );
+ BSONObj k = _manager->getShardKey().key();
+ BSONObjBuilder r;
+
+ BSONObjIterator i(k);
+ while( i.more() ) {
+ BSONElement e = i.next();
+ uassert( 10163 , "can only handle numbers here - which i think is correct" , e.isNumber() );
+ r.append( e.fieldName() , -1 * e.number() );
+ }
+
+ q.sort( r.obj() );
}
- BSONObj median = result.getObjectField( "median" ).getOwned();
+ // find the extreme key
+ BSONObj end = conn->findOne( _manager->getns() , q );
conn.done();
+ if ( end.isEmpty() )
+ return BSONObj();
+
+ return _manager->getShardKey().extractKey( end );
+ }
- if (median == getMin()){
- Query q;
- q.minKey(_min).maxKey(_max);
- q.sort(_manager->getShardKey().key());
+ void Chunk::pickMedianKey( BSONObj& medianKey ) const {
+ // Ask the mongod holding this chunk to figure out the split points.
+ ScopedDbConnection conn( getShard().getConnString() );
+ BSONObj result;
+ BSONObjBuilder cmd;
+ cmd.append( "splitVector" , _manager->getns() );
+ cmd.append( "keyPattern" , _manager->getShardKey().key() );
+ cmd.append( "min" , getMin() );
+ cmd.append( "max" , getMax() );
+ cmd.appendBool( "force" , true );
+ BSONObj cmdObj = cmd.obj();
- median = conn->findOne(_manager->getns(), q);
- median = _manager->getShardKey().extractKey( median );
+ if ( ! conn->runCommand( "admin" , cmdObj , result )) {
+ conn.done();
+ ostringstream os;
+ os << "splitVector command (median key) failed: " << result;
+ uassert( 13503 , os.str() , 0 );
}
-
- if ( median < getMin() || median >= getMax() ){
- stringstream ss;
- ss << "medianKey returned value out of range. "
- << " cmd: " << cmd
- << " result: " << result;
- uasserted( 13394 , ss.str() );
+
+ BSONObjIterator it( result.getObjectField( "splitKeys" ) );
+ if ( it.more() ) {
+ medianKey = it.next().Obj().getOwned();
}
-
- return median;
+
+ conn.done();
}
- void Chunk::pickSplitVector( vector<BSONObj>* splitPoints ) const {
+ void Chunk::pickSplitVector( vector<BSONObj>& splitPoints , int chunkSize /* bytes */, int maxPoints, int maxObjs ) const {
// Ask the mongod holding this chunk to figure out the split points.
ScopedDbConnection conn( getShard().getConnString() );
BSONObj result;
BSONObjBuilder cmd;
cmd.append( "splitVector" , _manager->getns() );
cmd.append( "keyPattern" , _manager->getShardKey().key() );
- cmd.append( "maxChunkSize" , Chunk::MaxChunkSize / (1<<20) );
+ cmd.append( "min" , getMin() );
+ cmd.append( "max" , getMax() );
+ cmd.append( "maxChunkSizeBytes" , chunkSize );
+ cmd.append( "maxSplitPoints" , maxPoints );
+ cmd.append( "maxChunkObjects" , maxObjs );
BSONObj cmdObj = cmd.obj();
- if ( ! conn->runCommand( "admin" , cmdObj , result )){
+ if ( ! conn->runCommand( "admin" , cmdObj , result )) {
+ conn.done();
ostringstream os;
os << "splitVector command failed: " << result;
uassert( 13345 , os.str() , 0 );
- }
+ }
BSONObjIterator it( result.getObjectField( "splitKeys" ) );
- while ( it.more() ){
- splitPoints->push_back( it.next().Obj().getOwned() );
+ while ( it.more() ) {
+ splitPoints.push_back( it.next().Obj().getOwned() );
}
conn.done();
}
- ChunkPtr Chunk::split(){
- vector<BSONObj> splitPoints;
- splitPoints.push_back( pickSplitPoint() );
- return multiSplit( splitPoints );
+ ChunkPtr Chunk::singleSplit( bool force , BSONObj& res ) {
+ vector<BSONObj> splitPoint;
+
+ // if splitting is not obligatory we may return early if there are not enough data
+ // we cap the number of objects that would fall in the first half (before the split point)
+ // the rationale is we'll find a split point without traversing all the data
+ if ( ! force ) {
+ vector<BSONObj> candidates;
+ const int maxPoints = 2;
+ const int maxObjs = 250000;
+ pickSplitVector( candidates , getManager()->getCurrentDesiredChunkSize() , maxPoints , maxObjs );
+ if ( candidates.size() <= 1 ) {
+ // no split points means there isn't enough data to split on
+ // 1 split point means we have between half the chunk size to full chunk size
+ // so we shouldn't split
+ log(1) << "chunk not full enough to trigger auto-split" << endl;
+ return ChunkPtr();
+ }
+
+ splitPoint.push_back( candidates.front() );
+
+ }
+ else {
+ // if forcing a split, use the chunk's median key
+ BSONObj medianKey;
+ pickMedianKey( medianKey );
+ if ( ! medianKey.isEmpty() )
+ splitPoint.push_back( medianKey );
+ }
+
+ // We assume that if the chunk being split is the first (or last) one on the collection, this chunk is
+ // likely to see more insertions. Instead of splitting mid-chunk, we use the very first (or last) key
+ // as a split point.
+ if ( minIsInf() ) {
+ splitPoint.clear();
+ BSONObj key = _getExtremeKey( 1 );
+ if ( ! key.isEmpty() ) {
+ splitPoint.push_back( key );
+ }
+
+ }
+ else if ( maxIsInf() ) {
+ splitPoint.clear();
+ BSONObj key = _getExtremeKey( -1 );
+ if ( ! key.isEmpty() ) {
+ splitPoint.push_back( key );
+ }
+ }
+
+ // Normally, we'd have a sound split point here if the chunk is not empty. It's also a good place to
+ // sanity check.
+ if ( splitPoint.empty() || _min == splitPoint.front() || _max == splitPoint.front() ) {
+ log() << "want to split chunk, but can't find split point chunk " << toString()
+ << " got: " << ( splitPoint.empty() ? "<empty>" : splitPoint.front().toString() ) << endl;
+ return ChunkPtr();
+ }
+
+ return multiSplit( splitPoint , res );
}
-
- ChunkPtr Chunk::multiSplit( const vector<BSONObj>& m ){
- const size_t maxSplitPoints = 256;
+
+ ChunkPtr Chunk::multiSplit( const vector<BSONObj>& m , BSONObj& res ) {
+ const size_t maxSplitPoints = 8192;
uassert( 10165 , "can't split as shard doesn't have a manager" , _manager );
uassert( 13332 , "need a split key to split chunk" , !m.empty() );
uassert( 13333 , "can't split a chunk in that many parts", m.size() < maxSplitPoints );
- uassert( 13003 , "can't split a chunk with only one distinct value" , _min.woCompare(_max) );
+ uassert( 13003 , "can't split a chunk with only one distinct value" , _min.woCompare(_max) );
- DistributedLock lockSetup( ConnectionString( modelServer() , ConnectionString::SYNC ) , getns() );
- dist_lock_try dlk( &lockSetup , string("split-") + toString() );
- uassert( 10166 , "locking namespace failed" , dlk.got() );
-
- {
- ShardChunkVersion onServer = getVersionOnConfigServer();
- ShardChunkVersion mine = _lastmod;
- if ( onServer > mine ){
- stringstream ss;
- ss << "mulitSplit failing because config not up to date"
- << " onServer: " << onServer.toString()
- << " mine: " << mine.toString();
-
- //reload config
- grid.getDBConfig(_manager->_ns)->getChunkManager(_manager->_ns, true);
-
- uasserted( 13387 , ss.str() );
- }
- }
+ ScopedDbConnection conn( getShard().getConnString() );
- BSONObjBuilder detail;
- appendShortVersion( "before" , detail );
- log(1) << "before split on " << m.size() << " points " << toString() << endl;
+ BSONObjBuilder cmd;
+ cmd.append( "splitChunk" , _manager->getns() );
+ cmd.append( "keyPattern" , _manager->getShardKey().key() );
+ cmd.append( "min" , getMin() );
+ cmd.append( "max" , getMax() );
+ cmd.append( "from" , getShard().getConnString() );
+ cmd.append( "splitKeys" , m );
+ cmd.append( "shardId" , genID() );
+ cmd.append( "configdb" , configServer.modelServer() );
+ BSONObj cmdObj = cmd.obj();
- // Iterate over the split points in 'm', splitting off a new chunk per entry. That chunk's range
- // covers until the next entry in 'm' or _max .
- vector<ChunkPtr> newChunks;
- vector<BSONObj>::const_iterator i = m.begin();
- BSONObj nextPoint = i->getOwned();
- _modified = true;
- do {
- BSONObj splitPoint = nextPoint;
- log(4) << "splitPoint: " << splitPoint << endl;
- nextPoint = (++i != m.end()) ? i->getOwned() : _max.getOwned();
- log(4) << "nextPoint: " << nextPoint << endl;
-
- if ( nextPoint <= splitPoint) {
- stringstream ss;
- ss << "multiSplit failing because keys min: " << splitPoint << " and max: " << nextPoint
- << " do not define a valid chunk";
- uasserted( 13395, ss.str() );
- }
+ if ( ! conn->runCommand( "admin" , cmdObj , res )) {
+ warning() << "splitChunk failed - cmd: " << cmdObj << " result: " << res << endl;
+ conn.done();
+
+ // reloading won't stricly solve all problems, e.g. the collection's metdata lock can be taken
+ // but we issue here so that mongos may refresh wihtout needing to be written/read against
+ _manager->_reload();
+
+ return ChunkPtr();
+ }
- ChunkPtr c( new Chunk( _manager, splitPoint , nextPoint , _shard) );
- c->_modified = true;
- newChunks.push_back( c );
- } while ( i != m.end() );
+ conn.done();
+ _manager->_reload();
- // Have the chunk manager reflect the key change for the first chunk and create an entry for every
- // new chunk spawned by it.
+ // The previous multisplit logic adjusted the boundaries of 'this' chunk. Any call to 'this' object hereafter
+ // will see a different _max for the chunk.
+ // TODO Untie this dependency since, for metadata purposes, the reload() above already fixed boundaries
{
rwlock lk( _manager->_lock , true );
setMax(m[0].getOwned());
DEV assert( shared_from_this() );
_manager->_chunkMap[_max] = shared_from_this();
-
- for ( vector<ChunkPtr>::const_iterator it = newChunks.begin(); it != newChunks.end(); ++it ){
- ChunkPtr s = *it;
- _manager->_chunkMap[s->getMax()] = s;
- }
- }
-
- log(1) << "after split adjusted range: " << toString() << endl;
- for ( vector<ChunkPtr>::const_iterator it = newChunks.begin(); it != newChunks.end(); ++it ){
- ChunkPtr s = *it;
- log(1) << "after split created new chunk: " << s->toString() << endl;
- }
-
- // Save the new key boundaries in the configDB.
- _manager->save( false );
-
- // Log all these changes in the configDB's log. We log a simple split differently than a multi-split.
- if ( newChunks.size() == 1) {
- appendShortVersion( "left" , detail );
- newChunks[0]->appendShortVersion( "right" , detail );
- configServer.logChange( "split" , _manager->getns(), detail.obj() );
-
- } else {
- BSONObj beforeDetailObj = detail.obj();
- BSONObj firstDetailObj = beforeDetailObj.getOwned();
- const int newChunksSize = newChunks.size();
-
- BSONObjBuilder firstDetail;
- firstDetail.appendElements( beforeDetailObj );
- firstDetail.append( "number" , 0 );
- firstDetail.append( "of" , newChunksSize );
- appendShortVersion( "chunk" , firstDetail );
- configServer.logChange( "multi-split" , _manager->getns() , firstDetail.obj() );
-
- for ( int i=0; i < newChunksSize; i++ ){
- BSONObjBuilder chunkDetail;
- chunkDetail.appendElements( beforeDetailObj );
- chunkDetail.append( "number", i+1 );
- chunkDetail.append( "of" , newChunksSize );
- newChunks[i]->appendShortVersion( "chunk" , chunkDetail );
- configServer.logChange( "multi-split" , _manager->getns() , chunkDetail.obj() );
- }
}
- return newChunks[0];
+ // return the second half, if a single split, or the first new chunk, if a multisplit.
+ return _manager->findChunk( m[0] );
}
- bool Chunk::moveAndCommit( const Shard& to , string& errmsg ){
+ bool Chunk::moveAndCommit( const Shard& to , long long chunkSize /* bytes */, BSONObj& res ) {
uassert( 10167 , "can't move shard to its current location!" , getShard() != to );
-
+
log() << "moving chunk ns: " << _manager->getns() << " moving ( " << toString() << ") " << _shard.toString() << " -> " << to.toString() << endl;
-
+
Shard from = _shard;
-
+
ScopedDbConnection fromconn( from);
- BSONObj res;
bool worked = fromconn->runCommand( "admin" ,
- BSON( "moveChunk" << _manager->getns() <<
- "from" << from.getConnString() <<
- "to" << to.getConnString() <<
- "min" << _min <<
- "max" << _max <<
- "shardId" << genID() <<
- "configdb" << configServer.modelServer()
- ) ,
+ BSON( "moveChunk" << _manager->getns() <<
+ "from" << from.getConnString() <<
+ "to" << to.getConnString() <<
+ "min" << _min <<
+ "max" << _max <<
+ "maxChunkSizeBytes" << chunkSize <<
+ "shardId" << genID() <<
+ "configdb" << configServer.modelServer()
+ ) ,
res
- );
-
+ );
+
fromconn.done();
- if ( worked ){
- _manager->_reload();
- return true;
- }
-
- errmsg = res["errmsg"].String();
- errmsg += " " + res.toString();
- return false;
+ // if succeeded, needs to reload to pick up the new location
+ // if failed, mongos may be stale
+ // reload is excessive here as the failure could be simply because collection metadata is taken
+ _manager->_reload();
+
+ return worked;
}
-
- bool Chunk::splitIfShould( long dataWritten ){
+
+ bool Chunk::splitIfShould( long dataWritten ) {
LastError::Disabled d( lastError.get() );
+
try {
- return _splitIfShould( dataWritten );
- }
- catch ( std::exception& e ){
- log( LL_ERROR ) << "splitIfShould failed: " << e.what() << endl;
- return false;
- }
- }
+ _dataWritten += dataWritten;
+ int splitThreshold = getManager()->getCurrentDesiredChunkSize();
+ if ( minIsInf() || maxIsInf() ) {
+ splitThreshold = (int) ((double)splitThreshold * .9);
+ }
- bool Chunk::_splitIfShould( long dataWritten ){
- _dataWritten += dataWritten;
-
- // split faster in early chunks helps spread out an initial load better
- int splitThreshold;
- const int minChunkSize = 1 << 20; // 1 MBytes
- int numChunks = getManager()->numChunks();
- if ( numChunks < 10 ){
- splitThreshold = max( MaxChunkSize / 4 , minChunkSize );
- } else if ( numChunks < 20 ){
- splitThreshold = max( MaxChunkSize / 2 , minChunkSize );
- } else {
- splitThreshold = max( MaxChunkSize , minChunkSize );
- }
-
- if ( minIsInf() || maxIsInf() ){
- splitThreshold = (int) ((double)splitThreshold * .9);
- }
+ if ( _dataWritten < splitThreshold / 5 )
+ return false;
- if ( _dataWritten < splitThreshold / 5 )
- return false;
-
- if ( ! chunkSplitLock.lock_try(0) )
- return false;
-
- rwlock lk( chunkSplitLock , 1 , true );
+ log(1) << "about to initiate autosplit: " << *this << " dataWritten: " << _dataWritten << " splitThreshold: " << splitThreshold << endl;
- log(3) << "\t splitIfShould : " << *this << endl;
+ _dataWritten = 0; // reset so we check often enough
- _dataWritten = 0;
-
- BSONObj splitPoint = pickSplitPoint();
- if ( splitPoint.isEmpty() || _min == splitPoint || _max == splitPoint) {
- log() << "SHARD PROBLEM** shard is too big, but can't split: " << toString() << endl;
- return false;
- }
+ BSONObj res;
+ ChunkPtr newShard = singleSplit( false /* does not force a split if not enough data */ , res );
+ if ( newShard.get() == NULL ) {
+ // singleSplit would have issued a message if we got here
+ _dataWritten = 0; // this means there wasn't enough data to split, so don't want to try again until considerable more data
+ return false;
+ }
- long size = getPhysicalSize();
- if ( size < splitThreshold )
- return false;
-
- log() << "autosplitting " << _manager->getns() << " size: " << size << " shard: " << toString()
- << " on: " << splitPoint << "(splitThreshold " << splitThreshold << ")" << endl;
+ log() << "autosplitted " << _manager->getns() << " shard: " << toString()
+ << " on: " << newShard->getMax() << "(splitThreshold " << splitThreshold << ")"
+#ifdef _DEBUG
+ << " size: " << getPhysicalSize() // slow - but can be usefule when debugging
+#endif
+ << endl;
- vector<BSONObj> splitPoints;
- splitPoints.push_back( splitPoint );
- ChunkPtr newShard = multiSplit( splitPoints );
+ moveIfShould( newShard );
- moveIfShould( newShard );
-
- return true;
+ return true;
+
+ }
+ catch ( std::exception& e ) {
+ // if the collection lock is taken (e.g. we're migrating), it is fine for the split to fail.
+ warning() << "could have autosplit on collection: " << _manager->getns() << " but: " << e.what() << endl;
+ return false;
+ }
}
- bool Chunk::moveIfShould( ChunkPtr newChunk ){
+ bool Chunk::moveIfShould( ChunkPtr newChunk ) {
ChunkPtr toMove;
-
- if ( newChunk->countObjects(2) <= 1 ){
+
+ if ( newChunk->countObjects(2) <= 1 ) {
toMove = newChunk;
}
- else if ( this->countObjects(2) <= 1 ){
+ else if ( this->countObjects(2) <= 1 ) {
DEV assert( shared_from_this() );
toMove = shared_from_this();
}
@@ -412,45 +375,46 @@ namespace mongo {
}
assert( toMove );
-
- Shard newLocation = Shard::pick();
- if ( getShard() == newLocation ){
- // if this is the best server, then we shouldn't do anything!
- log(1) << "not moving chunk: " << toString() << " b/c would move to same place " << newLocation.toString() << " -> " << getShard().toString() << endl;
+
+ Shard newLocation = Shard::pick( getShard() );
+ if ( getShard() == newLocation ) {
+ // if this is the best shard, then we shouldn't do anything (Shard::pick already logged our shard).
+ log(1) << "recently split chunk: " << toString() << "already in the best shard" << endl;
return 0;
}
log() << "moving chunk (auto): " << toMove->toString() << " to: " << newLocation.toString() << " #objects: " << toMove->countObjects() << endl;
- string errmsg;
- massert( 10412 , (string)"moveAndCommit failed: " + errmsg ,
- toMove->moveAndCommit( newLocation , errmsg ) );
-
+ BSONObj res;
+ massert( 10412 ,
+ str::stream() << "moveAndCommit failed: " << res ,
+ toMove->moveAndCommit( newLocation , MaxChunkSize , res ) );
+
return true;
}
- long Chunk::getPhysicalSize() const{
+ long Chunk::getPhysicalSize() const {
ScopedDbConnection conn( getShard().getConnString() );
-
+
BSONObj result;
- uassert( 10169 , "datasize failed!" , conn->runCommand( "admin" ,
- BSON( "datasize" << _manager->getns()
- << "keyPattern" << _manager->getShardKey().key()
- << "min" << getMin()
- << "max" << getMax()
- << "maxSize" << ( MaxChunkSize + 1 )
- << "estimate" << true
- ) , result ) );
-
+ uassert( 10169 , "datasize failed!" , conn->runCommand( "admin" ,
+ BSON( "datasize" << _manager->getns()
+ << "keyPattern" << _manager->getShardKey().key()
+ << "min" << getMin()
+ << "max" << getMax()
+ << "maxSize" << ( MaxChunkSize + 1 )
+ << "estimate" << true
+ ) , result ) );
+
conn.done();
return (long)result["size"].number();
}
- int Chunk::countObjects(int maxCount) const {
+ int Chunk::countObjects(int maxCount) const {
static const BSONObj fields = BSON("_id" << 1 );
ShardConnection conn( getShard() , _manager->getns() );
-
+
// not using regular count as this is more flexible and supports $min/$max
Query q = Query().minKey(_min).maxKey(_max);
int n;
@@ -458,33 +422,33 @@ namespace mongo {
auto_ptr<DBClientCursor> c = conn->query(_manager->getns(), q, maxCount, 0, &fields);
assert( c.get() );
n = c->itcount();
- }
+ }
conn.done();
return n;
}
- void Chunk::appendShortVersion( const char * name , BSONObjBuilder& b ){
+ void Chunk::appendShortVersion( const char * name , BSONObjBuilder& b ) {
BSONObjBuilder bb( b.subobjStart( name ) );
bb.append( "min" , _min );
bb.append( "max" , _max );
bb.done();
}
-
- bool Chunk::operator==( const Chunk& s ) const{
- return
+
+ bool Chunk::operator==( const Chunk& s ) const {
+ return
_manager->getShardKey().compare( _min , s._min ) == 0 &&
_manager->getShardKey().compare( _max , s._max ) == 0
;
}
- void Chunk::serialize(BSONObjBuilder& to,ShardChunkVersion myLastMod){
-
+ void Chunk::serialize(BSONObjBuilder& to,ShardChunkVersion myLastMod) {
+
to.append( "_id" , genID( _manager->getns() , _min ) );
- if ( myLastMod.isSet() ){
+ if ( myLastMod.isSet() ) {
to.appendTimestamp( "lastmod" , myLastMod );
}
- else if ( _lastmod.isSet() ){
+ else if ( _lastmod.isSet() ) {
assert( _lastmod > 0 && _lastmod < 1000 );
to.appendTimestamp( "lastmod" , _lastmod );
}
@@ -503,15 +467,15 @@ namespace mongo {
buf << ns << "-";
BSONObjIterator i(o);
- while ( i.more() ){
+ while ( i.more() ) {
BSONElement e = i.next();
buf << e.fieldName() << "_" << e.toString(false, true);
}
return buf.str();
}
-
- void Chunk::unserialize(const BSONObj& from){
+
+ void Chunk::unserialize(const BSONObj& from) {
string ns = from.getStringField( "ns" );
_shard.reset( from.getStringField( "shard" ) );
@@ -520,15 +484,15 @@ namespace mongo {
BSONElement e = from["minDotted"];
- if (e.eoo()){
+ if (e.eoo()) {
_min = from.getObjectField( "min" ).getOwned();
_max = from.getObjectField( "max" ).getOwned();
- }
+ }
else { // TODO delete this case after giving people a chance to migrate
_min = e.embeddedObject().getOwned();
_max = from.getObjectField( "maxDotted" ).getOwned();
}
-
+
uassert( 10170 , "Chunk needs a ns" , ! ns.empty() );
uassert( 13327 , "Chunk ns must match server ns" , ns == _manager->getns() );
@@ -538,26 +502,13 @@ namespace mongo {
uassert( 10173 , "Chunk needs a max" , ! _max.isEmpty() );
}
- string Chunk::modelServer() const {
- // TODO: this could move around?
- return configServer.modelServer();
- }
-
- ShardChunkVersion Chunk::getVersionOnConfigServer() const {
- ScopedDbConnection conn( modelServer() );
- BSONObj o = conn->findOne( ShardNS::chunk , BSON( "_id" << genID() ) );
- conn.done();
- return o["lastmod"];
- }
-
string Chunk::toString() const {
stringstream ss;
ss << "ns:" << _manager->getns() << " at: " << _shard.toString() << " lastmod: " << _lastmod.toString() << " min: " << _min << " max: " << _max;
return ss.str();
}
-
-
- ShardKeyPattern Chunk::skey() const{
+
+ ShardKeyPattern Chunk::skey() const {
return _manager->getShardKey();
}
@@ -565,75 +516,66 @@ namespace mongo {
AtomicUInt ChunkManager::NextSequenceNumber = 1;
- ChunkManager::ChunkManager( DBConfig * config , string ns , ShardKeyPattern pattern , bool unique ) :
- _config( config ) , _ns( ns ) ,
- _key( pattern ) , _unique( unique ) ,
- _sequenceNumber( ++NextSequenceNumber ), _lock("rw:ChunkManager")
- {
- _reload_inlock();
-
- if ( _chunkMap.empty() ){
- ChunkPtr c( new Chunk(this, _key.globalMin(), _key.globalMax(), config->getPrimary()) );
- c->setModified( true );
-
- _chunkMap[c->getMax()] = c;
- _chunkRanges.reloadAll(_chunkMap);
-
- _shards.insert(c->getShard());
-
- save_inlock( true );
- log() << "no chunks for:" << ns << " so creating first: " << c->toString() << endl;
- }
+ ChunkManager::ChunkManager( string ns , ShardKeyPattern pattern , bool unique ) :
+ _ns( ns ) , _key( pattern ) , _unique( unique ) , _lock("rw:ChunkManager"),
+ _nsLock( ConnectionString( configServer.modelServer() , ConnectionString::SYNC ) , ns ) {
+ _reload_inlock(); // will set _sequenceNumber
}
-
- ChunkManager::~ChunkManager(){
+
+ ChunkManager::~ChunkManager() {
_chunkMap.clear();
_chunkRanges.clear();
_shards.clear();
}
-
- void ChunkManager::_reload(){
+
+ void ChunkManager::_reload() {
rwlock lk( _lock , true );
_reload_inlock();
}
- void ChunkManager::_reload_inlock(){
+ void ChunkManager::_reload_inlock() {
int tries = 3;
- while (tries--){
+ while (tries--) {
_chunkMap.clear();
_chunkRanges.clear();
_shards.clear();
_load();
- if (_isValid()){
+ if (_isValid()) {
_chunkRanges.reloadAll(_chunkMap);
+
+ // The shard versioning mechanism hinges on keeping track of the number of times we reloaded ChunkManager's.
+ // Increasing this number here will prompt checkShardVersion() to refresh the connection-level versions to
+ // the most up to date value.
+ _sequenceNumber = ++NextSequenceNumber;
+
return;
}
- if (_chunkMap.size() < 10){
+ if (_chunkMap.size() < 10) {
_printChunks();
}
+
sleepmillis(10 * (3-tries));
- sleepsecs(10);
}
- msgasserted(13282, "Couldn't load a valid config for " + _ns + " after 3 tries. Giving up");
-
+
+ msgasserted(13282, "Couldn't load a valid config for " + _ns + " after 3 attempts. Please try again.");
+
}
- void ChunkManager::_load(){
- static Chunk temp(0);
-
- ScopedDbConnection conn( temp.modelServer() );
+ void ChunkManager::_load() {
+ ScopedDbConnection conn( configServer.modelServer() );
- auto_ptr<DBClientCursor> cursor = conn->query(temp.getNS(), QUERY("ns" << _ns).sort("lastmod",1), 0, 0, 0, 0,
- (DEBUG_BUILD ? 2 : 1000000)); // batch size. Try to induce potential race conditions in debug builds
+ // TODO really need the sort?
+ auto_ptr<DBClientCursor> cursor = conn->query( Chunk::chunkMetadataNS, QUERY("ns" << _ns).sort("lastmod",1), 0, 0, 0, 0,
+ (DEBUG_BUILD ? 2 : 1000000)); // batch size. Try to induce potential race conditions in debug builds
assert( cursor.get() );
- while ( cursor->more() ){
+ while ( cursor->more() ) {
BSONObj d = cursor->next();
- if ( d["isMaxMarker"].trueValue() ){
+ if ( d["isMaxMarker"].trueValue() ) {
continue;
}
-
+
ChunkPtr c( new Chunk( this ) );
c->unserialize( d );
@@ -655,10 +597,10 @@ namespace mongo {
ENSURE(allOfType(MaxKey, prior(_chunkMap.end())->second->getMax()));
// Make sure there are no gaps or overlaps
- for (ChunkMap::const_iterator it=boost::next(_chunkMap.begin()), end=_chunkMap.end(); it != end; ++it){
+ for (ChunkMap::const_iterator it=boost::next(_chunkMap.begin()), end=_chunkMap.end(); it != end; ++it) {
ChunkMap::const_iterator last = prior(it);
- if (!(it->second->getMin() == last->second->getMax())){
+ if (!(it->second->getMin() == last->second->getMax())) {
PRINT(it->second->toString());
PRINT(it->second->getMin());
PRINT(last->second->getMax());
@@ -677,54 +619,101 @@ namespace mongo {
}
}
- bool ChunkManager::hasShardKey( const BSONObj& obj ){
+ bool ChunkManager::hasShardKey( const BSONObj& obj ) {
return _key.hasShardKey( obj );
}
- ChunkPtr ChunkManager::findChunk( const BSONObj & obj , bool retry ){
+ void ChunkManager::createFirstChunk( const Shard& shard ) {
+ assert( _chunkMap.size() == 0 );
+
+ ChunkPtr c( new Chunk(this, _key.globalMin(), _key.globalMax(), shard ) );
+
+ // this is the first chunk; start the versioning from scratch
+ ShardChunkVersion version;
+ version.incMajor();
+
+ // build update for the chunk collection
+ BSONObjBuilder chunkBuilder;
+ c->serialize( chunkBuilder , version );
+ BSONObj chunkCmd = chunkBuilder.obj();
+
+ log() << "about to create first chunk for: " << _ns << endl;
+
+ ScopedDbConnection conn( configServer.modelServer() );
+ BSONObj res;
+ conn->update( Chunk::chunkMetadataNS, QUERY( "_id" << c->genID() ), chunkCmd, true, false );
+
+ string errmsg = conn->getLastError();
+ if ( errmsg.size() ) {
+ stringstream ss;
+ ss << "saving first chunk failed. cmd: " << chunkCmd << " result: " << errmsg;
+ log( LL_ERROR ) << ss.str() << endl;
+ msgasserted( 13592 , ss.str() ); // assert(13592)
+ }
+
+ conn.done();
+
+ // every instance of ChunkManager has a unique sequence number; callers of ChunkManager may
+ // inquiry about whether there were changes in chunk configuration (see re/load() calls) since
+ // the last access to ChunkManager by checking the sequence number
+ _sequenceNumber = ++NextSequenceNumber;
+
+ _chunkMap[c->getMax()] = c;
+ _chunkRanges.reloadAll(_chunkMap);
+ _shards.insert(c->getShard());
+ c->setLastmod(version);
+
+ // the ensure index will have the (desired) indirect effect of creating the collection on the
+ // assigned shard, as it sets up the index over the sharding keys.
+ ensureIndex_inlock();
+
+ log() << "successfully created first chunk for " << c->toString() << endl;
+ }
+
+ ChunkPtr ChunkManager::findChunk( const BSONObj & obj , bool retry ) {
BSONObj key = _key.extractKey(obj);
-
+
{
- rwlock lk( _lock , false );
-
+ rwlock lk( _lock , false );
+
BSONObj foo;
ChunkPtr c;
{
ChunkMap::iterator it = _chunkMap.upper_bound(key);
- if (it != _chunkMap.end()){
+ if (it != _chunkMap.end()) {
foo = it->first;
c = it->second;
}
}
-
- if ( c ){
+
+ if ( c ) {
if ( c->contains( obj ) )
return c;
-
+
PRINT(foo);
PRINT(*c);
PRINT(key);
-
+
_reload_inlock();
massert(13141, "Chunk map pointed to incorrect chunk", false);
}
}
- if ( retry ){
+ if ( retry ) {
stringstream ss;
ss << "couldn't find a chunk aftry retry which should be impossible extracted: " << key;
throw UserException( 8070 , ss.str() );
}
-
+
log() << "ChunkManager: couldn't find chunk for: " << key << " going to retry" << endl;
_reload_inlock();
return findChunk( obj , true );
}
ChunkPtr ChunkManager::findChunkOnServer( const Shard& shard ) const {
- rwlock lk( _lock , false );
-
- for ( ChunkMap::const_iterator i=_chunkMap.begin(); i!=_chunkMap.end(); ++i ){
+ rwlock lk( _lock , false );
+
+ for ( ChunkMap::const_iterator i=_chunkMap.begin(); i!=_chunkMap.end(); ++i ) {
ChunkPtr c = i->second;
if ( c->getShard() == shard )
return c;
@@ -733,20 +722,33 @@ namespace mongo {
return ChunkPtr();
}
- void ChunkManager::getShardsForQuery( set<Shard>& shards , const BSONObj& query ){
- rwlock lk( _lock , false );
+ void ChunkManager::getShardsForQuery( set<Shard>& shards , const BSONObj& query ) {
+ rwlock lk( _lock , false );
DEV PRINT(query);
//TODO look into FieldRangeSetOr
FieldRangeOrSet fros(_ns.c_str(), query, false);
- uassert(13088, "no support for special queries yet", fros.getSpecial().empty());
+
+ const string special = fros.getSpecial();
+ if (special == "2d") {
+ BSONForEach(field, query) {
+ if (getGtLtOp(field) == BSONObj::opNEAR) {
+ uassert(13501, "use geoNear command rather than $near query", false);
+ // TODO: convert to geoNear rather than erroring out
+ }
+ // $within queries are fine
+ }
+ }
+ else if (!special.empty()) {
+ uassert(13502, "unrecognized special query type: " + special, false);
+ }
do {
boost::scoped_ptr<FieldRangeSet> frs (fros.topFrs());
{
// special case if most-significant field isn't in query
FieldRange range = frs->range(_key.key().firstElement().fieldName());
- if ( !range.nontrivial() ){
+ if ( !range.nontrivial() ) {
DEV PRINT(range.nontrivial());
getAllShards(shards);
return;
@@ -754,7 +756,7 @@ namespace mongo {
}
BoundList ranges = frs->indexBounds(_key.key(), 1);
- for (BoundList::const_iterator it=ranges.begin(), end=ranges.end(); it != end; ++it){
+ for (BoundList::const_iterator it=ranges.begin(), end=ranges.end(); it != end; ++it) {
BSONObj minObj = it->first.replaceFieldNames(_key.key());
BSONObj maxObj = it->second.replaceFieldNames(_key.key());
@@ -765,35 +767,36 @@ namespace mongo {
min = _chunkRanges.upper_bound(minObj);
max = _chunkRanges.upper_bound(maxObj);
- assert(min != _chunkRanges.ranges().end());
+ massert( 13507 , str::stream() << "invalid chunk config minObj: " << minObj , min != _chunkRanges.ranges().end());
// make max non-inclusive like end iterators
if(max != _chunkRanges.ranges().end())
++max;
- for (ChunkRangeMap::const_iterator it=min; it != max; ++it){
+ for (ChunkRangeMap::const_iterator it=min; it != max; ++it) {
shards.insert(it->second->getShard());
}
// once we know we need to visit all shards no need to keep looping
//if (shards.size() == _shards.size())
- //return;
+ //return;
}
if (fros.moreOrClauses())
fros.popOrClause();
- } while (fros.moreOrClauses());
+ }
+ while (fros.moreOrClauses());
}
- void ChunkManager::getShardsForRange(set<Shard>& shards, const BSONObj& min, const BSONObj& max){
+ void ChunkManager::getShardsForRange(set<Shard>& shards, const BSONObj& min, const BSONObj& max) {
uassert(13405, "min must have shard key", hasShardKey(min));
uassert(13406, "max must have shard key", hasShardKey(max));
ChunkRangeMap::const_iterator it = _chunkRanges.upper_bound(min);
ChunkRangeMap::const_iterator end = _chunkRanges.lower_bound(max);
- for (; it!=end; ++ it){
+ for (; it!=end; ++ it) {
shards.insert(it->second->getShard());
// once we know we need to visit all shards no need to keep looping
@@ -802,282 +805,165 @@ namespace mongo {
}
}
- void ChunkManager::getAllShards( set<Shard>& all ){
- rwlock lk( _lock , false );
+ void ChunkManager::getAllShards( set<Shard>& all ) {
+ rwlock lk( _lock , false );
all.insert(_shards.begin(), _shards.end());
}
-
- void ChunkManager::ensureIndex_inlock(){
+
+ void ChunkManager::ensureIndex_inlock() {
//TODO in parallel?
- for ( set<Shard>::const_iterator i=_shards.begin(); i!=_shards.end(); ++i ){
+ for ( set<Shard>::const_iterator i=_shards.begin(); i!=_shards.end(); ++i ) {
ScopedDbConnection conn( i->getConnString() );
- conn->ensureIndex( getns() , getShardKey().key() , _unique );
+ conn->ensureIndex( getns() , getShardKey().key() , _unique , "" , false /* do not cache ensureIndex SERVER-1691 */ );
conn.done();
}
}
-
- void ChunkManager::drop( ChunkManagerPtr me ){
- rwlock lk( _lock , true );
+
+ void ChunkManager::drop( ChunkManagerPtr me ) {
+ rwlock lk( _lock , true );
configServer.logChange( "dropCollection.start" , _ns , BSONObj() );
-
- DistributedLock lockSetup( ConnectionString( configServer.modelServer() , ConnectionString::SYNC ) , getns() );
- dist_lock_try dlk( &lockSetup , "drop" );
- uassert( 13331 , "locking namespace failed" , dlk.got() );
-
+
+ dist_lock_try dlk( &_nsLock , "drop" );
+ uassert( 13331 , "collection's metadata is undergoing changes. Please try again." , dlk.got() );
+
uassert( 10174 , "config servers not all up" , configServer.allUp() );
-
+
set<Shard> seen;
-
+
log(1) << "ChunkManager::drop : " << _ns << endl;
// lock all shards so no one can do a split/migrate
- for ( ChunkMap::const_iterator i=_chunkMap.begin(); i!=_chunkMap.end(); ++i ){
+ for ( ChunkMap::const_iterator i=_chunkMap.begin(); i!=_chunkMap.end(); ++i ) {
ChunkPtr c = i->second;
seen.insert( c->getShard() );
}
-
- log(1) << "ChunkManager::drop : " << _ns << "\t all locked" << endl;
+
+ log(1) << "ChunkManager::drop : " << _ns << "\t all locked" << endl;
// wipe my meta-data
_chunkMap.clear();
_chunkRanges.clear();
_shards.clear();
-
+
// delete data from mongod
- for ( set<Shard>::iterator i=seen.begin(); i!=seen.end(); i++ ){
+ for ( set<Shard>::iterator i=seen.begin(); i!=seen.end(); i++ ) {
ScopedDbConnection conn( *i );
conn->dropCollection( _ns );
conn.done();
}
-
- log(1) << "ChunkManager::drop : " << _ns << "\t removed shard data" << endl;
- // clean up database meta-data
- uassert( 10176 , "no sharding data?" , _config->removeSharding( _ns ) );
-
+ log(1) << "ChunkManager::drop : " << _ns << "\t removed shard data" << endl;
+
// remove chunk data
- static Chunk temp(0);
- ScopedDbConnection conn( temp.modelServer() );
- conn->remove( temp.getNS() , BSON( "ns" << _ns ) );
+ ScopedDbConnection conn( configServer.modelServer() );
+ conn->remove( Chunk::chunkMetadataNS , BSON( "ns" << _ns ) );
conn.done();
- log(1) << "ChunkManager::drop : " << _ns << "\t removed chunk data" << endl;
-
- for ( set<Shard>::iterator i=seen.begin(); i!=seen.end(); i++ ){
+ log(1) << "ChunkManager::drop : " << _ns << "\t removed chunk data" << endl;
+
+ for ( set<Shard>::iterator i=seen.begin(); i!=seen.end(); i++ ) {
ScopedDbConnection conn( *i );
BSONObj res;
if ( ! setShardVersion( conn.conn() , _ns , 0 , true , res ) )
- throw UserException( 8071 , (string)"OH KNOW, cleaning up after drop failed: " + res.toString() );
+ throw UserException( 8071 , str::stream() << "cleaning up after drop failed: " << res );
conn.done();
}
- log(1) << "ChunkManager::drop : " << _ns << "\t DONE" << endl;
+ log(1) << "ChunkManager::drop : " << _ns << "\t DONE" << endl;
configServer.logChange( "dropCollection" , _ns , BSONObj() );
}
-
- void ChunkManager::save( bool major ){
- rwlock lk( _lock , true );
- save_inlock( major );
- }
-
- void ChunkManager::save_inlock( bool major ){
-
- ShardChunkVersion a = getVersion_inlock();
- assert( a > 0 || _chunkMap.size() <= 1 );
- ShardChunkVersion nextChunkVersion = a;
- nextChunkVersion.inc( major );
-
- vector<ChunkPtr> toFix;
- vector<ShardChunkVersion> newVersions;
-
- BSONObjBuilder cmdBuilder;
- BSONArrayBuilder updates( cmdBuilder.subarrayStart( "applyOps" ) );
-
-
- int numOps = 0;
- for ( ChunkMap::const_iterator i=_chunkMap.begin(); i!=_chunkMap.end(); ++i ){
- ChunkPtr c = i->second;
- if ( ! c->getModified() )
- continue;
-
- numOps++;
- _sequenceNumber = ++NextSequenceNumber;
-
- ShardChunkVersion myVersion = nextChunkVersion;
- nextChunkVersion.incMinor();
- toFix.push_back( c );
- newVersions.push_back( myVersion );
-
- BSONObjBuilder op;
- op.append( "op" , "u" );
- op.appendBool( "b" , true );
- op.append( "ns" , ShardNS::chunk );
-
- BSONObjBuilder n( op.subobjStart( "o" ) );
- c->serialize( n , myVersion );
- n.done();
-
- BSONObjBuilder q( op.subobjStart( "o2" ) );
- q.append( "_id" , c->genID() );
- q.done();
-
- updates.append( op.obj() );
- }
-
- if ( numOps == 0 )
- return;
-
- updates.done();
-
- if ( a > 0 || _chunkMap.size() > 1 ){
- BSONArrayBuilder temp( cmdBuilder.subarrayStart( "preCondition" ) );
- BSONObjBuilder b;
- b.append( "ns" , ShardNS::chunk );
- b.append( "q" , BSON( "query" << BSON( "ns" << _ns ) << "orderby" << BSON( "lastmod" << -1 ) ) );
- {
- BSONObjBuilder bb( b.subobjStart( "res" ) );
- bb.appendTimestamp( "lastmod" , a );
- bb.done();
- }
- temp.append( b.obj() );
- temp.done();
- }
- BSONObj cmd = cmdBuilder.obj();
-
- log(7) << "ChunkManager::save update: " << cmd << endl;
-
- ScopedDbConnection conn( Chunk(0).modelServer() );
- BSONObj res;
- bool ok = conn->runCommand( "config" , cmd , res );
- conn.done();
-
- if ( ! ok ){
- stringstream ss;
- ss << "saving chunks failed. cmd: " << cmd << " result: " << res;
- log( LL_ERROR ) << ss.str() << endl;
- msgasserted( 13327 , ss.str() );
- }
-
- for ( unsigned i=0; i<toFix.size(); i++ ){
- toFix[i]->_lastmod = newVersions[i];
- toFix[i]->setModified( false );
- }
-
- massert( 10417 , "how did version get smalled" , getVersion_inlock() >= a );
-
- ensureIndex_inlock(); // TODO: this is too aggressive - but not really sooo bad
- }
-
void ChunkManager::maybeChunkCollection() {
uassert( 13346 , "can't pre-split already splitted collection" , (_chunkMap.size() == 1) );
ChunkPtr soleChunk = _chunkMap.begin()->second;
vector<BSONObj> splitPoints;
- soleChunk->pickSplitVector( &splitPoints );
- if ( splitPoints.empty() ){
+ soleChunk->pickSplitVector( splitPoints , Chunk::MaxChunkSize );
+ if ( splitPoints.empty() ) {
log(1) << "not enough data to warrant chunking " << getns() << endl;
return;
}
- soleChunk->multiSplit( splitPoints );
- }
-
- ShardChunkVersion ChunkManager::getVersionOnConfigServer() const {
- static Chunk temp(0);
-
- ScopedDbConnection conn( temp.modelServer() );
-
- auto_ptr<DBClientCursor> cursor = conn->query(temp.getNS(), QUERY("ns" << _ns).sort("lastmod",1), 1 );
- assert( cursor.get() );
- BSONObj o;
- if ( cursor->more() )
- o = cursor->next();
- conn.done();
-
- return o["lastmod"];
+ BSONObj res;
+ ChunkPtr p;
+ p = soleChunk->multiSplit( splitPoints , res );
+ if ( p.get() == NULL ) {
+ log( LL_WARNING ) << "could not split '" << getns() << "': " << res << endl;
+ return;
+ }
}
- ShardChunkVersion ChunkManager::getVersion( const Shard& shard ) const{
- rwlock lk( _lock , false );
+ ShardChunkVersion ChunkManager::getVersion( const Shard& shard ) const {
+ rwlock lk( _lock , false );
// TODO: cache or something?
-
+
ShardChunkVersion max = 0;
- for ( ChunkMap::const_iterator i=_chunkMap.begin(); i!=_chunkMap.end(); ++i ){
+ for ( ChunkMap::const_iterator i=_chunkMap.begin(); i!=_chunkMap.end(); ++i ) {
ChunkPtr c = i->second;
DEV assert( c );
if ( c->getShard() != shard )
continue;
- if ( c->_lastmod > max )
- max = c->_lastmod;
- }
+ if ( c->getLastmod() > max )
+ max = c->getLastmod();
+ }
return max;
}
- ShardChunkVersion ChunkManager::getVersion() const{
- rwlock lk( _lock , false );
- return getVersion_inlock();
- }
-
- ShardChunkVersion ChunkManager::getVersion_inlock() const{
+ ShardChunkVersion ChunkManager::getVersion() const {
+ rwlock lk( _lock , false );
+
ShardChunkVersion max = 0;
-
- for ( ChunkMap::const_iterator i=_chunkMap.begin(); i!=_chunkMap.end(); ++i ){
+
+ for ( ChunkMap::const_iterator i=_chunkMap.begin(); i!=_chunkMap.end(); ++i ) {
ChunkPtr c = i->second;
- if ( c->_lastmod > max )
- max = c->_lastmod;
- }
+ if ( c->getLastmod() > max )
+ max = c->getLastmod();
+ }
return max;
}
string ChunkManager::toString() const {
- rwlock lk( _lock , false );
+ rwlock lk( _lock , false );
stringstream ss;
ss << "ChunkManager: " << _ns << " key:" << _key.toString() << '\n';
- for ( ChunkMap::const_iterator i=_chunkMap.begin(); i!=_chunkMap.end(); ++i ){
+ for ( ChunkMap::const_iterator i=_chunkMap.begin(); i!=_chunkMap.end(); ++i ) {
const ChunkPtr c = i->second;
ss << "\t" << c->toString() << '\n';
}
return ss.str();
}
- void ChunkManager::_migrationNotification(Chunk* c){
- _chunkRanges.reloadRange(_chunkMap, c->getMin(), c->getMax());
- _shards.insert(c->getShard());
- }
-
-
- void ChunkRangeManager::assertValid() const{
+ void ChunkRangeManager::assertValid() const {
if (_ranges.empty())
return;
try {
// No Nulls
- for (ChunkRangeMap::const_iterator it=_ranges.begin(), end=_ranges.end(); it != end; ++it){
+ for (ChunkRangeMap::const_iterator it=_ranges.begin(), end=_ranges.end(); it != end; ++it) {
assert(it->second);
}
-
+
// Check endpoints
assert(allOfType(MinKey, _ranges.begin()->second->getMin()));
assert(allOfType(MaxKey, prior(_ranges.end())->second->getMax()));
// Make sure there are no gaps or overlaps
- for (ChunkRangeMap::const_iterator it=boost::next(_ranges.begin()), end=_ranges.end(); it != end; ++it){
+ for (ChunkRangeMap::const_iterator it=boost::next(_ranges.begin()), end=_ranges.end(); it != end; ++it) {
ChunkRangeMap::const_iterator last = prior(it);
assert(it->second->getMin() == last->second->getMax());
}
// Check Map keys
- for (ChunkRangeMap::const_iterator it=_ranges.begin(), end=_ranges.end(); it != end; ++it){
+ for (ChunkRangeMap::const_iterator it=_ranges.begin(), end=_ranges.end(); it != end; ++it) {
assert(it->first == it->second->getMax());
}
// Make sure we match the original chunks
const ChunkMap chunks = _ranges.begin()->second->getManager()->_chunkMap;
- for ( ChunkMap::const_iterator i=chunks.begin(); i!=chunks.end(); ++i ){
+ for ( ChunkMap::const_iterator i=chunks.begin(); i!=chunks.end(); ++i ) {
const ChunkPtr chunk = i->second;
ChunkRangeMap::const_iterator min = _ranges.upper_bound(chunk->getMin());
@@ -1090,8 +976,9 @@ namespace mongo {
assert(min->second->contains( chunk->getMin() ));
assert(min->second->contains( chunk->getMax() ) || (min->second->getMax() == chunk->getMax()));
}
-
- } catch (...) {
+
+ }
+ catch (...) {
log( LL_ERROR ) << "\t invalid ChunkRangeMap! printing ranges:" << endl;
for (ChunkRangeMap::const_iterator it=_ranges.begin(), end=_ranges.end(); it != end; ++it)
@@ -1101,15 +988,15 @@ namespace mongo {
}
}
- void ChunkRangeManager::reloadRange(const ChunkMap& chunks, const BSONObj& min, const BSONObj& max){
- if (_ranges.empty()){
+ void ChunkRangeManager::reloadRange(const ChunkMap& chunks, const BSONObj& min, const BSONObj& max) {
+ if (_ranges.empty()) {
reloadAll(chunks);
return;
}
-
+
ChunkRangeMap::iterator low = _ranges.upper_bound(min);
ChunkRangeMap::iterator high = _ranges.lower_bound(max);
-
+
assert(low != _ranges.end());
assert(high != _ranges.end());
assert(low->second);
@@ -1135,10 +1022,10 @@ namespace mongo {
// merge low-end if possible
low = _ranges.upper_bound(min);
assert(low != _ranges.end());
- if (low != _ranges.begin()){
+ if (low != _ranges.begin()) {
shared_ptr<ChunkRange> a = prior(low)->second;
shared_ptr<ChunkRange> b = low->second;
- if (a->getShard() == b->getShard()){
+ if (a->getShard() == b->getShard()) {
shared_ptr<ChunkRange> cr (new ChunkRange(*a, *b));
_ranges.erase(prior(low));
_ranges.erase(low); // invalidates low
@@ -1150,10 +1037,10 @@ namespace mongo {
// merge high-end if possible
high = _ranges.lower_bound(max);
- if (high != prior(_ranges.end())){
+ if (high != prior(_ranges.end())) {
shared_ptr<ChunkRange> a = high->second;
shared_ptr<ChunkRange> b = boost::next(high)->second;
- if (a->getShard() == b->getShard()){
+ if (a->getShard() == b->getShard()) {
shared_ptr<ChunkRange> cr (new ChunkRange(*a, *b));
_ranges.erase(boost::next(high));
_ranges.erase(high); //invalidates high
@@ -1164,15 +1051,15 @@ namespace mongo {
DEV assertValid();
}
- void ChunkRangeManager::reloadAll(const ChunkMap& chunks){
+ void ChunkRangeManager::reloadAll(const ChunkMap& chunks) {
_ranges.clear();
_insertRange(chunks.begin(), chunks.end());
DEV assertValid();
}
- void ChunkRangeManager::_insertRange(ChunkMap::const_iterator begin, const ChunkMap::const_iterator end){
- while (begin != end){
+ void ChunkRangeManager::_insertRange(ChunkMap::const_iterator begin, const ChunkMap::const_iterator end) {
+ while (begin != end) {
ChunkMap::const_iterator first = begin;
Shard shard = first->second->getShard();
while (begin != end && (begin->second->getShard() == shard))
@@ -1182,32 +1069,50 @@ namespace mongo {
_ranges[cr->getMax()] = cr;
}
}
-
+
+ int ChunkManager::getCurrentDesiredChunkSize() const {
+ // split faster in early chunks helps spread out an initial load better
+ const int minChunkSize = 1 << 20; // 1 MBytes
+
+ int splitThreshold = Chunk::MaxChunkSize;
+
+ int nc = numChunks();
+
+ if ( nc < 10 ) {
+ splitThreshold = max( splitThreshold / 4 , minChunkSize );
+ }
+ else if ( nc < 20 ) {
+ splitThreshold = max( splitThreshold / 2 , minChunkSize );
+ }
+
+ return splitThreshold;
+ }
+
class ChunkObjUnitTest : public UnitTest {
public:
- void runShard(){
+ void runShard() {
ChunkPtr c;
assert( ! c );
c.reset( new Chunk( 0 ) );
assert( c );
}
-
- void runShardChunkVersion(){
+
+ void runShardChunkVersion() {
vector<ShardChunkVersion> all;
all.push_back( ShardChunkVersion(1,1) );
all.push_back( ShardChunkVersion(1,2) );
all.push_back( ShardChunkVersion(2,1) );
all.push_back( ShardChunkVersion(2,2) );
-
- for ( unsigned i=0; i<all.size(); i++ ){
- for ( unsigned j=i+1; j<all.size(); j++ ){
+
+ for ( unsigned i=0; i<all.size(); i++ ) {
+ for ( unsigned j=i+1; j<all.size(); j++ ) {
assert( all[i] < all[j] );
}
}
}
- void run(){
+ void run() {
runShard();
runShardChunkVersion();
log(1) << "shardObjTest passed" << endl;
@@ -1217,7 +1122,11 @@ namespace mongo {
// ----- to be removed ---
extern OID serverID;
- bool setShardVersion( DBClientBase & conn , const string& ns , ShardChunkVersion version , bool authoritative , BSONObj& result ){
+
+ // NOTE (careful when deprecating)
+ // currently the sharding is enabled because of a write or read (as opposed to a split or migrate), the shard learns
+ // its name and through the 'setShardVersion' command call
+ bool setShardVersion( DBClientBase & conn , const string& ns , ShardChunkVersion version , bool authoritative , BSONObj& result ) {
BSONObjBuilder cmdBuilder;
cmdBuilder.append( "setShardVersion" , ns.c_str() );
cmdBuilder.append( "configdb" , configServer.modelServer() );
@@ -1230,9 +1139,9 @@ namespace mongo {
cmdBuilder.append( "shard" , s.getName() );
cmdBuilder.append( "shardHost" , s.getConnString() );
BSONObj cmd = cmdBuilder.obj();
-
+
log(1) << " setShardVersion " << s.getName() << " " << conn.getServerAddress() << " " << ns << " " << cmd << " " << &conn << endl;
-
+
return conn.runCommand( "admin" , cmd , result );
}
diff --git a/s/chunk.h b/s/chunk.h
index 82f2300..43c0d95 100644
--- a/s/chunk.h
+++ b/s/chunk.h
@@ -1,9 +1,4 @@
-// shard.h
-
-/*
- A "shard" is a database (replica pair typically) which represents
- one partition of the overall database.
-*/
+// @file chunk.h
/**
* Copyright (C) 2008 10gen Inc.
@@ -24,16 +19,17 @@
#pragma once
#include "../pch.h"
-#include "../client/dbclient.h"
-#include "../client/model.h"
+
#include "../bson/util/atomic_int.h"
+#include "../client/dbclient.h"
+#include "../client/distlock.h"
+
#include "shardkey.h"
#include "shard.h"
-#include "config.h"
#include "util.h"
namespace mongo {
-
+
class DBConfig;
class Chunk;
class ChunkRange;
@@ -46,130 +42,187 @@ namespace mongo {
// key is max for each Chunk or ChunkRange
typedef map<BSONObj,ChunkPtr,BSONObjCmp> ChunkMap;
typedef map<BSONObj,shared_ptr<ChunkRange>,BSONObjCmp> ChunkRangeMap;
-
+
+ typedef shared_ptr<ChunkManager> ChunkManagerPtr;
+
/**
config.chunks
{ ns : "alleyinsider.fs.chunks" , min : {} , max : {} , server : "localhost:30001" }
-
+
x is in a shard iff
min <= x < max
- */
+ */
class Chunk : boost::noncopyable, public boost::enable_shared_from_this<Chunk> {
public:
-
Chunk( ChunkManager * info );
Chunk( ChunkManager * info , const BSONObj& min, const BSONObj& max, const Shard& shard);
-
- const BSONObj& getMin() const { return _min; }
- const BSONObj& getMax() const { return _max; }
-
- void setMin(const BSONObj& o){
- _min = o;
- }
- void setMax(const BSONObj& o){
- _max = o;
- }
-
- string getns() const;
- Shard getShard() const { return _shard; }
+ //
+ // serialization support
+ //
- void setShard( const Shard& shard );
-
- bool contains( const BSONObj& obj ) const;
+ void serialize(BSONObjBuilder& to, ShardChunkVersion myLastMod=0);
+ void unserialize(const BSONObj& from);
- string toString() const;
+ //
+ // chunk boundary support
+ //
- friend ostream& operator << (ostream& out, const Chunk& c){ return (out << c.toString()); }
+ const BSONObj& getMin() const { return _min; }
+ const BSONObj& getMax() const { return _max; }
+ void setMin(const BSONObj& o) { _min = o; }
+ void setMax(const BSONObj& o) { _max = o; }
- bool operator==(const Chunk& s) const;
-
- bool operator!=(const Chunk& s) const{
- return ! ( *this == s );
- }
-
// if min/max key is pos/neg infinity
bool minIsInf() const;
bool maxIsInf() const;
- BSONObj pickSplitPoint() const;
- ChunkPtr split();
+ bool contains( const BSONObj& obj ) const;
- void pickSplitVector( vector<BSONObj>* splitPoints ) const;
- ChunkPtr multiSplit( const vector<BSONObj>& splitPoints );
+ string genID() const;
+ static string genID( const string& ns , const BSONObj& min );
+
+ //
+ // chunk version support
+ //
+
+ void appendShortVersion( const char * name , BSONObjBuilder& b );
+
+ ShardChunkVersion getLastmod() const { return _lastmod; }
+ void setLastmod( ShardChunkVersion v ) { _lastmod = v; }
+
+ //
+ // split support
+ //
- /**
- * @return size of shard in bytes
- * talks to mongod to do this
- */
- long getPhysicalSize() const;
-
- int countObjects(int maxcount=0) const;
-
/**
* if the amount of data written nears the max size of a shard
* then we check the real size, and if its too big, we split
+ * @return if something was split
*/
bool splitIfShould( long dataWritten );
-
- /*
+
+ /**
+ * Splits this chunk at a non-specificed split key to be chosen by the mongod holding this chunk.
+ *
+ * @param force if set to true, will split the chunk regardless if the split is really necessary size wise
+ * if set to false, will only split if the chunk has reached the currently desired maximum size
+ * @param res the object containing details about the split execution
+ * @return if found a key, return a pointer to the first chunk, otherwise return a null pointer
+ */
+ ChunkPtr singleSplit( bool force , BSONObj& res );
+
+ /**
+ * Splits this chunk at the given key (or keys)
+ *
+ * @param splitPoints the vector of keys that should be used to divide this chunk
+ * @param res the object containing details about the split execution
+ * @return shared pointer to the first new Chunk or null pointer if failed
+ */
+ ChunkPtr multiSplit( const vector<BSONObj>& splitPoints , BSONObj& res );
+
+ /**
+ * Asks the mongod holding this chunk to find a key that approximately divides this chunk in two
+ *
+ * @param medianKey the key that divides this chunk, if there is one, or empty
+ */
+ void pickMedianKey( BSONObj& medianKey ) const;
+
+ /**
+ * @param splitPoints vector to be filled in
+ * @param chunkSize chunk size to target in bytes
+ * @param maxPoints limits the number of split points that are needed, zero is max (optional)
+ * @param maxObjs limits the number of objects in each chunk, zero is as max (optional)
+ */
+ void pickSplitVector( vector<BSONObj>& splitPoints , int chunkSize , int maxPoints = 0, int maxObjs = 0) const;
+
+ //
+ // migration support
+ //
+
+ /**
* moves either this shard or newShard if it makes sense too
+ *
* @return whether or not a shard was moved
*/
bool moveIfShould( ChunkPtr newShard = ChunkPtr() );
- bool moveAndCommit( const Shard& to , string& errmsg );
+ /**
+ * Issues a migrate request for this chunk
+ *
+ * @param to shard to move this chunk to
+ * @param chunSize maximum number of bytes beyond which the migrate should no go trhough
+ * @param res the object containing details about the migrate execution
+ * @return true if move was successful
+ */
+ bool moveAndCommit( const Shard& to , long long chunkSize , BSONObj& res );
- const char * getNS(){ return "config.chunks"; }
- void serialize(BSONObjBuilder& to, ShardChunkVersion myLastMod=0);
- void unserialize(const BSONObj& from);
- string modelServer() const;
-
- void appendShortVersion( const char * name , BSONObjBuilder& b );
+ /**
+ * @return size of shard in bytes
+ * talks to mongod to do this
+ */
+ long getPhysicalSize() const;
+
+ //
+ // chunk size support
+ int countObjects(int maxcount=0) const;
+
+ //
+ // public constants
+ //
+
+ static string chunkMetadataNS;
static int MaxChunkSize;
- string genID() const;
- static string genID( const string& ns , const BSONObj& min );
+ //
+ // accessors and helpers
+ //
- const ChunkManager* getManager() const { return _manager; }
-
- bool getModified() { return _modified; }
- void setModified( bool modified ) { _modified = modified; }
+ string toString() const;
- ShardChunkVersion getVersionOnConfigServer() const;
- private:
+ friend ostream& operator << (ostream& out, const Chunk& c) { return (out << c.toString()); }
+ bool operator==(const Chunk& s) const;
+ bool operator!=(const Chunk& s) const { return ! ( *this == s ); }
- bool _splitIfShould( long dataWritten );
+ string getns() const;
+ const char * getNS() { return "config.chunks"; }
+ Shard getShard() const { return _shard; }
+ const ChunkManager* getManager() const { return _manager; }
+ private:
// main shard info
-
+
ChunkManager * _manager;
- ShardKeyPattern skey() const;
BSONObj _min;
BSONObj _max;
Shard _shard;
ShardChunkVersion _lastmod;
- bool _modified;
-
// transient stuff
long _dataWritten;
-
+
// methods, etc..
-
- void _split( BSONObj& middle );
- friend class ChunkManager;
- friend class ShardObjUnitTest;
+ /**
+ * if sort 1, return lowest key
+ * if sort -1, return highest key
+ * will return empty object if have none
+ */
+ BSONObj _getExtremeKey( int sort ) const;
+
+ /** initializes _dataWritten with a random value so that a mongos restart wouldn't cause delay in splitting */
+ void _setDataWritten();
+
+ ShardKeyPattern skey() const;
};
- class ChunkRange{
+ class ChunkRange {
public:
- const ChunkManager* getManager() const{ return _manager; }
- Shard getShard() const{ return _shard; }
+ const ChunkManager* getManager() const { return _manager; }
+ Shard getShard() const { return _shard; }
const BSONObj& getMin() const { return _min; }
const BSONObj& getMax() const { return _max; }
@@ -181,11 +234,10 @@ namespace mongo {
: _manager(begin->second->getManager())
, _shard(begin->second->getShard())
, _min(begin->second->getMin())
- , _max(prior(end)->second->getMax())
- {
+ , _max(prior(end)->second->getMax()) {
assert( begin != end );
- DEV while (begin != end){
+ DEV while (begin != end) {
assert(begin->second->getManager() == _manager);
assert(begin->second->getShard() == _shard);
++begin;
@@ -197,14 +249,13 @@ namespace mongo {
: _manager(min.getManager())
, _shard(min.getShard())
, _min(min.getMin())
- , _max(max.getMax())
- {
+ , _max(max.getMax()) {
assert(min.getShard() == max.getShard());
assert(min.getManager() == max.getManager());
assert(min.getMax() == max.getMin());
}
- friend ostream& operator<<(ostream& out, const ChunkRange& cr){
+ friend ostream& operator<<(ostream& out, const ChunkRange& cr) {
return (out << "ChunkRange(min=" << cr._min << ", max=" << cr._max << ", shard=" << cr._shard <<")");
}
@@ -239,7 +290,7 @@ namespace mongo {
};
/* config.sharding
- { ns: 'alleyinsider.fs.chunks' ,
+ { ns: 'alleyinsider.fs.chunks' ,
key: { ts : 1 } ,
shards: [ { min: 1, max: 100, server: a } , { min: 101, max: 200 , server : b } ]
}
@@ -247,75 +298,61 @@ namespace mongo {
class ChunkManager {
public:
- ChunkManager( DBConfig * config , string ns , ShardKeyPattern pattern , bool unique );
+ ChunkManager( string ns , ShardKeyPattern pattern , bool unique );
virtual ~ChunkManager();
string getns() const { return _ns; }
-
+
int numChunks() const { rwlock lk( _lock , false ); return _chunkMap.size(); }
bool hasShardKey( const BSONObj& obj );
+ void createFirstChunk( const Shard& shard );
ChunkPtr findChunk( const BSONObj& obj , bool retry = false );
ChunkPtr findChunkOnServer( const Shard& shard ) const;
-
- ShardKeyPattern& getShardKey(){ return _key; }
+
const ShardKeyPattern& getShardKey() const { return _key; }
- bool isUnique(){ return _unique; }
+ bool isUnique() const { return _unique; }
void maybeChunkCollection();
-
+
void getShardsForQuery( set<Shard>& shards , const BSONObj& query );
void getAllShards( set<Shard>& all );
void getShardsForRange(set<Shard>& shards, const BSONObj& min, const BSONObj& max); // [min, max)
- void save( bool major );
-
string toString() const;
ShardChunkVersion getVersion( const Shard& shard ) const;
ShardChunkVersion getVersion() const;
- /**
- * actually does a query on the server
- * doesn't look at any local data
- */
- ShardChunkVersion getVersionOnConfigServer() const;
-
/**
* this is just an increasing number of how many ChunkManagers we have so we know if something has been updated
*/
- unsigned long long getSequenceNumber(){
- return _sequenceNumber;
- }
-
- void getInfo( BSONObjBuilder& b ){
+ unsigned long long getSequenceNumber() const { return _sequenceNumber; }
+
+ void getInfo( BSONObjBuilder& b ) {
b.append( "key" , _key.key() );
b.appendBool( "unique" , _unique );
}
-
+
/**
* @param me - so i don't get deleted before i'm done
*/
void drop( ChunkManagerPtr me );
void _printChunks() const;
-
+
+ int getCurrentDesiredChunkSize() const;
+
private:
-
void _reload();
void _reload_inlock();
void _load();
- void save_inlock( bool major );
- ShardChunkVersion getVersion_inlock() const;
void ensureIndex_inlock();
-
- DBConfig * _config;
+
string _ns;
ShardKeyPattern _key;
bool _unique;
-
- map<string,unsigned long long> _maxMarkers;
ChunkMap _chunkMap;
ChunkRangeManager _chunkRanges;
@@ -323,11 +360,9 @@ namespace mongo {
set<Shard> _shards;
unsigned long long _sequenceNumber;
-
- mutable RWLock _lock;
- // This should only be called from Chunk after it has been migrated
- void _migrationNotification(Chunk* c);
+ mutable RWLock _lock;
+ DistributedLock _nsLock;
friend class Chunk;
friend class ChunkRangeManager; // only needed for CRM::assertValid()
@@ -362,12 +397,14 @@ namespace mongo {
/*
struct chunk_lock {
chunk_lock( const Chunk* c ){
-
+
}
-
+
Chunk _c;
};
*/
inline string Chunk::genID() const { return genID(_manager->getns(), _min); }
+ bool setShardVersion( DBClientBase & conn , const string& ns , ShardChunkVersion version , bool authoritative , BSONObj& result );
+
} // namespace mongo
diff --git a/s/client.cpp b/s/client.cpp
new file mode 100644
index 0000000..b8559b6
--- /dev/null
+++ b/s/client.cpp
@@ -0,0 +1,292 @@
+// s/client.cpp
+
+/**
+ * Copyright (C) 2008 10gen Inc.
+ *
+ * This program is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License, version 3,
+ * as published by the Free Software Foundation.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU Affero General Public License for more details.
+ *
+ * You should have received a copy of the GNU Affero General Public License
+ * along with this program. If not, see <http://www.gnu.org/licenses/>.
+ */
+
+#include "pch.h"
+#include "server.h"
+
+#include "../db/commands.h"
+#include "../db/dbmessage.h"
+#include "../db/stats/counters.h"
+
+#include "../client/connpool.h"
+
+#include "client.h"
+#include "request.h"
+#include "config.h"
+#include "chunk.h"
+#include "stats.h"
+#include "cursors.h"
+#include "grid.h"
+#include "s/writeback_listener.h"
+
+namespace mongo {
+
+ ClientInfo::ClientInfo( int clientId ) : _id( clientId ) {
+ _cur = &_a;
+ _prev = &_b;
+ _autoSplitOk = true;
+ newRequest();
+ }
+
+ ClientInfo::~ClientInfo() {
+ if ( _lastAccess ) {
+ scoped_lock lk( _clientsLock );
+ Cache::iterator i = _clients.find( _id );
+ if ( i != _clients.end() ) {
+ _clients.erase( i );
+ }
+ }
+ }
+
+ void ClientInfo::addShard( const string& shard ) {
+ _cur->insert( shard );
+ _sinceLastGetError.insert( shard );
+ }
+
+ void ClientInfo::newRequest( AbstractMessagingPort* p ) {
+
+ if ( p ) {
+ HostAndPort r = p->remote();
+ if ( _remote.port() == -1 )
+ _remote = r;
+ else if ( _remote != r ) {
+ stringstream ss;
+ ss << "remotes don't match old [" << _remote.toString() << "] new [" << r.toString() << "]";
+ throw UserException( 13134 , ss.str() );
+ }
+ }
+
+ _lastAccess = (int) time(0);
+
+ set<string> * temp = _cur;
+ _cur = _prev;
+ _prev = temp;
+ _cur->clear();
+ }
+
+ void ClientInfo::disconnect() {
+ _lastAccess = 0;
+ }
+
+ ClientInfo * ClientInfo::get( int clientId , bool create ) {
+
+ if ( ! clientId )
+ clientId = getClientId();
+
+ if ( ! clientId ) {
+ ClientInfo * info = _tlInfo.get();
+ if ( ! info ) {
+ info = new ClientInfo( 0 );
+ _tlInfo.reset( info );
+ }
+ info->newRequest();
+ return info;
+ }
+
+ scoped_lock lk( _clientsLock );
+ Cache::iterator i = _clients.find( clientId );
+ if ( i != _clients.end() )
+ return i->second;
+ if ( ! create )
+ return 0;
+ ClientInfo * info = new ClientInfo( clientId );
+ _clients[clientId] = info;
+ return info;
+ }
+
+ void ClientInfo::disconnect( int clientId ) {
+ if ( ! clientId )
+ return;
+
+ scoped_lock lk( _clientsLock );
+ Cache::iterator i = _clients.find( clientId );
+ if ( i == _clients.end() )
+ return;
+
+ ClientInfo* ci = i->second;
+ ci->disconnect();
+ delete ci;
+ _clients.erase( i );
+ }
+
+ void ClientInfo::_addWriteBack( vector<WBInfo>& all , const BSONObj& gle ) {
+ BSONElement w = gle["writeback"];
+
+ if ( w.type() != jstOID )
+ return;
+
+ BSONElement cid = gle["connectionId"];
+
+ if ( cid.eoo() ) {
+ error() << "getLastError writeback can't work because of version mis-match" << endl;
+ return;
+ }
+
+ all.push_back( WBInfo( cid.numberLong() , w.OID() ) );
+ }
+
+ vector<BSONObj> ClientInfo::_handleWriteBacks( vector<WBInfo>& all , bool fromWriteBackListener ) {
+ vector<BSONObj> res;
+
+ if ( fromWriteBackListener ) {
+ LOG(1) << "not doing recusrive writeback" << endl;
+ return res;
+ }
+
+ if ( all.size() == 0 )
+ return res;
+
+ for ( unsigned i=0; i<all.size(); i++ ) {
+ res.push_back( WriteBackListener::waitFor( all[i].connectionId , all[i].id ) );
+ }
+
+ return res;
+ }
+
+
+
+ bool ClientInfo::getLastError( const BSONObj& options , BSONObjBuilder& result , bool fromWriteBackListener ) {
+ set<string> * shards = getPrev();
+
+ if ( shards->size() == 0 ) {
+ result.appendNull( "err" );
+ return true;
+ }
+
+ vector<WBInfo> writebacks;
+
+ // handle single server
+ if ( shards->size() == 1 ) {
+ string theShard = *(shards->begin() );
+
+ ShardConnection conn( theShard , "" );
+
+ BSONObj res;
+ bool ok = conn->runCommand( "admin" , options , res );
+ res = res.getOwned();
+ conn.done();
+
+
+ _addWriteBack( writebacks , res );
+
+ // hit other machines just to block
+ for ( set<string>::const_iterator i=sinceLastGetError().begin(); i!=sinceLastGetError().end(); ++i ) {
+ string temp = *i;
+ if ( temp == theShard )
+ continue;
+
+ ShardConnection conn( temp , "" );
+ _addWriteBack( writebacks , conn->getLastErrorDetailed() );
+ conn.done();
+ }
+ clearSinceLastGetError();
+
+ if ( writebacks.size() ){
+ vector<BSONObj> v = _handleWriteBacks( writebacks , fromWriteBackListener );
+ if ( v.size() == 0 && fromWriteBackListener ) {
+ // ok
+ }
+ else {
+ assert( v.size() == 1 );
+ result.appendElements( v[0] );
+ result.appendElementsUnique( res );
+ result.append( "initialGLEHost" , theShard );
+ }
+ }
+ else {
+ result.append( "singleShard" , theShard );
+ result.appendElements( res );
+ }
+
+ return ok;
+ }
+
+ BSONArrayBuilder bbb( result.subarrayStart( "shards" ) );
+
+ long long n = 0;
+
+ // hit each shard
+ vector<string> errors;
+ vector<BSONObj> errorObjects;
+ for ( set<string>::iterator i = shards->begin(); i != shards->end(); i++ ) {
+ string theShard = *i;
+ bbb.append( theShard );
+ ShardConnection conn( theShard , "" );
+ BSONObj res;
+ bool ok = conn->runCommand( "admin" , options , res );
+ _addWriteBack( writebacks, res );
+
+ string temp = DBClientWithCommands::getLastErrorString( res );
+ if ( conn->type() != ConnectionString::SYNC && ( ok == false || temp.size() ) ) {
+ errors.push_back( temp );
+ errorObjects.push_back( res );
+ }
+ n += res["n"].numberLong();
+ conn.done();
+ }
+
+ bbb.done();
+
+ result.appendNumber( "n" , n );
+
+ // hit other machines just to block
+ for ( set<string>::const_iterator i=sinceLastGetError().begin(); i!=sinceLastGetError().end(); ++i ) {
+ string temp = *i;
+ if ( shards->count( temp ) )
+ continue;
+
+ ShardConnection conn( temp , "" );
+ _addWriteBack( writebacks, conn->getLastErrorDetailed() );
+ conn.done();
+ }
+ clearSinceLastGetError();
+
+ if ( errors.size() == 0 ) {
+ result.appendNull( "err" );
+ _handleWriteBacks( writebacks , fromWriteBackListener );
+ return true;
+ }
+
+ result.append( "err" , errors[0].c_str() );
+
+ {
+ // errs
+ BSONArrayBuilder all( result.subarrayStart( "errs" ) );
+ for ( unsigned i=0; i<errors.size(); i++ ) {
+ all.append( errors[i].c_str() );
+ }
+ all.done();
+ }
+
+ {
+ // errObjects
+ BSONArrayBuilder all( result.subarrayStart( "errObjects" ) );
+ for ( unsigned i=0; i<errorObjects.size(); i++ ) {
+ all.append( errorObjects[i] );
+ }
+ all.done();
+ }
+ _handleWriteBacks( writebacks , fromWriteBackListener );
+ return true;
+ }
+
+ ClientInfo::Cache& ClientInfo::_clients = *(new ClientInfo::Cache());
+ mongo::mutex ClientInfo::_clientsLock("_clientsLock");
+ boost::thread_specific_ptr<ClientInfo> ClientInfo::_tlInfo;
+
+} // namespace mongo
diff --git a/s/client.h b/s/client.h
new file mode 100644
index 0000000..bd4295f
--- /dev/null
+++ b/s/client.h
@@ -0,0 +1,120 @@
+// client.h
+
+/*
+ * Copyright (C) 2010 10gen Inc.
+ *
+ * This program is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License, version 3,
+ * as published by the Free Software Foundation.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU Affero General Public License for more details.
+ *
+ * You should have received a copy of the GNU Affero General Public License
+ * along with this program. If not, see <http://www.gnu.org/licenses/>.
+ */
+
+#include "../pch.h"
+
+namespace mongo {
+
+ /**
+ * holds information about a client connected to a mongos
+ * 1 per client socket
+ * currently implemented with a thread local
+ */
+ class ClientInfo {
+
+ typedef map<int,ClientInfo*> Cache;
+
+ public:
+ ClientInfo( int clientId );
+ ~ClientInfo();
+
+ /** new request from client, adjusts internal state */
+ void newRequest( AbstractMessagingPort* p = 0 );
+
+ /** client disconnected */
+ void disconnect();
+
+ /**
+ * @return remote socket address of the client
+ */
+ HostAndPort getRemote() const { return _remote; }
+
+ /**
+ * notes that this client use this shard
+ * keeps track of all shards accessed this request
+ */
+ void addShard( const string& shard );
+
+ /**
+ * gets shards used on the previous request
+ */
+ set<string> * getPrev() const { return _prev; };
+
+ /**
+ * gets all shards we've accessed since the last time we called clearSinceLastGetError
+ */
+ const set<string>& sinceLastGetError() const { return _sinceLastGetError; }
+
+ /**
+ * clears list of shards we've talked to
+ */
+ void clearSinceLastGetError() { _sinceLastGetError.clear(); }
+
+ /**
+ * calls getLastError
+ * resets shards since get last error
+ * @return if the command was ok or if there was an error
+ */
+ bool getLastError( const BSONObj& options , BSONObjBuilder& result , bool fromWriteBackListener = false );
+
+ /** @return if its ok to auto split from this client */
+ bool autoSplitOk() const { return _autoSplitOk; }
+
+ void noAutoSplit() { _autoSplitOk = false; }
+
+ static ClientInfo * get( int clientId = 0 , bool create = true );
+ static void disconnect( int clientId );
+
+ private:
+
+ struct WBInfo {
+ WBInfo( ConnectionId c , OID o ) : connectionId( c ) , id( o ) {}
+ ConnectionId connectionId;
+ OID id;
+ };
+
+ // for getLastError
+ void _addWriteBack( vector<WBInfo>& all , const BSONObj& o );
+ vector<BSONObj> _handleWriteBacks( vector<WBInfo>& all , bool fromWriteBackListener );
+
+
+ int _id; // unique client id
+ HostAndPort _remote; // server:port of remote socket end
+
+ // we use _a and _b to store shards we've talked to on the current request and the previous
+ // we use 2 so we can flip for getLastError type operations
+
+ set<string> _a; // actual set for _cur or _prev
+ set<string> _b; // "
+
+ set<string> * _cur; // pointer to _a or _b depending on state
+ set<string> * _prev; // ""
+
+
+ set<string> _sinceLastGetError; // all shards accessed since last getLastError
+
+ int _lastAccess;
+ bool _autoSplitOk;
+
+ static mongo::mutex _clientsLock;
+ static Cache& _clients;
+ static boost::thread_specific_ptr<ClientInfo> _tlInfo;
+ };
+
+
+}
diff --git a/s/commands_admin.cpp b/s/commands_admin.cpp
index 551b8a9..532161a 100644
--- a/s/commands_admin.cpp
+++ b/s/commands_admin.cpp
@@ -29,6 +29,7 @@
#include "../util/message.h"
#include "../util/processinfo.h"
#include "../util/stringutils.h"
+#include "../util/version.h"
#include "../client/connpool.h"
@@ -41,6 +42,8 @@
#include "grid.h"
#include "strategy.h"
#include "stats.h"
+#include "writeback_listener.h"
+#include "client.h"
namespace mongo {
@@ -48,7 +51,7 @@ namespace mongo {
class GridAdminCmd : public Command {
public:
- GridAdminCmd( const char * n ) : Command( n , false, tolowerString(n).c_str() ){
+ GridAdminCmd( const char * n ) : Command( n , false, tolowerString(n).c_str() ) {
}
virtual bool slaveOk() const {
return true;
@@ -58,7 +61,7 @@ namespace mongo {
}
// all grid commands are designed not to lock
- virtual LockType locktype() const { return NONE; }
+ virtual LockType locktype() const { return NONE; }
};
// --------------- misc commands ----------------------
@@ -69,31 +72,34 @@ namespace mongo {
virtual void help( stringstream& help ) const {
help << " shows status/reachability of servers in the cluster";
}
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
result.append("configserver", configServer.getPrimary().getConnString() );
result.append("isdbgrid", 1);
return true;
}
} netstat;
-
+
class ServerStatusCmd : public Command {
public:
- ServerStatusCmd() : Command( "serverStatus" , true ){
+ ServerStatusCmd() : Command( "serverStatus" , true ) {
_started = time(0);
}
-
+
virtual bool slaveOk() const { return true; }
- virtual LockType locktype() const { return NONE; }
-
+ virtual LockType locktype() const { return NONE; }
+
bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool fromRepl) {
+ result.append( "host" , prettyHostName() );
+ result.append("version", versionString);
+ result.append("process","mongos");
result.append("uptime",(double) (time(0)-_started));
result.appendDate( "localTime" , jsTime() );
{
BSONObjBuilder t( result.subobjStart( "mem" ) );
-
+
ProcessInfo p;
- if ( p.supported() ){
+ if ( p.supported() ) {
t.appendNumber( "resident" , p.getResidentSize() );
t.appendNumber( "virtual" , p.getVirtualMemorySize() );
t.appendBool( "supported" , true );
@@ -102,7 +108,7 @@ namespace mongo {
result.append( "note" , "not all mem info support on this platform" );
t.appendBool( "supported" , false );
}
-
+
t.done();
}
@@ -112,7 +118,7 @@ namespace mongo {
bb.append( "available" , connTicketHolder.available() );
bb.done();
}
-
+
{
BSONObjBuilder bb( result.subobjStart( "extra_info" ) );
bb.append("note", "fields vary by platform");
@@ -120,7 +126,7 @@ namespace mongo {
p.getExtraInfo(bb);
bb.done();
}
-
+
result.append( "opcounters" , globalOpCounters.getObj() );
{
BSONObjBuilder bb( result.subobjStart( "ops" ) );
@@ -130,7 +136,7 @@ namespace mongo {
}
result.append( "shardCursorType" , shardedCursorTypes.getObj() );
-
+
{
BSONObjBuilder asserts( result.subobjStart( "asserts" ) );
asserts.append( "regular" , assertionCount.regular );
@@ -141,6 +147,13 @@ namespace mongo {
asserts.done();
}
+ {
+ BSONObjBuilder bb( result.subobjStart( "network" ) );
+ networkCounter.append( bb );
+ bb.done();
+ }
+
+
return 1;
}
@@ -149,34 +162,34 @@ namespace mongo {
class FsyncCommand : public GridAdminCmd {
public:
- FsyncCommand() : GridAdminCmd( "fsync" ){}
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
- if ( cmdObj["lock"].trueValue() ){
+ FsyncCommand() : GridAdminCmd( "fsync" ) {}
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
+ if ( cmdObj["lock"].trueValue() ) {
errmsg = "can't do lock through mongos";
return false;
}
-
+
BSONObjBuilder sub;
bool ok = true;
int numFiles = 0;
-
+
vector<Shard> shards;
Shard::getAllShards( shards );
- for ( vector<Shard>::iterator i=shards.begin(); i!=shards.end(); i++ ){
+ for ( vector<Shard>::iterator i=shards.begin(); i!=shards.end(); i++ ) {
Shard s = *i;
BSONObj x = s.runCommand( "admin" , "fsync" );
sub.append( s.getName() , x );
- if ( ! x["ok"].trueValue() ){
+ if ( ! x["ok"].trueValue() ) {
ok = false;
errmsg = x["errmsg"].String();
}
-
+
numFiles += x["numFiles"].numberInt();
}
-
+
result.append( "numFiles" , numFiles );
result.append( "all" , sub.obj() );
return ok;
@@ -192,43 +205,43 @@ namespace mongo {
help << " example: { moveprimary : 'foo' , to : 'localhost:9999' }";
// TODO: locking?
}
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
string dbname = cmdObj.firstElement().valuestrsafe();
- if ( dbname.size() == 0 ){
+ if ( dbname.size() == 0 ) {
errmsg = "no db";
return false;
}
- if ( dbname == "config" ){
+ if ( dbname == "config" ) {
errmsg = "can't move config db";
return false;
}
DBConfigPtr config = grid.getDBConfig( dbname , false );
- if ( ! config ){
+ if ( ! config ) {
errmsg = "can't find db!";
return false;
}
string to = cmdObj["to"].valuestrsafe();
- if ( ! to.size() ){
+ if ( ! to.size() ) {
errmsg = "you have to specify where you want to move it";
return false;
}
Shard s = Shard::make( to );
- if ( config->getPrimary() == s.getConnString() ){
+ if ( config->getPrimary() == s.getConnString() ) {
errmsg = "thats already the primary";
return false;
}
- if ( ! grid.knowAboutShard( s.getConnString() ) ){
+ if ( ! grid.knowAboutShard( s.getConnString() ) ) {
errmsg = "that server isn't known to me";
return false;
}
-
- log() << "movePrimary: moving " << dbname << " primary from: " << config->getPrimary().toString()
+
+ log() << "movePrimary: moving " << dbname << " primary from: " << config->getPrimary().toString()
<< " to: " << s.toString() << endl;
// TODO LOCKING: this is not safe with multiple mongos
@@ -241,7 +254,7 @@ namespace mongo {
bool worked = toconn->runCommand( dbname.c_str() , BSON( "clone" << config->getPrimary().getConnString() ) , cloneRes );
toconn.done();
- if ( ! worked ){
+ if ( ! worked ) {
log() << "clone failed" << cloneRes << endl;
errmsg = "clone failed";
return false;
@@ -264,25 +277,25 @@ namespace mongo {
class EnableShardingCmd : public GridAdminCmd {
public:
- EnableShardingCmd() : GridAdminCmd( "enableSharding" ){}
+ EnableShardingCmd() : GridAdminCmd( "enableSharding" ) {}
virtual void help( stringstream& help ) const {
help
- << "Enable sharding for a db. (Use 'shardcollection' command afterwards.)\n"
- << " { enablesharding : \"<dbname>\" }\n";
+ << "Enable sharding for a db. (Use 'shardcollection' command afterwards.)\n"
+ << " { enablesharding : \"<dbname>\" }\n";
}
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
string dbname = cmdObj.firstElement().valuestrsafe();
- if ( dbname.size() == 0 ){
+ if ( dbname.size() == 0 ) {
errmsg = "no db";
return false;
}
DBConfigPtr config = grid.getDBConfig( dbname );
- if ( config->isShardingEnabled() ){
+ if ( config->isShardingEnabled() ) {
errmsg = "already enabled";
return false;
}
-
+
log() << "enabling sharding on: " << dbname << endl;
config->enableSharding();
@@ -295,46 +308,46 @@ namespace mongo {
class ShardCollectionCmd : public GridAdminCmd {
public:
- ShardCollectionCmd() : GridAdminCmd( "shardCollection" ){}
+ ShardCollectionCmd() : GridAdminCmd( "shardCollection" ) {}
virtual void help( stringstream& help ) const {
help
- << "Shard a collection. Requires key. Optional unique. Sharding must already be enabled for the database.\n"
- << " { enablesharding : \"<dbname>\" }\n";
+ << "Shard a collection. Requires key. Optional unique. Sharding must already be enabled for the database.\n"
+ << " { enablesharding : \"<dbname>\" }\n";
}
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
string ns = cmdObj.firstElement().valuestrsafe();
- if ( ns.size() == 0 ){
+ if ( ns.size() == 0 ) {
errmsg = "no ns";
return false;
}
DBConfigPtr config = grid.getDBConfig( ns );
- if ( ! config->isShardingEnabled() ){
+ if ( ! config->isShardingEnabled() ) {
errmsg = "sharding not enabled for db";
return false;
}
- if ( config->isSharded( ns ) ){
+ if ( config->isSharded( ns ) ) {
errmsg = "already sharded";
return false;
}
BSONObj key = cmdObj.getObjectField( "key" );
- if ( key.isEmpty() ){
+ if ( key.isEmpty() ) {
errmsg = "no shard key";
return false;
}
- BSONForEach(e, key){
- if (!e.isNumber() || e.number() != 1.0){
+ BSONForEach(e, key) {
+ if (!e.isNumber() || e.number() != 1.0) {
errmsg = "shard keys must all be ascending";
return false;
}
}
- if ( ns.find( ".system." ) != string::npos ){
+ if ( ns.find( ".system." ) != string::npos ) {
errmsg = "can't shard system namespaces";
return false;
}
@@ -344,10 +357,10 @@ namespace mongo {
// 1. A unique index must have the sharding key as its prefix. Otherwise maintainig uniqueness would
// require coordinated access to all shards. Trying to shard a collection with such an index is not
// allowed.
- //
+ //
// 2. Sharding a collection requires an index over the sharding key. That index must be create upfront.
// The rationale is that sharding a non-empty collection would need to create the index and that could
- // be slow. Requiring the index upfront allows the admin to plan before sharding and perhaps use
+ // be slow. Requiring the index upfront allows the admin to plan before sharding and perhaps use
// background index construction. One exception to the rule: empty collections. It's fairly easy to
// create the index as part of the sharding process.
//
@@ -358,20 +371,20 @@ namespace mongo {
bool hasShardIndex = false;
ScopedDbConnection conn( config->getPrimary() );
- BSONObjBuilder b;
- b.append( "ns" , ns );
+ BSONObjBuilder b;
+ b.append( "ns" , ns );
auto_ptr<DBClientCursor> cursor = conn->query( config->getName() + ".system.indexes" , b.obj() );
- while ( cursor->more() ){
+ while ( cursor->more() ) {
BSONObj idx = cursor->next();
// Is index key over the sharding key? Remember that.
- if ( key.woCompare( idx["key"].embeddedObjectUserCheck() ) == 0 ){
+ if ( key.woCompare( idx["key"].embeddedObjectUserCheck() ) == 0 ) {
hasShardIndex = true;
}
// Not a unique index? Move on.
- if ( idx["unique"].eoo() || ! idx["unique"].Bool() )
+ if ( idx["unique"].eoo() || ! idx["unique"].trueValue() )
continue;
// Shard key is prefix of unique index? Move on.
@@ -384,17 +397,31 @@ namespace mongo {
}
BSONObj res = conn->findOne( config->getName() + ".system.namespaces" , BSON( "name" << ns ) );
- if ( res["options"].type() == Object && res["options"].embeddedObject()["capped"].trueValue() ){
+ if ( res["options"].type() == Object && res["options"].embeddedObject()["capped"].trueValue() ) {
errmsg = "can't shard capped collection";
conn.done();
return false;
}
- if ( ! hasShardIndex && ( conn->count( ns ) != 0 ) ){
+ if ( hasShardIndex ) {
+ // make sure there are no null entries in the sharding index
+ BSONObjBuilder cmd;
+ cmd.append( "checkShardingIndex" , ns );
+ cmd.append( "keyPattern" , key );
+ BSONObj cmdObj = cmd.obj();
+ if ( ! conn->runCommand( "admin" , cmdObj , res )) {
+ errmsg = res["errmsg"].str();
+ conn.done();
+ return false;
+ }
+ }
+
+ if ( ! hasShardIndex && ( conn->count( ns ) != 0 ) ) {
errmsg = "please create an index over the sharding key before sharding.";
+ conn.done();
return false;
}
-
+
conn.done();
}
@@ -409,26 +436,26 @@ namespace mongo {
class GetShardVersion : public GridAdminCmd {
public:
- GetShardVersion() : GridAdminCmd( "getShardVersion" ){}
+ GetShardVersion() : GridAdminCmd( "getShardVersion" ) {}
virtual void help( stringstream& help ) const {
help << " example: { getShardVersion : 'alleyinsider.foo' } ";
}
-
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
string ns = cmdObj.firstElement().valuestrsafe();
- if ( ns.size() == 0 ){
+ if ( ns.size() == 0 ) {
errmsg = "need to speciy fully namespace";
return false;
}
-
+
DBConfigPtr config = grid.getDBConfig( ns );
- if ( ! config->isSharded( ns ) ){
+ if ( ! config->isSharded( ns ) ) {
errmsg = "ns not sharded.";
return false;
}
-
+
ChunkManagerPtr cm = config->getChunkManager( ns );
- if ( ! cm ){
+ if ( ! cm ) {
errmsg = "no chunk manager?";
return false;
}
@@ -439,144 +466,141 @@ namespace mongo {
}
} getShardVersionCmd;
- class SplitCollectionHelper : public GridAdminCmd {
+ class SplitCollectionCmd : public GridAdminCmd {
public:
- SplitCollectionHelper( const char * name ) : GridAdminCmd( name ) , _name( name ){}
+ SplitCollectionCmd() : GridAdminCmd( "split" ) {}
virtual void help( stringstream& help ) const {
help
- << " example: { split : 'alleyinsider.blog.posts' , find : { ts : 1 } } - split the shard that contains give key \n"
- << " example: { split : 'alleyinsider.blog.posts' , middle : { ts : 1 } } - split the shard that contains the key with this as the middle \n"
- << " NOTE: this does not move move the chunks, it merely creates a logical seperation \n"
- ;
+ << " example: - split the shard that contains give key \n"
+ << " { split : 'alleyinsider.blog.posts' , find : { ts : 1 } }\n"
+ << " example: - split the shard that contains the key with this as the middle \n"
+ << " { split : 'alleyinsider.blog.posts' , middle : { ts : 1 } }\n"
+ << " NOTE: this does not move move the chunks, it merely creates a logical seperation \n"
+ ;
}
- virtual bool _split( BSONObjBuilder& result , string&errmsg , const string& ns , ChunkManagerPtr manager , ChunkPtr old , BSONObj middle ) = 0;
-
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
ShardConnection::sync();
string ns = cmdObj.firstElement().valuestrsafe();
- if ( ns.size() == 0 ){
+ if ( ns.size() == 0 ) {
errmsg = "no ns";
return false;
}
DBConfigPtr config = grid.getDBConfig( ns );
- if ( ! config->isSharded( ns ) ){
+ if ( ! config->isSharded( ns ) ) {
errmsg = "ns not sharded. have to shard before can split";
return false;
}
BSONObj find = cmdObj.getObjectField( "find" );
- if ( find.isEmpty() ){
+ if ( find.isEmpty() ) {
find = cmdObj.getObjectField( "middle" );
- if ( find.isEmpty() ){
+ if ( find.isEmpty() ) {
errmsg = "need to specify find or middle";
return false;
}
}
-
- ChunkManagerPtr info = config->getChunkManager( ns );
- ChunkPtr old = info->findChunk( find );
-
- return _split( result , errmsg , ns , info , old , cmdObj.getObjectField( "middle" ) );
- }
-
- protected:
- string _name;
- };
-
- class SplitValueCommand : public SplitCollectionHelper {
- public:
- SplitValueCommand() : SplitCollectionHelper( "splitValue" ){}
- virtual bool _split( BSONObjBuilder& result , string& errmsg , const string& ns , ChunkManagerPtr manager , ChunkPtr old , BSONObj middle ){
-
- result << "shardinfo" << old->toString();
-
- result.appendBool( "auto" , middle.isEmpty() );
-
- if ( middle.isEmpty() )
- middle = old->pickSplitPoint();
- result.append( "middle" , middle );
-
- return true;
- }
+ ChunkManagerPtr info = config->getChunkManager( ns );
+ ChunkPtr chunk = info->findChunk( find );
+ BSONObj middle = cmdObj.getObjectField( "middle" );
- } splitValueCmd;
+ assert( chunk.get() );
+ log() << "splitting: " << ns << " shard: " << chunk << endl;
+ BSONObj res;
+ ChunkPtr p;
+ if ( middle.isEmpty() ) {
+ p = chunk->singleSplit( true /* force a split even if not enough data */ , res );
- class SplitCollection : public SplitCollectionHelper {
- public:
- SplitCollection() : SplitCollectionHelper( "split" ){}
- virtual bool _split( BSONObjBuilder& result , string& errmsg , const string& ns , ChunkManagerPtr manager , ChunkPtr old , BSONObj middle ){
- assert( old.get() );
- log() << "splitting: " << ns << " shard: " << old << endl;
-
- if ( middle.isEmpty() )
- old->split();
+ }
else {
+ // sanity check if the key provided is a valid split point
+ if ( ( middle == chunk->getMin() ) || ( middle == chunk->getMax() ) ) {
+ errmsg = "cannot split on initial or final chunk's key";
+ return false;
+ }
+
vector<BSONObj> splitPoints;
splitPoints.push_back( middle );
- old->multiSplit( splitPoints );
+ p = chunk->multiSplit( splitPoints , res );
}
+ if ( p.get() == NULL ) {
+ errmsg = "split failed";
+ result.append( "cause" , res );
+ return false;
+ }
+ config->getChunkManager( ns , true );
return true;
}
-
-
} splitCollectionCmd;
class MoveChunkCmd : public GridAdminCmd {
public:
- MoveChunkCmd() : GridAdminCmd( "moveChunk" ){}
+ MoveChunkCmd() : GridAdminCmd( "moveChunk" ) {}
virtual void help( stringstream& help ) const {
help << "{ movechunk : 'test.foo' , find : { num : 1 } , to : 'localhost:30001' }";
}
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
ShardConnection::sync();
Timer t;
string ns = cmdObj.firstElement().valuestrsafe();
- if ( ns.size() == 0 ){
+ if ( ns.size() == 0 ) {
errmsg = "no ns";
return false;
}
DBConfigPtr config = grid.getDBConfig( ns );
- if ( ! config->isSharded( ns ) ){
+ if ( ! config->isSharded( ns ) ) {
errmsg = "ns not sharded. have to shard before can move a chunk";
return false;
}
BSONObj find = cmdObj.getObjectField( "find" );
- if ( find.isEmpty() ){
+ if ( find.isEmpty() ) {
errmsg = "need to specify find. see help";
return false;
}
string toString = cmdObj["to"].valuestrsafe();
- if ( ! toString.size() ){
+ if ( ! toString.size() ) {
errmsg = "you have to specify where you want to move the chunk";
return false;
}
-
+
Shard to = Shard::make( toString );
+ // so far, chunk size serves test purposes; it may or may not become a supported parameter
+ long long maxChunkSizeBytes = cmdObj["maxChunkSizeBytes"].numberLong();
+ if ( maxChunkSizeBytes == 0 ) {
+ maxChunkSizeBytes = Chunk::MaxChunkSize;
+ }
+
tlog() << "CMD: movechunk: " << cmdObj << endl;
ChunkManagerPtr info = config->getChunkManager( ns );
ChunkPtr c = info->findChunk( find );
const Shard& from = c->getShard();
- if ( from == to ){
+ if ( from == to ) {
errmsg = "that chunk is already on that shard";
return false;
}
-
- if ( ! c->moveAndCommit( to , errmsg ) )
+
+ BSONObj res;
+ if ( ! c->moveAndCommit( to , maxChunkSizeBytes , res ) ) {
+ errmsg = "move failed";
+ result.append( "cause" , res );
return false;
+ }
+
+ // pre-emptively reload the config to get new version info
+ config->getChunkManager( ns , true );
result.append( "millis" , t.millis() );
return true;
@@ -591,12 +615,12 @@ namespace mongo {
virtual void help( stringstream& help ) const {
help << "list all shards of the system";
}
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
ScopedDbConnection conn( configServer.getPrimary() );
vector<BSONObj> all;
auto_ptr<DBClientCursor> cursor = conn->query( "config.shards" , BSONObj() );
- while ( cursor->more() ){
+ while ( cursor->more() ) {
BSONObj o = cursor->next();
all.push_back( o );
}
@@ -608,27 +632,27 @@ namespace mongo {
}
} listShardsCmd;
- /* a shard is a single mongod server or a replica pair. add it (them) to the cluster as a storage partition. */
+ /* a shard is a single mongod server or a replica pair. add it (them) to the cluster as a storage partition. */
class AddShard : public GridAdminCmd {
public:
AddShard() : GridAdminCmd("addShard") { }
virtual void help( stringstream& help ) const {
help << "add a new shard to the system";
}
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
errmsg.clear();
// get replica set component hosts
ConnectionString servers = ConnectionString::parse( cmdObj.firstElement().valuestrsafe() , errmsg );
- if ( ! errmsg.empty() ){
+ if ( ! errmsg.empty() ) {
log() << "addshard request " << cmdObj << " failed:" << errmsg << endl;
return false;
}
// using localhost in server names implies every other process must use locahost addresses too
vector<HostAndPort> serverAddrs = servers.getServers();
- for ( size_t i = 0 ; i < serverAddrs.size() ; i++ ){
- if ( serverAddrs[i].isLocalHost() != grid.allowLocalHost() ){
+ for ( size_t i = 0 ; i < serverAddrs.size() ; i++ ) {
+ if ( serverAddrs[i].isLocalHost() != grid.allowLocalHost() ) {
errmsg = "can't use localhost as a shard since all shards need to communicate. "
"either use all shards and configdbs in localhost or all in actual IPs " ;
log() << "addshard request " << cmdObj << " failed: attempt to mix localhosts and IPs" << endl;
@@ -636,7 +660,7 @@ namespace mongo {
}
// it's fine if mongods of a set all use default port
- if ( ! serverAddrs[i].hasPort() ){
+ if ( ! serverAddrs[i].hasPort() ) {
serverAddrs[i].setPort( CmdLine::ShardServerPort );
}
}
@@ -645,15 +669,15 @@ namespace mongo {
string name = "";
if ( cmdObj["name"].type() == String ) {
name = cmdObj["name"].valuestrsafe();
- }
+ }
// maxSize is the space usage cap in a shard in MBs
long long maxSize = 0;
- if ( cmdObj[ ShardFields::maxSize.name() ].isNumber() ){
+ if ( cmdObj[ ShardFields::maxSize.name() ].isNumber() ) {
maxSize = cmdObj[ ShardFields::maxSize.name() ].numberLong();
}
-
- if ( ! grid.addShard( &name , servers , maxSize , errmsg ) ){
+
+ if ( ! grid.addShard( &name , servers , maxSize , errmsg ) ) {
log() << "addshard request " << cmdObj << " failed: " << errmsg << endl;
return false;
}
@@ -673,10 +697,10 @@ namespace mongo {
virtual void help( stringstream& help ) const {
help << "remove a shard to the system.";
}
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
string target = cmdObj.firstElement().valuestrsafe();
Shard s = Shard::make( target );
- if ( ! grid.knowAboutShard( s.getConnString() ) ){
+ if ( ! grid.knowAboutShard( s.getConnString() ) ) {
errmsg = "unknown shard";
return false;
}
@@ -687,7 +711,7 @@ namespace mongo {
BSONObj searchDoc = BSON( "_id" << s.getName() );
BSONObj drainingDoc = BSON( "_id" << s.getName() << ShardFields::draining(true) );
BSONObj shardDoc = conn->findOne( "config.shards", drainingDoc );
- if ( shardDoc.isEmpty() ){
+ if ( shardDoc.isEmpty() ) {
// TODO prevent move chunks to this shard.
@@ -696,7 +720,7 @@ namespace mongo {
conn->update( "config.shards" , searchDoc , newStatus, false /* do no upsert */);
errmsg = conn->getLastError();
- if ( errmsg.size() ){
+ if ( errmsg.size() ) {
log() << "error starting remove shard: " << s.getName() << " err: " << errmsg << endl;
return false;
}
@@ -704,7 +728,7 @@ namespace mongo {
Shard::reloadShardInfo();
result.append( "msg" , "draining started successfully" );
- result.append( "state" , "started" );
+ result.append( "state" , "started" );
result.append( "shard" , s.getName() );
conn.done();
return true;
@@ -716,12 +740,12 @@ namespace mongo {
long long chunkCount = conn->count( "config.chunks" , shardIDDoc );
BSONObj primaryDoc = BSON( "primary" << shardDoc[ "_id" ].str() );
long long dbCount = conn->count( "config.databases" , primaryDoc );
- if ( ( chunkCount == 0 ) && ( dbCount == 0 ) ){
- log() << "going to remove shard: " << s.getName() << endl;
+ if ( ( chunkCount == 0 ) && ( dbCount == 0 ) ) {
+ log() << "going to remove shard: " << s.getName() << endl;
conn->remove( "config.shards" , searchDoc );
errmsg = conn->getLastError();
- if ( errmsg.size() ){
+ if ( errmsg.size() ) {
log() << "error concluding remove shard: " << s.getName() << " err: " << errmsg << endl;
return false;
}
@@ -755,7 +779,7 @@ namespace mongo {
class IsDbGridCmd : public Command {
public:
- virtual LockType locktype() const { return NONE; }
+ virtual LockType locktype() const { return NONE; }
virtual bool slaveOk() const {
return true;
}
@@ -769,7 +793,7 @@ namespace mongo {
class CmdIsMaster : public Command {
public:
- virtual LockType locktype() const { return NONE; }
+ virtual LockType locktype() const { return NONE; }
virtual bool requiresAuth() { return false; }
virtual bool slaveOk() const {
return true;
@@ -777,10 +801,11 @@ namespace mongo {
virtual void help( stringstream& help ) const {
help << "test if this is master half of a replica pair";
}
- CmdIsMaster() : Command("ismaster") { }
+ CmdIsMaster() : Command("isMaster" , false , "ismaster") { }
virtual bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
- result.append("ismaster", 1.0 );
+ result.appendBool("ismaster", true );
result.append("msg", "isdbgrid");
+ result.appendNumber("maxBsonObjectSize", BSONObjMaxUserSize);
return true;
}
} ismaster;
@@ -794,23 +819,23 @@ namespace mongo {
virtual bool slaveOk() const {
return true;
}
- virtual LockType locktype() const { return NONE; }
+ virtual LockType locktype() const { return NONE; }
virtual bool requiresAuth() {
return false;
}
virtual void help( stringstream &help ) const {
help << "{whatsmyuri:1}";
- }
+ }
virtual bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
result << "you" << ClientInfo::get()->getRemote();
return true;
}
} cmdWhatsMyUri;
-
+
class CmdShardingGetPrevError : public Command {
public:
- virtual LockType locktype() const { return NONE; }
+ virtual LockType locktype() const { return NONE; }
virtual bool requiresAuth() { return false; }
virtual bool slaveOk() const {
return true;
@@ -827,7 +852,7 @@ namespace mongo {
class CmdShardingGetLastError : public Command {
public:
- virtual LockType locktype() const { return NONE; }
+ virtual LockType locktype() const { return NONE; }
virtual bool requiresAuth() { return false; }
virtual bool slaveOk() const {
return true;
@@ -836,196 +861,147 @@ namespace mongo {
help << "check for an error on the last command executed";
}
CmdShardingGetLastError() : Command("getLastError" , false , "getlasterror") { }
-
- void addWriteBack( vector<OID>& all , const BSONObj& o ){
- BSONElement e = o["writeback"];
- if ( e.type() == jstOID )
- all.push_back( e.OID() );
- }
-
- void handleWriteBacks( vector<OID>& all ){
- if ( all.size() == 0 )
- return;
-
- for ( unsigned i=0; i<all.size(); i++ ){
- waitForWriteback( all[i] );
- }
- }
-
virtual bool run(const string& dbName, BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
LastError *le = lastError.disableForCommand();
{
assert( le );
- if ( le->msg.size() && le->nPrev == 1 ){
+ if ( le->msg.size() && le->nPrev == 1 ) {
le->appendSelf( result );
return true;
}
}
-
+
ClientInfo * client = ClientInfo::get();
- set<string> * shards = client->getPrev();
-
- if ( shards->size() == 0 ){
- result.appendNull( "err" );
- return true;
- }
+ return client->getLastError( cmdObj , result );
+ }
+ } cmdGetLastError;
- //log() << "getlasterror enter: " << shards->size() << endl;
+ }
+ class CmdShardingResetError : public Command {
+ public:
+ CmdShardingResetError() : Command( "resetError" , false , "reseterror" ) {}
- vector<OID> writebacks;
-
- // handle single server
- if ( shards->size() == 1 ){
- string theShard = *(shards->begin() );
- result.append( "theshard" , theShard.c_str() );
- ShardConnection conn( theShard , "" );
- BSONObj res;
- bool ok = conn->runCommand( dbName , cmdObj , res );
- //log() << "\t" << res << endl;
- result.appendElements( res );
- conn.done();
- result.append( "singleShard" , theShard );
- addWriteBack( writebacks , res );
-
- // hit other machines just to block
- for ( set<string>::const_iterator i=client->sinceLastGetError().begin(); i!=client->sinceLastGetError().end(); ++i ){
- string temp = *i;
- if ( temp == theShard )
- continue;
-
- ShardConnection conn( temp , "" );
- addWriteBack( writebacks , conn->getLastErrorDetailed() );
- conn.done();
- }
- client->clearSinceLastGetError();
- handleWriteBacks( writebacks );
- return ok;
- }
-
- BSONArrayBuilder bbb( result.subarrayStart( "shards" ) );
-
- long long n = 0;
-
- // hit each shard
- vector<string> errors;
- for ( set<string>::iterator i = shards->begin(); i != shards->end(); i++ ){
- string theShard = *i;
- bbb.append( theShard );
- ShardConnection conn( theShard , "" );
- BSONObj res;
- bool ok = conn->runCommand( dbName , cmdObj , res );
- addWriteBack( writebacks, res );
- string temp = DBClientWithCommands::getLastErrorString( res );
- if ( ok == false || temp.size() )
- errors.push_back( temp );
- n += res["n"].numberLong();
- conn.done();
- }
-
- bbb.done();
-
- result.appendNumber( "n" , n );
-
- // hit other machines just to block
- for ( set<string>::const_iterator i=client->sinceLastGetError().begin(); i!=client->sinceLastGetError().end(); ++i ){
- string temp = *i;
- if ( shards->count( temp ) )
- continue;
-
- ShardConnection conn( temp , "" );
- addWriteBack( writebacks, conn->getLastErrorDetailed() );
- conn.done();
- }
- client->clearSinceLastGetError();
+ virtual LockType locktype() const { return NONE; }
+ virtual bool requiresAuth() { return false; }
+ virtual bool slaveOk() const {
+ return true;
+ }
- if ( errors.size() == 0 ){
- result.appendNull( "err" );
- handleWriteBacks( writebacks );
- return true;
- }
-
- result.append( "err" , errors[0].c_str() );
-
- BSONObjBuilder all;
- for ( unsigned i=0; i<errors.size(); i++ ){
- all.append( all.numStr( i ) , errors[i].c_str() );
- }
- result.appendArray( "errs" , all.obj() );
- handleWriteBacks( writebacks );
- return true;
+ bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool /*fromRepl*/) {
+ LastError *le = lastError.get();
+ if ( le )
+ le->reset();
+
+ ClientInfo * client = ClientInfo::get();
+ set<string> * shards = client->getPrev();
+
+ for ( set<string>::iterator i = shards->begin(); i != shards->end(); i++ ) {
+ string theShard = *i;
+ ShardConnection conn( theShard , "" );
+ BSONObj res;
+ conn->runCommand( dbName , cmdObj , res );
+ conn.done();
}
- } cmdGetLastError;
-
- }
-
+
+ return true;
+ }
+ } cmdShardingResetError;
+
class CmdListDatabases : public Command {
public:
- CmdListDatabases() : Command("listDatabases", false , "listdatabases" ) {}
+ CmdListDatabases() : Command("listDatabases", true , "listdatabases" ) {}
virtual bool logTheOp() { return false; }
virtual bool slaveOk() const { return true; }
virtual bool slaveOverrideOk() { return true; }
virtual bool adminOnly() const { return true; }
- virtual LockType locktype() const { return NONE; }
+ virtual LockType locktype() const { return NONE; }
virtual void help( stringstream& help ) const { help << "list databases on cluster"; }
-
+
bool run(const string& , BSONObj& jsobj, string& errmsg, BSONObjBuilder& result, bool /*fromRepl*/) {
vector<Shard> shards;
Shard::getAllShards( shards );
-
+
map<string,long long> sizes;
map< string,shared_ptr<BSONObjBuilder> > dbShardInfo;
- for ( vector<Shard>::iterator i=shards.begin(); i!=shards.end(); i++ ){
+ for ( vector<Shard>::iterator i=shards.begin(); i!=shards.end(); i++ ) {
Shard s = *i;
BSONObj x = s.runCommand( "admin" , "listDatabases" );
BSONObjIterator j( x["databases"].Obj() );
- while ( j.more() ){
+ while ( j.more() ) {
BSONObj theDB = j.next().Obj();
-
+
string name = theDB["name"].String();
long long size = theDB["sizeOnDisk"].numberLong();
long long& totalSize = sizes[name];
- if ( size == 1 ){
+ if ( size == 1 ) {
if ( totalSize <= 1 )
totalSize = 1;
}
else
totalSize += size;
-
+
shared_ptr<BSONObjBuilder>& bb = dbShardInfo[name];
if ( ! bb.get() )
bb.reset( new BSONObjBuilder() );
bb->appendNumber( s.getName() , size );
}
-
+
}
-
+
long long totalSize = 0;
BSONArrayBuilder bb( result.subarrayStart( "databases" ) );
- for ( map<string,long long>::iterator i=sizes.begin(); i!=sizes.end(); ++i ){
+ for ( map<string,long long>::iterator i=sizes.begin(); i!=sizes.end(); ++i ) {
string name = i->first;
+
+ if ( name == "local" ) {
+ // we don't return local
+ // since all shards have their own independant local
+ continue;
+ }
+
long long size = i->second;
totalSize += size;
-
+
BSONObjBuilder temp;
temp.append( "name" , name );
- temp.appendNumber( "size" , size );
+ temp.appendNumber( "sizeOnDisk" , size );
temp.appendBool( "empty" , size == 1 );
temp.append( "shards" , dbShardInfo[name]->obj() );
-
+
bb.append( temp.obj() );
}
+
+ if ( sizes.find( "config" ) == sizes.end() ){
+ ScopedDbConnection conn( configServer.getPrimary() );
+ BSONObj x;
+ if ( conn->simpleCommand( "config" , &x , "dbstats" ) ){
+ BSONObjBuilder b;
+ b.append( "name" , "config" );
+ b.appendBool( "empty" , false );
+ if ( x["fileSize"].type() )
+ b.appendAs( x["fileSize"] , "sizeOnDisk" );
+ else
+ b.append( "sizeOnDisk" , 1 );
+ bb.append( b.obj() );
+ }
+ else {
+ bb.append( BSON( "name" << "config" ) );
+ }
+ conn.done();
+ }
+
bb.done();
result.appendNumber( "totalSize" , totalSize );
result.appendNumber( "totalSizeMb" , totalSize / ( 1024 * 1024 ) );
-
+
return 1;
}
@@ -1038,9 +1014,9 @@ namespace mongo {
virtual bool slaveOk() const { return true; }
virtual bool slaveOverrideOk() { return true; }
virtual bool adminOnly() const { return true; }
- virtual LockType locktype() const { return NONE; }
+ virtual LockType locktype() const { return NONE; }
virtual void help( stringstream& help ) const { help << "Not supported sharded"; }
-
+
bool run(const string& , BSONObj& jsobj, string& errmsg, BSONObjBuilder& /*result*/, bool /*fromRepl*/) {
errmsg = "closeAllDatabases isn't supported through mongos";
return false;
@@ -1048,4 +1024,22 @@ namespace mongo {
} cmdCloseAllDatabases;
+ class CmdReplSetGetStatus : public Command {
+ public:
+ CmdReplSetGetStatus() : Command("replSetGetStatus"){}
+ virtual bool logTheOp() { return false; }
+ virtual bool slaveOk() const { return true; }
+ virtual bool adminOnly() const { return true; }
+ virtual LockType locktype() const { return NONE; }
+ virtual void help( stringstream& help ) const { help << "Not supported through mongos"; }
+
+ bool run(const string& , BSONObj& jsobj, string& errmsg, BSONObjBuilder& /*result*/, bool /*fromRepl*/) {
+ if ( jsobj["forShell"].trueValue() )
+ lastError.disableForCommand();
+
+ errmsg = "replSetGetStatus is not supported through mongos";
+ return false;
+ }
+ } cmdReplSetGetStatus;
+
} // namespace mongo
diff --git a/s/commands_public.cpp b/s/commands_public.cpp
index 80d5cc9..02000a0 100644
--- a/s/commands_public.cpp
+++ b/s/commands_public.cpp
@@ -33,10 +33,10 @@
namespace mongo {
namespace dbgrid_pub_cmds {
-
+
class PublicGridCommand : public Command {
public:
- PublicGridCommand( const char* n, const char* oldname=NULL ) : Command( n, false, oldname ){
+ PublicGridCommand( const char* n, const char* oldname=NULL ) : Command( n, false, oldname ) {
}
virtual bool slaveOk() const {
return true;
@@ -46,18 +46,18 @@ namespace mongo {
}
// all grid commands are designed not to lock
- virtual LockType locktype() const { return NONE; }
+ virtual LockType locktype() const { return NONE; }
protected:
- bool passthrough( DBConfigPtr conf, const BSONObj& cmdObj , BSONObjBuilder& result ){
+ bool passthrough( DBConfigPtr conf, const BSONObj& cmdObj , BSONObjBuilder& result ) {
return _passthrough(conf->getName(), conf, cmdObj, result);
}
- bool adminPassthrough( DBConfigPtr conf, const BSONObj& cmdObj , BSONObjBuilder& result ){
+ bool adminPassthrough( DBConfigPtr conf, const BSONObj& cmdObj , BSONObjBuilder& result ) {
return _passthrough("admin", conf, cmdObj, result);
}
-
+
private:
- bool _passthrough(const string& db, DBConfigPtr conf, const BSONObj& cmdObj , BSONObjBuilder& result ){
+ bool _passthrough(const string& db, DBConfigPtr conf, const BSONObj& cmdObj , BSONObjBuilder& result ) {
ShardConnection conn( conf->getPrimary() , "" );
BSONObj res;
bool ok = conn->runCommand( db , cmdObj , res );
@@ -75,33 +75,33 @@ namespace mongo {
virtual bool adminOnly() const { return false; }
// all grid commands are designed not to lock
- virtual LockType locktype() const { return NONE; }
+ virtual LockType locktype() const { return NONE; }
// default impl uses all shards for DB
- virtual void getShards(const string& dbName , BSONObj& cmdObj, set<Shard>& shards){
+ virtual void getShards(const string& dbName , BSONObj& cmdObj, set<Shard>& shards) {
DBConfigPtr conf = grid.getDBConfig( dbName , false );
conf->getAllShards(shards);
}
-
+
virtual void aggregateResults(const vector<BSONObj>& results, BSONObjBuilder& output) {}
// don't override
- virtual bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& output, bool){
+ virtual bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& output, bool) {
set<Shard> shards;
getShards(dbName, cmdObj, shards);
list< shared_ptr<Future::CommandResult> > futures;
- for ( set<Shard>::const_iterator i=shards.begin(), end=shards.end() ; i != end ; i++ ){
+ for ( set<Shard>::const_iterator i=shards.begin(), end=shards.end() ; i != end ; i++ ) {
futures.push_back( Future::spawnCommand( i->getConnString() , dbName , cmdObj ) );
}
-
+
vector<BSONObj> results;
BSONObjBuilder subobj (output.subobjStart("raw"));
BSONObjBuilder errors;
- for ( list< shared_ptr<Future::CommandResult> >::iterator i=futures.begin(); i!=futures.end(); i++ ){
+ for ( list< shared_ptr<Future::CommandResult> >::iterator i=futures.begin(); i!=futures.end(); i++ ) {
shared_ptr<Future::CommandResult> res = *i;
- if ( ! res->join() ){
+ if ( ! res->join() ) {
errors.appendAs(res->result()["errmsg"], res->getServer());
}
results.push_back( res->result() );
@@ -111,11 +111,11 @@ namespace mongo {
subobj.done();
BSONObj errobj = errors.done();
- if (! errobj.isEmpty()){
+ if (! errobj.isEmpty()) {
errmsg = errobj.toString(false, true);
return false;
}
-
+
aggregateResults(results, output);
return true;
}
@@ -126,39 +126,40 @@ namespace mongo {
public:
AllShardsCollectionCommand(const char* n, const char* oldname=NULL) : RunOnAllShardsCommand(n, oldname) {}
- virtual void getShards(const string& dbName , BSONObj& cmdObj, set<Shard>& shards){
+ virtual void getShards(const string& dbName , BSONObj& cmdObj, set<Shard>& shards) {
string fullns = dbName + '.' + cmdObj.firstElement().valuestrsafe();
-
+
DBConfigPtr conf = grid.getDBConfig( dbName , false );
-
- if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ){
+
+ if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ) {
shards.insert(conf->getShard(fullns));
- } else {
+ }
+ else {
conf->getChunkManager(fullns)->getAllShards(shards);
}
}
};
-
+
class NotAllowedOnShardedCollectionCmd : public PublicGridCommand {
public:
- NotAllowedOnShardedCollectionCmd( const char * n ) : PublicGridCommand( n ){}
+ NotAllowedOnShardedCollectionCmd( const char * n ) : PublicGridCommand( n ) {}
virtual string getFullNS( const string& dbName , const BSONObj& cmdObj ) = 0;
-
- virtual bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+
+ virtual bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
string fullns = getFullNS( dbName , cmdObj );
-
+
DBConfigPtr conf = grid.getDBConfig( dbName , false );
-
- if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ){
+
+ if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ) {
return passthrough( conf , cmdObj , result );
}
errmsg = "can't do command: " + name + " on sharded collection";
return false;
}
};
-
+
// ----
class DropIndexesCmd : public AllShardsCollectionCommand {
@@ -194,7 +195,7 @@ namespace mongo {
long long indexSize = 0;
long long fileSize = 0;
- for (vector<BSONObj>::const_iterator it(results.begin()), end(results.end()); it != end; ++it){
+ for (vector<BSONObj>::const_iterator it(results.begin()), end(results.end()); it != end; ++it) {
const BSONObj& b = *it;
objects += b["objects"].numberLong();
dataSize += b["dataSize"].numberLong();
@@ -219,23 +220,24 @@ namespace mongo {
class DropCmd : public PublicGridCommand {
public:
- DropCmd() : PublicGridCommand( "drop" ){}
- bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ DropCmd() : PublicGridCommand( "drop" ) {}
+ bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
string collection = cmdObj.firstElement().valuestrsafe();
string fullns = dbName + "." + collection;
-
+
DBConfigPtr conf = grid.getDBConfig( dbName , false );
-
+
log() << "DROP: " << fullns << endl;
-
- if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ){
+
+ if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ) {
return passthrough( conf , cmdObj , result );
}
-
+
ChunkManagerPtr cm = conf->getChunkManager( fullns );
massert( 10418 , "how could chunk manager be null!" , cm );
-
+
cm->drop( cm );
+ uassert( 13512 , "drop collection attempted on non-sharded collection" , conf->removeSharding( fullns ) );
return 1;
}
@@ -243,25 +245,25 @@ namespace mongo {
class DropDBCmd : public PublicGridCommand {
public:
- DropDBCmd() : PublicGridCommand( "dropDatabase" ){}
- bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
-
+ DropDBCmd() : PublicGridCommand( "dropDatabase" ) {}
+ bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
+
BSONElement e = cmdObj.firstElement();
-
- if ( ! e.isNumber() || e.number() != 1 ){
+
+ if ( ! e.isNumber() || e.number() != 1 ) {
errmsg = "invalid params";
return 0;
}
-
+
DBConfigPtr conf = grid.getDBConfig( dbName , false );
-
+
log() << "DROP DATABASE: " << dbName << endl;
- if ( ! conf ){
+ if ( ! conf ) {
result.append( "info" , "database didn't exist" );
return true;
}
-
+
if ( ! conf->dropDatabase( errmsg ) )
return false;
@@ -272,8 +274,8 @@ namespace mongo {
class RenameCollectionCmd : public PublicGridCommand {
public:
- RenameCollectionCmd() : PublicGridCommand( "renameCollection" ){}
- bool run(const string& dbName, BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ RenameCollectionCmd() : PublicGridCommand( "renameCollection" ) {}
+ bool run(const string& dbName, BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
string fullnsFrom = cmdObj.firstElement().valuestrsafe();
string dbNameFrom = nsToDatabase( fullnsFrom.c_str() );
DBConfigPtr confFrom = grid.getDBConfig( dbNameFrom , false );
@@ -297,18 +299,19 @@ namespace mongo {
class CopyDBCmd : public PublicGridCommand {
public:
- CopyDBCmd() : PublicGridCommand( "copydb" ){}
- bool run(const string& dbName, BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ CopyDBCmd() : PublicGridCommand( "copydb" ) {}
+ bool run(const string& dbName, BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
string todb = cmdObj.getStringField("todb");
uassert(13402, "need a todb argument", !todb.empty());
-
+
DBConfigPtr confTo = grid.getDBConfig( todb );
uassert(13398, "cant copy to sharded DB", !confTo->isShardingEnabled());
string fromhost = cmdObj.getStringField("fromhost");
- if (!fromhost.empty()){
+ if (!fromhost.empty()) {
return adminPassthrough( confTo , cmdObj , result );
- } else {
+ }
+ else {
string fromdb = cmdObj.getStringField("fromdb");
uassert(13399, "need a fromdb argument", !fromdb.empty());
@@ -317,7 +320,7 @@ namespace mongo {
uassert(13401, "cant copy from sharded DB", !confFrom->isShardingEnabled());
BSONObjBuilder b;
- BSONForEach(e, cmdObj){
+ BSONForEach(e, cmdObj) {
if (strcmp(e.fieldName(), "fromhost") != 0)
b.append(e);
}
@@ -328,67 +331,67 @@ namespace mongo {
}
}
- }copyDBCmd;
+ } copyDBCmd;
class CountCmd : public PublicGridCommand {
public:
CountCmd() : PublicGridCommand("count") { }
- bool run(const string& dbName, BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool l){
+ bool run(const string& dbName, BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool l) {
string collection = cmdObj.firstElement().valuestrsafe();
string fullns = dbName + "." + collection;
-
+
BSONObj filter;
if ( cmdObj["query"].isABSONObj() )
filter = cmdObj["query"].Obj();
-
+
DBConfigPtr conf = grid.getDBConfig( dbName , false );
-
- if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ){
+
+ if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ) {
ShardConnection conn( conf->getPrimary() , fullns );
BSONObj temp;
bool ok = conn->runCommand( dbName , cmdObj , temp );
conn.done();
-
- if ( ok ){
+
+ if ( ok ) {
result.append( temp["n"] );
return true;
}
-
- if ( temp["code"].numberInt() != StaleConfigInContextCode ){
+
+ if ( temp["code"].numberInt() != StaleConfigInContextCode ) {
errmsg = temp["errmsg"].String();
result.appendElements( temp );
return false;
}
-
+
// this collection got sharded
ChunkManagerPtr cm = conf->getChunkManager( fullns , true );
- if ( ! cm ){
+ if ( ! cm ) {
errmsg = "should be sharded now";
result.append( "root" , temp );
return false;
}
}
-
+
long long total = 0;
map<string,long long> shardCounts;
-
+
ChunkManagerPtr cm = conf->getChunkManager( fullns );
- while ( true ){
- if ( ! cm ){
+ while ( true ) {
+ if ( ! cm ) {
// probably unsharded now
return run( dbName , cmdObj , errmsg , result , l );
}
-
+
set<Shard> shards;
cm->getShardsForQuery( shards , filter );
assert( shards.size() );
-
+
bool hadToBreak = false;
- for (set<Shard>::iterator it=shards.begin(), end=shards.end(); it != end; ++it){
+ for (set<Shard>::iterator it=shards.begin(), end=shards.end(); it != end; ++it) {
ShardConnection conn(*it, fullns);
- if ( conn.setVersion() ){
+ if ( conn.setVersion() ) {
total = 0;
shardCounts.clear();
cm = conf->getChunkManager( fullns );
@@ -396,19 +399,19 @@ namespace mongo {
hadToBreak = true;
break;
}
-
+
BSONObj temp;
bool ok = conn->runCommand( dbName , BSON( "count" << collection << "query" << filter ) , temp );
conn.done();
-
- if ( ok ){
+
+ if ( ok ) {
long long mine = temp["n"].numberLong();
total += mine;
shardCounts[it->getName()] = mine;
continue;
}
-
- if ( StaleConfigInContextCode == temp["code"].numberInt() ){
+
+ if ( StaleConfigInContextCode == temp["code"].numberInt() ) {
// my version is old
total = 0;
shardCounts.clear();
@@ -425,7 +428,7 @@ namespace mongo {
if ( ! hadToBreak )
break;
}
-
+
total = applySkipLimit( total , cmdObj );
result.appendNumber( "n" , total );
BSONObjBuilder temp( result.subobjStart( "shards" ) );
@@ -439,13 +442,13 @@ namespace mongo {
class CollectionStats : public PublicGridCommand {
public:
CollectionStats() : PublicGridCommand("collStats", "collstats") { }
- bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
string collection = cmdObj.firstElement().valuestrsafe();
string fullns = dbName + "." + collection;
-
+
DBConfigPtr conf = grid.getDBConfig( dbName , false );
-
- if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ){
+
+ if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ) {
result.append( "ns" , fullns );
result.appendBool("sharded", false);
result.append( "primary" , conf->getPrimary().getName() );
@@ -458,17 +461,17 @@ namespace mongo {
set<Shard> servers;
cm->getAllShards(servers);
-
+
BSONObjBuilder shardStats;
long long count=0;
long long size=0;
long long storageSize=0;
int nindexes=0;
bool warnedAboutIndexes = false;
- for ( set<Shard>::iterator i=servers.begin(); i!=servers.end(); i++ ){
+ for ( set<Shard>::iterator i=servers.begin(); i!=servers.end(); i++ ) {
ScopedDbConnection conn( *i );
BSONObj res;
- if ( ! conn->runCommand( dbName , cmdObj , res ) ){
+ if ( ! conn->runCommand( dbName , cmdObj , res ) ) {
errmsg = "failed on shard: " + res.toString();
return false;
}
@@ -480,19 +483,19 @@ namespace mongo {
int myIndexes = res["nindexes"].numberInt();
- if ( nindexes == 0 ){
+ if ( nindexes == 0 ) {
nindexes = myIndexes;
}
- else if ( nindexes == myIndexes ){
+ else if ( nindexes == myIndexes ) {
// no-op
}
else {
// hopefully this means we're building an index
-
+
if ( myIndexes > nindexes )
nindexes = myIndexes;
-
- if ( ! warnedAboutIndexes ){
+
+ if ( ! warnedAboutIndexes ) {
result.append( "warning" , "indexes don't all match - ok if ensureIndex is running" );
warnedAboutIndexes = true;
}
@@ -510,7 +513,7 @@ namespace mongo {
result.append("nchunks", cm->numChunks());
result.append("shards", shardStats.obj());
-
+
return true;
}
} collectionStatsCmd;
@@ -518,19 +521,19 @@ namespace mongo {
class FindAndModifyCmd : public PublicGridCommand {
public:
FindAndModifyCmd() : PublicGridCommand("findAndModify", "findandmodify") { }
- bool run(const string& dbName, BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& dbName, BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
string collection = cmdObj.firstElement().valuestrsafe();
string fullns = dbName + "." + collection;
-
+
DBConfigPtr conf = grid.getDBConfig( dbName , false );
-
- if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ){
+
+ if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ) {
return passthrough( conf , cmdObj , result);
}
-
+
ChunkManagerPtr cm = conf->getChunkManager( fullns );
massert( 13002 , "how could chunk manager be null!" , cm );
-
+
BSONObj filter = cmdObj.getObjectField("query");
uassert(13343, "query for sharded findAndModify must have shardkey", cm->hasShardKey(filter));
@@ -542,11 +545,11 @@ namespace mongo {
bool ok = conn->runCommand( conf->getName() , cmdObj , res );
conn.done();
- if (ok || (strcmp(res["errmsg"].valuestrsafe(), "No matching object found") != 0)){
+ if (ok || (strcmp(res["errmsg"].valuestrsafe(), "No matching object found") != 0)) {
result.appendElements(res);
return ok;
}
-
+
return true;
}
@@ -555,18 +558,18 @@ namespace mongo {
class DataSizeCmd : public PublicGridCommand {
public:
DataSizeCmd() : PublicGridCommand("dataSize", "datasize") { }
- bool run(const string& dbName, BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& dbName, BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
string fullns = cmdObj.firstElement().String();
-
+
DBConfigPtr conf = grid.getDBConfig( dbName , false );
-
- if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ){
+
+ if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ) {
return passthrough( conf , cmdObj , result);
}
-
+
ChunkManagerPtr cm = conf->getChunkManager( fullns );
massert( 13407 , "how could chunk manager be null!" , cm );
-
+
BSONObj min = cmdObj.getObjectField( "min" );
BSONObj max = cmdObj.getObjectField( "max" );
BSONObj keyPattern = cmdObj.getObjectField( "keyPattern" );
@@ -580,13 +583,13 @@ namespace mongo {
set<Shard> shards;
cm->getShardsForRange(shards, min, max);
- for ( set<Shard>::iterator i=shards.begin(), end=shards.end() ; i != end; ++i ){
+ for ( set<Shard>::iterator i=shards.begin(), end=shards.end() ; i != end; ++i ) {
ScopedDbConnection conn( *i );
BSONObj res;
bool ok = conn->runCommand( conf->getName() , cmdObj , res );
conn.done();
-
- if ( ! ok ){
+
+ if ( ! ok ) {
result.appendElements( res );
return false;
}
@@ -607,64 +610,64 @@ namespace mongo {
class ConvertToCappedCmd : public NotAllowedOnShardedCollectionCmd {
public:
- ConvertToCappedCmd() : NotAllowedOnShardedCollectionCmd("convertToCapped"){}
-
- virtual string getFullNS( const string& dbName , const BSONObj& cmdObj ){
+ ConvertToCappedCmd() : NotAllowedOnShardedCollectionCmd("convertToCapped") {}
+
+ virtual string getFullNS( const string& dbName , const BSONObj& cmdObj ) {
return dbName + "." + cmdObj.firstElement().valuestrsafe();
}
-
+
} convertToCappedCmd;
class GroupCmd : public NotAllowedOnShardedCollectionCmd {
public:
- GroupCmd() : NotAllowedOnShardedCollectionCmd("group"){}
-
- virtual string getFullNS( const string& dbName , const BSONObj& cmdObj ){
+ GroupCmd() : NotAllowedOnShardedCollectionCmd("group") {}
+
+ virtual string getFullNS( const string& dbName , const BSONObj& cmdObj ) {
return dbName + "." + cmdObj.firstElement().embeddedObjectUserCheck()["ns"].valuestrsafe();
}
-
+
} groupCmd;
class DistinctCmd : public PublicGridCommand {
public:
- DistinctCmd() : PublicGridCommand("distinct"){}
+ DistinctCmd() : PublicGridCommand("distinct") {}
virtual void help( stringstream &help ) const {
help << "{ distinct : 'collection name' , key : 'a.b' , query : {} }";
}
- bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
string collection = cmdObj.firstElement().valuestrsafe();
string fullns = dbName + "." + collection;
DBConfigPtr conf = grid.getDBConfig( dbName , false );
-
- if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ){
+
+ if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ) {
return passthrough( conf , cmdObj , result );
}
-
+
ChunkManagerPtr cm = conf->getChunkManager( fullns );
massert( 10420 , "how could chunk manager be null!" , cm );
BSONObj query = getQuery(cmdObj);
set<Shard> shards;
cm->getShardsForQuery(shards, query);
-
+
set<BSONObj,BSONObjCmp> all;
int size = 32;
-
- for ( set<Shard>::iterator i=shards.begin(), end=shards.end() ; i != end; ++i ){
+
+ for ( set<Shard>::iterator i=shards.begin(), end=shards.end() ; i != end; ++i ) {
ShardConnection conn( *i , fullns );
BSONObj res;
bool ok = conn->runCommand( conf->getName() , cmdObj , res );
conn.done();
-
- if ( ! ok ){
+
+ if ( ! ok ) {
result.appendElements( res );
return false;
}
-
+
BSONObjIterator it( res["values"].embeddedObject() );
- while ( it.more() ){
+ while ( it.more() ) {
BSONElement nxt = it.next();
BSONObjBuilder temp(32);
temp.appendAs( nxt , "" );
@@ -672,13 +675,13 @@ namespace mongo {
}
}
-
+
BSONObjBuilder b( size );
int n=0;
- for ( set<BSONObj,BSONObjCmp>::iterator i = all.begin() ; i != all.end(); i++ ){
- b.appendAs( i->firstElement() , b.numStr( n++ ).c_str() );
+ for ( set<BSONObj,BSONObjCmp>::iterator i = all.begin() ; i != all.end(); i++ ) {
+ b.appendAs( i->firstElement() , b.numStr( n++ ) );
}
-
+
result.appendArray( "values" , b.obj() );
return true;
}
@@ -686,11 +689,11 @@ namespace mongo {
class FileMD5Cmd : public PublicGridCommand {
public:
- FileMD5Cmd() : PublicGridCommand("filemd5"){}
+ FileMD5Cmd() : PublicGridCommand("filemd5") {}
virtual void help( stringstream &help ) const {
help << " example: { filemd5 : ObjectId(aaaaaaa) , root : \"fs\" }";
}
- bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
string fullns = dbName;
fullns += ".";
{
@@ -702,17 +705,17 @@ namespace mongo {
fullns += ".chunks";
DBConfigPtr conf = grid.getDBConfig( dbName , false );
-
- if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ){
+
+ if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ) {
return passthrough( conf , cmdObj , result );
}
-
+
ChunkManagerPtr cm = conf->getChunkManager( fullns );
massert( 13091 , "how could chunk manager be null!" , cm );
uassert( 13092 , "GridFS chunks collection can only be sharded on files_id", cm->getShardKey().key() == BSON("files_id" << 1));
ChunkPtr chunk = cm->findChunk( BSON("files_id" << cmdObj.firstElement()) );
-
+
ShardConnection conn( chunk->getShard() , fullns );
BSONObj res;
bool ok = conn->runCommand( conf->getName() , cmdObj , res );
@@ -723,104 +726,254 @@ namespace mongo {
}
} fileMD5Cmd;
+ class Geo2dFindNearCmd : public PublicGridCommand {
+ public:
+ Geo2dFindNearCmd() : PublicGridCommand( "geoNear" ) {}
+ void help(stringstream& h) const { h << "http://www.mongodb.org/display/DOCS/Geospatial+Indexing#GeospatialIndexing-geoNearCommand"; }
+
+ bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
+ string collection = cmdObj.firstElement().valuestrsafe();
+ string fullns = dbName + "." + collection;
+
+ DBConfigPtr conf = grid.getDBConfig( dbName , false );
+
+ if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ) {
+ return passthrough( conf , cmdObj , result );
+ }
+
+ ChunkManagerPtr cm = conf->getChunkManager( fullns );
+ massert( 13500 , "how could chunk manager be null!" , cm );
+
+ BSONObj query = getQuery(cmdObj);
+ set<Shard> shards;
+ cm->getShardsForQuery(shards, query);
+
+ int limit = 100;
+ if (cmdObj["num"].isNumber())
+ limit = cmdObj["num"].numberInt();
+
+ list< shared_ptr<Future::CommandResult> > futures;
+ BSONArrayBuilder shardArray;
+ for ( set<Shard>::const_iterator i=shards.begin(), end=shards.end() ; i != end ; i++ ) {
+ futures.push_back( Future::spawnCommand( i->getConnString() , dbName , cmdObj ) );
+ shardArray.append(i->getName());
+ }
+
+ multimap<double, BSONObj> results; // TODO: maybe use merge-sort instead
+ string nearStr;
+ double time = 0;
+ double btreelocs = 0;
+ double nscanned = 0;
+ double objectsLoaded = 0;
+ for ( list< shared_ptr<Future::CommandResult> >::iterator i=futures.begin(); i!=futures.end(); i++ ) {
+ shared_ptr<Future::CommandResult> res = *i;
+ if ( ! res->join() ) {
+ errmsg = res->result()["errmsg"].String();
+ return false;
+ }
+
+ nearStr = res->result()["near"].String();
+ time += res->result()["stats"]["time"].Number();
+ btreelocs += res->result()["stats"]["btreelocs"].Number();
+ nscanned += res->result()["stats"]["nscanned"].Number();
+ objectsLoaded += res->result()["stats"]["objectsLoaded"].Number();
+
+ BSONForEach(obj, res->result()["results"].embeddedObject()) {
+ results.insert(make_pair(obj["dis"].Number(), obj.embeddedObject().getOwned()));
+ }
+
+ // TODO: maybe shrink results if size() > limit
+ }
+
+ result.append("ns" , fullns);
+ result.append("near", nearStr);
+
+ int outCount = 0;
+ double totalDistance = 0;
+ double maxDistance = 0;
+ {
+ BSONArrayBuilder sub (result.subarrayStart("results"));
+ for (multimap<double, BSONObj>::const_iterator it(results.begin()), end(results.end()); it!= end && outCount < limit; ++it, ++outCount) {
+ totalDistance += it->first;
+ maxDistance = it->first; // guaranteed to be highest so far
+
+ sub.append(it->second);
+ }
+ sub.done();
+ }
+
+ {
+ BSONObjBuilder sub (result.subobjStart("stats"));
+ sub.append("time", time);
+ sub.append("btreelocs", btreelocs);
+ sub.append("nscanned", nscanned);
+ sub.append("objectsLoaded", objectsLoaded);
+ sub.append("avgDistance", totalDistance / outCount);
+ sub.append("maxDistance", maxDistance);
+ sub.append("shards", shardArray.arr());
+ sub.done();
+ }
+
+ return true;
+ }
+ } geo2dFindNearCmd;
+
class MRCmd : public PublicGridCommand {
public:
- MRCmd() : PublicGridCommand( "mapreduce" ){}
-
- string getTmpName( const string& coll ){
+ MRCmd() : PublicGridCommand( "mapreduce" ) {}
+
+ string getTmpName( const string& coll ) {
static int inc = 1;
stringstream ss;
ss << "tmp.mrs." << coll << "_" << time(0) << "_" << inc++;
return ss.str();
}
- BSONObj fixForShards( const BSONObj& orig , const string& output ){
+ BSONObj fixForShards( const BSONObj& orig , const string& output, BSONObj& customOut , string& badShardedField ) {
BSONObjBuilder b;
BSONObjIterator i( orig );
- while ( i.more() ){
+ while ( i.more() ) {
BSONElement e = i.next();
string fn = e.fieldName();
- if ( fn == "map" ||
- fn == "mapreduce" ||
- fn == "reduce" ||
- fn == "query" ||
- fn == "sort" ||
- fn == "scope" ||
- fn == "verbose" ){
+ if ( fn == "map" ||
+ fn == "mapreduce" ||
+ fn == "mapparams" ||
+ fn == "reduce" ||
+ fn == "query" ||
+ fn == "sort" ||
+ fn == "scope" ||
+ fn == "verbose" ) {
b.append( e );
}
- else if ( fn == "keeptemp" ||
- fn == "out" ||
- fn == "finalize" ){
+ else if ( fn == "out" ||
+ fn == "finalize" ) {
// we don't want to copy these
+ if (fn == "out" && e.type() == Object) {
+ // check if there is a custom output
+ BSONObj out = e.embeddedObject();
+ if (out.hasField("db"))
+ customOut = out;
+ }
}
else {
- uassert( 10177 , (string)"don't know mr field: " + fn , 0 );
+ badShardedField = fn;
+ return BSONObj();
}
}
b.append( "out" , output );
return b.obj();
}
-
- bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+
+ bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
Timer t;
string collection = cmdObj.firstElement().valuestrsafe();
string fullns = dbName + "." + collection;
+ const string shardedOutputCollection = getTmpName( collection );
+
+ string badShardedField;
+ BSONObj customOut;
+ BSONObj shardedCommand = fixForShards( cmdObj , shardedOutputCollection, customOut , badShardedField );
+
+ bool customOutDB = ! customOut.isEmpty() && customOut.hasField( "db" );
+
DBConfigPtr conf = grid.getDBConfig( dbName , false );
- if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ){
+ if ( ! conf || ! conf->isShardingEnabled() || ! conf->isSharded( fullns ) ) {
+ if ( customOutDB ) {
+ errmsg = "can't use out 'db' with non-sharded db";
+ return false;
+ }
return passthrough( conf , cmdObj , result );
}
-
+
+ if ( badShardedField.size() ) {
+ errmsg = str::stream() << "unknown m/r field for sharding: " << badShardedField;
+ return false;
+ }
+
BSONObjBuilder timingBuilder;
ChunkManagerPtr cm = conf->getChunkManager( fullns );
BSONObj q;
- if ( cmdObj["query"].type() == Object ){
+ if ( cmdObj["query"].type() == Object ) {
q = cmdObj["query"].embeddedObjectUserCheck();
}
-
+
set<Shard> shards;
cm->getShardsForQuery( shards , q );
-
- const string shardedOutputCollection = getTmpName( collection );
-
- BSONObj shardedCommand = fixForShards( cmdObj , shardedOutputCollection );
-
+
+
BSONObjBuilder finalCmd;
finalCmd.append( "mapreduce.shardedfinish" , cmdObj );
finalCmd.append( "shardedOutputCollection" , shardedOutputCollection );
- list< shared_ptr<Future::CommandResult> > futures;
-
- for ( set<Shard>::iterator i=shards.begin(), end=shards.end() ; i != end ; i++ ){
- futures.push_back( Future::spawnCommand( i->getConnString() , dbName , shardedCommand ) );
- }
- BSONObjBuilder shardresults;
- for ( list< shared_ptr<Future::CommandResult> >::iterator i=futures.begin(); i!=futures.end(); i++ ){
- shared_ptr<Future::CommandResult> res = *i;
- if ( ! res->join() ){
- errmsg = "mongod mr failed: ";
- errmsg += res->result().toString();
- return 0;
+ {
+ // we need to use our connections to the shard
+ // so filtering is done correctly for un-owned docs
+ // so we allocate them in our thread
+ // and hand off
+
+ vector< shared_ptr<ShardConnection> > shardConns;
+
+ list< shared_ptr<Future::CommandResult> > futures;
+
+ for ( set<Shard>::iterator i=shards.begin(), end=shards.end() ; i != end ; i++ ) {
+ shared_ptr<ShardConnection> temp( new ShardConnection( i->getConnString() , fullns ) );
+ assert( temp->get() );
+ futures.push_back( Future::spawnCommand( i->getConnString() , dbName , shardedCommand , temp->get() ) );
+ shardConns.push_back( temp );
+ }
+
+ bool failed = false;
+
+ BSONObjBuilder shardresults;
+ for ( list< shared_ptr<Future::CommandResult> >::iterator i=futures.begin(); i!=futures.end(); i++ ) {
+ shared_ptr<Future::CommandResult> res = *i;
+ if ( ! res->join() ) {
+ error() << "sharded m/r failed on shard: " << res->getServer() << " error: " << res->result() << endl;
+ result.append( "cause" , res->result() );
+ errmsg = "mongod mr failed: ";
+ errmsg += res->result().toString();
+ failed = true;
+ continue;
+ }
+ shardresults.append( res->getServer() , res->result() );
}
- shardresults.append( res->getServer() , res->result() );
+
+ for ( unsigned i=0; i<shardConns.size(); i++ )
+ shardConns[i]->done();
+
+ if ( failed )
+ return 0;
+
+ finalCmd.append( "shards" , shardresults.obj() );
+ timingBuilder.append( "shards" , t.millis() );
}
-
- finalCmd.append( "shards" , shardresults.obj() );
- timingBuilder.append( "shards" , t.millis() );
Timer t2;
- ShardConnection conn( conf->getPrimary() , fullns );
+ // by default the target database is same as input
+ Shard outServer = conf->getPrimary();
+ string outns = fullns;
+ if ( customOutDB ) {
+ // have to figure out shard for the output DB
+ BSONElement elmt = customOut.getField("db");
+ string outdb = elmt.valuestrsafe();
+ outns = outdb + "." + collection;
+ DBConfigPtr conf2 = grid.getDBConfig( outdb , true );
+ outServer = conf2->getPrimary();
+ }
+ log() << "customOut: " << customOut << " outServer: " << outServer << endl;
+
+ ShardConnection conn( outServer , outns );
BSONObj finalResult;
bool ok = conn->runCommand( dbName , finalCmd.obj() , finalResult );
conn.done();
- if ( ! ok ){
+ if ( ! ok ) {
errmsg = "final reduce failed: ";
errmsg += finalResult.toString();
return 0;
@@ -830,22 +983,22 @@ namespace mongo {
result.appendElements( finalResult );
result.append( "timeMillis" , t.millis() );
result.append( "timing" , timingBuilder.obj() );
-
+
return 1;
}
} mrCmd;
-
+
class ApplyOpsCmd : public PublicGridCommand {
public:
- ApplyOpsCmd() : PublicGridCommand( "applyOps" ){}
-
- virtual bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ ApplyOpsCmd() : PublicGridCommand( "applyOps" ) {}
+
+ virtual bool run(const string& dbName , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
errmsg = "applyOps not allowed through mongos";
return false;
}
-
+
} applyOpsCmd;
-
+
}
}
diff --git a/s/config.cpp b/s/config.cpp
index 1ad15d5..35a3be2 100644
--- a/s/config.cpp
+++ b/s/config.cpp
@@ -25,17 +25,17 @@
#include "../db/pdfile.h"
#include "../db/cmdline.h"
-#include "server.h"
-#include "config.h"
#include "chunk.h"
+#include "config.h"
#include "grid.h"
+#include "server.h"
namespace mongo {
int ConfigServer::VERSION = 3;
Shard Shard::EMPTY;
- string ShardNS::shard = "config.shards";
+ string ShardNS::shard = "config.shards";
string ShardNS::database = "config.databases";
string ShardNS::collection = "config.collections";
string ShardNS::chunk = "config.chunks";
@@ -45,42 +45,41 @@ namespace mongo {
BSONField<bool> ShardFields::draining("draining");
BSONField<long long> ShardFields::maxSize ("maxSize");
- BSONField<long long> ShardFields::currSize("currSize");
OID serverID;
/* --- DBConfig --- */
- DBConfig::CollectionInfo::CollectionInfo( DBConfig * db , const BSONObj& in ){
+ DBConfig::CollectionInfo::CollectionInfo( const BSONObj& in ) {
_dirty = false;
_dropped = in["dropped"].trueValue();
if ( in["key"].isABSONObj() )
- shard( db , in["_id"].String() , in["key"].Obj() , in["unique"].trueValue() );
+ shard( in["_id"].String() , in["key"].Obj() , in["unique"].trueValue() );
}
- void DBConfig::CollectionInfo::shard( DBConfig * db , const string& ns , const ShardKeyPattern& key , bool unique ){
- _cm.reset( new ChunkManager( db, ns , key , unique ) );
+ void DBConfig::CollectionInfo::shard( const string& ns , const ShardKeyPattern& key , bool unique ) {
+ _cm.reset( new ChunkManager( ns , key , unique ) );
_dirty = true;
_dropped = false;
}
- void DBConfig::CollectionInfo::unshard(){
+ void DBConfig::CollectionInfo::unshard() {
_cm.reset();
_dropped = true;
_dirty = true;
}
-
- void DBConfig::CollectionInfo::save( const string& ns , DBClientBase* conn ){
+
+ void DBConfig::CollectionInfo::save( const string& ns , DBClientBase* conn ) {
BSONObj key = BSON( "_id" << ns );
-
+
BSONObjBuilder val;
val.append( "_id" , ns );
val.appendDate( "lastmod" , time(0) );
val.appendBool( "dropped" , _dropped );
if ( _cm )
_cm->getInfo( val );
-
+
conn->update( ShardNS::collection , key , val.obj() , true );
string err = conn->getLastError();
uassert( 13473 , (string)"failed to save collection (" + ns + "): " + err , err.size() == 0 );
@@ -88,14 +87,14 @@ namespace mongo {
_dirty = false;
}
- bool DBConfig::isSharded( const string& ns ){
+ bool DBConfig::isSharded( const string& ns ) {
if ( ! _shardingEnabled )
return false;
scoped_lock lk( _lock );
return _isSharded( ns );
}
- bool DBConfig::_isSharded( const string& ns ){
+ bool DBConfig::_isSharded( const string& ns ) {
if ( ! _shardingEnabled )
return false;
Collections::iterator i = _collections.find( ns );
@@ -105,25 +104,28 @@ namespace mongo {
}
- const Shard& DBConfig::getShard( const string& ns ){
+ const Shard& DBConfig::getShard( const string& ns ) {
if ( isSharded( ns ) )
return Shard::EMPTY;
-
+
uassert( 10178 , "no primary!" , _primary.ok() );
return _primary;
}
-
- void DBConfig::enableSharding(){
+
+ void DBConfig::enableSharding() {
if ( _shardingEnabled )
return;
+
+ assert( _name != "config" );
+
scoped_lock lk( _lock );
- _shardingEnabled = true;
+ _shardingEnabled = true;
_save();
}
-
- ChunkManagerPtr DBConfig::shardCollection( const string& ns , ShardKeyPattern fieldsAndOrder , bool unique ){
+
+ ChunkManagerPtr DBConfig::shardCollection( const string& ns , ShardKeyPattern fieldsAndOrder , bool unique ) {
uassert( 8042 , "db doesn't have sharding enabled" , _shardingEnabled );
-
+
scoped_lock lk( _lock );
CollectionInfo& ci = _collections[ns];
@@ -131,35 +133,48 @@ namespace mongo {
log() << "enable sharding on: " << ns << " with shard key: " << fieldsAndOrder << endl;
- ci.shard( this , ns , fieldsAndOrder , unique );
- ci.getCM()->maybeChunkCollection();
-
+ // From this point on, 'ns' is going to be treated as a sharded collection. We assume this is the first
+ // time it is seen by the sharded system and thus create the first chunk for the collection. All the remaining
+ // chunks will be created as a by-product of splitting.
+ ci.shard( ns , fieldsAndOrder , unique );
+ ChunkManagerPtr cm = ci.getCM();
+ uassert( 13449 , "collections already sharded" , (cm->numChunks() == 0) );
+ cm->createFirstChunk( getPrimary() );
_save();
- return ci.getCM();
+
+ try {
+ cm->maybeChunkCollection();
+ }
+ catch ( UserException& e ) {
+ // failure to chunk is not critical enough to abort the command (and undo the _save()'d configDB state)
+ log() << "couldn't chunk recently created collection: " << ns << " " << e << endl;
+ }
+
+ return cm;
}
- bool DBConfig::removeSharding( const string& ns ){
- if ( ! _shardingEnabled ){
+ bool DBConfig::removeSharding( const string& ns ) {
+ if ( ! _shardingEnabled ) {
return false;
}
-
+
scoped_lock lk( _lock );
-
+
Collections::iterator i = _collections.find( ns );
if ( i == _collections.end() )
return false;
-
+
CollectionInfo& ci = _collections[ns];
if ( ! ci.isSharded() )
return false;
-
+
ci.unshard();
_save();
return true;
}
-
- ChunkManagerPtr DBConfig::getChunkManager( const string& ns , bool shouldReload ){
+
+ ChunkManagerPtr DBConfig::getChunkManager( const string& ns , bool shouldReload ) {
scoped_lock lk( _lock );
if ( shouldReload )
@@ -170,93 +185,80 @@ namespace mongo {
return ci.getCM();
}
- void DBConfig::setPrimary( string s ){
+ void DBConfig::setPrimary( string s ) {
scoped_lock lk( _lock );
_primary.reset( s );
_save();
}
-
- void DBConfig::serialize(BSONObjBuilder& to){
+
+ void DBConfig::serialize(BSONObjBuilder& to) {
to.append("_id", _name);
to.appendBool("partitioned", _shardingEnabled );
to.append("primary", _primary.getName() );
}
-
- bool DBConfig::unserialize(const BSONObj& from){
+
+ void DBConfig::unserialize(const BSONObj& from) {
log(1) << "DBConfig unserialize: " << _name << " " << from << endl;
assert( _name == from["_id"].String() );
_shardingEnabled = from.getBoolField("partitioned");
_primary.reset( from.getStringField("primary") );
- // this is a temporary migration thing
+ // In the 1.5.x series, we used to have collection metadata nested in the database entry. The 1.6.x series
+ // had migration code that ported that info to where it belongs now: the 'collections' collection. We now
+ // just assert that we're not migrating from a 1.5.x directly into a 1.7.x without first converting.
BSONObj sharded = from.getObjectField( "sharded" );
- if ( sharded.isEmpty() )
- return false;
-
- BSONObjIterator i(sharded);
- while ( i.more() ){
- BSONElement e = i.next();
- uassert( 10182 , "sharded things have to be objects" , e.type() == Object );
-
- BSONObj c = e.embeddedObject();
- uassert( 10183 , "key has to be an object" , c["key"].type() == Object );
-
- _collections[e.fieldName()].shard( this , e.fieldName() , c["key"].Obj() , c["unique"].trueValue() );
- }
- return true;
+ if ( ! sharded.isEmpty() )
+ uasserted( 13509 , "can't migrate from 1.5.x release to the current one; need to upgrade to 1.6.x first");
}
- bool DBConfig::load(){
+ bool DBConfig::load() {
scoped_lock lk( _lock );
return _load();
}
- bool DBConfig::_load(){
+ bool DBConfig::_load() {
ScopedDbConnection conn( configServer.modelServer() );
-
- BSONObj o = conn->findOne( ShardNS::database , BSON( "_id" << _name ) );
+ BSONObj o = conn->findOne( ShardNS::database , BSON( "_id" << _name ) );
- if ( o.isEmpty() ){
+ if ( o.isEmpty() ) {
conn.done();
return false;
}
-
- if ( unserialize( o ) )
- _save();
-
+
+ unserialize( o );
+
BSONObjBuilder b;
b.appendRegex( "_id" , (string)"^" + _name + "." );
-
auto_ptr<DBClientCursor> cursor = conn->query( ShardNS::collection ,b.obj() );
assert( cursor.get() );
- while ( cursor->more() ){
+ while ( cursor->more() ) {
BSONObj o = cursor->next();
- _collections[o["_id"].String()] = CollectionInfo( this , o );
+ _collections[o["_id"].String()] = CollectionInfo( o );
}
-
- conn.done();
+
+ conn.done();
return true;
}
- void DBConfig::_save(){
+ void DBConfig::_save() {
ScopedDbConnection conn( configServer.modelServer() );
-
+
BSONObj n;
{
BSONObjBuilder b;
serialize(b);
n = b.obj();
}
-
+
conn->update( ShardNS::database , BSON( "_id" << _name ) , n , true );
string err = conn->getLastError();
uassert( 13396 , (string)"DBConfig save failed: " + err , err.size() == 0 );
-
- for ( Collections::iterator i=_collections.begin(); i!=_collections.end(); ++i ){
+
+ for ( Collections::iterator i=_collections.begin(); i!=_collections.end(); ++i ) {
if ( ! i->second.isDirty() )
continue;
i->second.save( i->first , conn.get() );
@@ -265,18 +267,17 @@ namespace mongo {
conn.done();
}
-
- bool DBConfig::reload(){
+ bool DBConfig::reload() {
scoped_lock lk( _lock );
return _reload();
}
-
- bool DBConfig::_reload(){
+
+ bool DBConfig::_reload() {
// TODO: i don't think is 100% correct
return _load();
}
-
- bool DBConfig::dropDatabase( string& errmsg ){
+
+ bool DBConfig::dropDatabase( string& errmsg ) {
/**
* 1) make sure everything is up
* 2) update config server
@@ -287,81 +288,88 @@ namespace mongo {
log() << "DBConfig::dropDatabase: " << _name << endl;
configServer.logChange( "dropDatabase.start" , _name , BSONObj() );
-
+
// 1
- if ( ! configServer.allUp( errmsg ) ){
+ if ( ! configServer.allUp( errmsg ) ) {
log(1) << "\t DBConfig::dropDatabase not all up" << endl;
return 0;
}
-
+
// 2
grid.removeDB( _name );
{
ScopedDbConnection conn( configServer.modelServer() );
conn->remove( ShardNS::database , BSON( "_id" << _name ) );
+ errmsg = conn->getLastError();
+ if ( ! errmsg.empty() ) {
+ log() << "could not drop '" << _name << "': " << errmsg << endl;
+ conn.done();
+ return false;
+ }
+
conn.done();
}
- if ( ! configServer.allUp( errmsg ) ){
+ if ( ! configServer.allUp( errmsg ) ) {
log() << "error removing from config server even after checking!" << endl;
return 0;
}
log(1) << "\t removed entry from config server for: " << _name << endl;
-
+
set<Shard> allServers;
// 3
- while ( true ){
- int num;
+ while ( true ) {
+ int num = 0;
if ( ! _dropShardedCollections( num , allServers , errmsg ) )
return 0;
log() << " DBConfig::dropDatabase: " << _name << " dropped sharded collections: " << num << endl;
if ( num == 0 )
break;
}
-
+
// 4
{
ScopedDbConnection conn( _primary );
BSONObj res;
- if ( ! conn->dropDatabase( _name , &res ) ){
+ if ( ! conn->dropDatabase( _name , &res ) ) {
errmsg = res.toString();
return 0;
}
conn.done();
}
-
+
// 5
- for ( set<Shard>::iterator i=allServers.begin(); i!=allServers.end(); i++ ){
+ for ( set<Shard>::iterator i=allServers.begin(); i!=allServers.end(); i++ ) {
ScopedDbConnection conn( *i );
BSONObj res;
- if ( ! conn->dropDatabase( _name , &res ) ){
+ if ( ! conn->dropDatabase( _name , &res ) ) {
errmsg = res.toString();
return 0;
}
- conn.done();
+ conn.done();
}
-
+
log(1) << "\t dropped primary db for: " << _name << endl;
configServer.logChange( "dropDatabase" , _name , BSONObj() );
return true;
}
- bool DBConfig::_dropShardedCollections( int& num, set<Shard>& allServers , string& errmsg ){
+ bool DBConfig::_dropShardedCollections( int& num, set<Shard>& allServers , string& errmsg ) {
num = 0;
set<string> seen;
- while ( true ){
+ while ( true ) {
Collections::iterator i = _collections.begin();
- for ( ; i != _collections.end(); ++i ){
+ for ( ; i != _collections.end(); ++i ) {
if ( i->second.isSharded() )
break;
}
-
+
if ( i == _collections.end() )
break;
- if ( seen.count( i->first ) ){
+ if ( seen.count( i->first ) ) {
errmsg = "seen a collection twice!";
return false;
}
@@ -371,19 +379,20 @@ namespace mongo {
i->second.getCM()->getAllShards( allServers );
i->second.getCM()->drop( i->second.getCM() );
-
+ uassert( 10176 , str::stream() << "shard state missing for " << i->first , removeSharding( i->first ) );
+
num++;
uassert( 10184 , "_dropShardedCollections too many collections - bailing" , num < 100000 );
log(2) << "\t\t dropped " << num << " so far" << endl;
}
-
+
return true;
}
-
- void DBConfig::getAllShards(set<Shard>& shards) const{
+
+ void DBConfig::getAllShards(set<Shard>& shards) const {
shards.insert(getPrimary());
- for (Collections::const_iterator it(_collections.begin()), end(_collections.end()); it != end; ++it){
- if (it->second.isSharded()){
+ for (Collections::const_iterator it(_collections.begin()), end(_collections.end()); it != end; ++it) {
+ if (it->second.isSharded()) {
it->second.getCM()->getAllShards(shards);
} // TODO: handle collections on non-primary shard
}
@@ -391,20 +400,20 @@ namespace mongo {
/* --- ConfigServer ---- */
- ConfigServer::ConfigServer() : DBConfig( "config" ){
+ ConfigServer::ConfigServer() : DBConfig( "config" ) {
_shardingEnabled = false;
}
-
+
ConfigServer::~ConfigServer() {
}
- bool ConfigServer::init( string s ){
+ bool ConfigServer::init( string s ) {
vector<string> configdbs;
splitStringDelim( s, &configdbs, ',' );
return init( configdbs );
}
- bool ConfigServer::init( vector<string> configHosts ){
+ bool ConfigServer::init( vector<string> configHosts ) {
uassert( 10187 , "need configdbs" , configHosts.size() );
string hn = getHostName();
@@ -412,19 +421,19 @@ namespace mongo {
sleepsecs(5);
dbexit( EXIT_BADOPTIONS );
}
-
+
set<string> hosts;
- for ( size_t i=0; i<configHosts.size(); i++ ){
+ for ( size_t i=0; i<configHosts.size(); i++ ) {
string host = configHosts[i];
hosts.insert( getHost( host , false ) );
configHosts[i] = getHost( host , true );
}
-
- for ( set<string>::iterator i=hosts.begin(); i!=hosts.end(); i++ ){
+
+ for ( set<string>::iterator i=hosts.begin(); i!=hosts.end(); i++ ) {
string host = *i;
bool ok = false;
- for ( int x=10; x>0; x-- ){
- if ( ! hostbyname( host.c_str() ).empty() ){
+ for ( int x=10; x>0; x-- ) {
+ if ( ! hostbyname( host.c_str() ).empty() ) {
ok = true;
break;
}
@@ -436,10 +445,10 @@ namespace mongo {
}
_config = configHosts;
-
+
string fullString;
joinStringDelim( configHosts, &fullString, ',' );
- _primary.setAddress( fullString , true );
+ _primary.setAddress( ConnectionString( fullString , ConnectionString::SYNC ) );
log(1) << " config string : " << fullString << endl;
return true;
@@ -448,14 +457,14 @@ namespace mongo {
bool ConfigServer::checkConfigServersConsistent( string& errmsg , int tries ) const {
if ( _config.size() == 1 )
return true;
-
+
if ( tries <= 0 )
return false;
-
+
unsigned firstGood = 0;
int up = 0;
vector<BSONObj> res;
- for ( unsigned i=0; i<_config.size(); i++ ){
+ for ( unsigned i=0; i<_config.size(); i++ ) {
BSONObj x;
try {
ScopedDbConnection conn( _config[i] );
@@ -469,125 +478,125 @@ namespace mongo {
}
conn.done();
}
- catch ( std::exception& ){
- log(LL_WARNING) << " couldn't check on config server:" << _config[i] << " ok for now" << endl;
+ catch ( SocketException& e ) {
+ warning() << " couldn't check on config server:" << _config[i] << " ok for now : " << e.toString() << endl;
}
res.push_back(x);
}
- if ( up == 0 ){
+ if ( up == 0 ) {
errmsg = "no config servers reachable";
return false;
}
- if ( up == 1 ){
+ if ( up == 1 ) {
log( LL_WARNING ) << "only 1 config server reachable, continuing" << endl;
return true;
}
BSONObj base = res[firstGood];
- for ( unsigned i=firstGood+1; i<res.size(); i++ ){
+ for ( unsigned i=firstGood+1; i<res.size(); i++ ) {
if ( res[i].isEmpty() )
continue;
string c1 = base.getFieldDotted( "collections.chunks" );
string c2 = res[i].getFieldDotted( "collections.chunks" );
-
+
string d1 = base.getFieldDotted( "collections.databases" );
string d2 = res[i].getFieldDotted( "collections.databases" );
if ( c1 == c2 && d1 == d2 )
continue;
-
+
stringstream ss;
ss << "config servers " << _config[firstGood] << " and " << _config[i] << " differ";
log( LL_WARNING ) << ss.str();
- if ( tries <= 1 ){
+ if ( tries <= 1 ) {
ss << "\n" << c1 << "\t" << c2 << "\n" << d1 << "\t" << d2;
errmsg = ss.str();
return false;
}
-
+
return checkConfigServersConsistent( errmsg , tries - 1 );
}
-
+
return true;
}
- bool ConfigServer::ok( bool checkConsistency ){
+ bool ConfigServer::ok( bool checkConsistency ) {
if ( ! _primary.ok() )
return false;
-
- if ( checkConsistency ){
+
+ if ( checkConsistency ) {
string errmsg;
- if ( ! checkConfigServersConsistent( errmsg ) ){
+ if ( ! checkConfigServersConsistent( errmsg ) ) {
log( LL_ERROR ) << "config servers not in sync! " << errmsg << endl;
return false;
}
}
-
+
return true;
}
- bool ConfigServer::allUp(){
+ bool ConfigServer::allUp() {
string errmsg;
return allUp( errmsg );
}
-
- bool ConfigServer::allUp( string& errmsg ){
+
+ bool ConfigServer::allUp( string& errmsg ) {
try {
ScopedDbConnection conn( _primary );
conn->getLastError();
conn.done();
return true;
}
- catch ( DBException& ){
+ catch ( DBException& ) {
log() << "ConfigServer::allUp : " << _primary.toString() << " seems down!" << endl;
errmsg = _primary.toString() + " seems down";
return false;
}
-
+
}
-
- int ConfigServer::dbConfigVersion(){
+
+ int ConfigServer::dbConfigVersion() {
ScopedDbConnection conn( _primary );
int version = dbConfigVersion( conn.conn() );
conn.done();
return version;
}
-
- int ConfigServer::dbConfigVersion( DBClientBase& conn ){
+
+ int ConfigServer::dbConfigVersion( DBClientBase& conn ) {
auto_ptr<DBClientCursor> c = conn.query( "config.version" , BSONObj() );
int version = 0;
- if ( c->more() ){
+ if ( c->more() ) {
BSONObj o = c->next();
version = o["version"].numberInt();
uassert( 10189 , "should only have 1 thing in config.version" , ! c->more() );
}
else {
- if ( conn.count( ShardNS::shard ) || conn.count( ShardNS::database ) ){
+ if ( conn.count( ShardNS::shard ) || conn.count( ShardNS::database ) ) {
version = 1;
}
}
-
+
return version;
}
-
- void ConfigServer::reloadSettings(){
+
+ void ConfigServer::reloadSettings() {
set<string> got;
-
+
ScopedDbConnection conn( _primary );
auto_ptr<DBClientCursor> c = conn->query( ShardNS::settings , BSONObj() );
assert( c.get() );
- while ( c->more() ){
+ while ( c->more() ) {
BSONObj o = c->next();
string name = o["_id"].valuestrsafe();
got.insert( name );
- if ( name == "chunksize" ){
+ if ( name == "chunksize" ) {
log(1) << "MaxChunkSize: " << o["value"] << endl;
Chunk::MaxChunkSize = o["value"].numberInt() * 1024 * 1024;
}
- else if ( name == "balancer" ){
+ else if ( name == "balancer" ) {
// ones we ignore here
}
else {
@@ -595,12 +604,12 @@ namespace mongo {
}
}
- if ( ! got.count( "chunksize" ) ){
+ if ( ! got.count( "chunksize" ) ) {
conn->insert( ShardNS::settings , BSON( "_id" << "chunksize" <<
"value" << (Chunk::MaxChunkSize / ( 1024 * 1024 ) ) ) );
}
-
-
+
+
// indexes
try {
conn->ensureIndex( ShardNS::chunk , BSON( "ns" << 1 << "min" << 1 ) , true );
@@ -608,66 +617,86 @@ namespace mongo {
conn->ensureIndex( ShardNS::chunk , BSON( "ns" << 1 << "lastmod" << 1 ) , true );
conn->ensureIndex( ShardNS::shard , BSON( "host" << 1 ) , true );
}
- catch ( std::exception& e ){
+ catch ( std::exception& e ) {
log( LL_WARNING ) << "couldn't create indexes on config db: " << e.what() << endl;
}
conn.done();
}
- string ConfigServer::getHost( string name , bool withPort ){
- if ( name.find( ":" ) != string::npos ){
+ string ConfigServer::getHost( string name , bool withPort ) {
+ if ( name.find( ":" ) != string::npos ) {
if ( withPort )
return name;
return name.substr( 0 , name.find( ":" ) );
}
- if ( withPort ){
+ if ( withPort ) {
stringstream ss;
ss << name << ":" << CmdLine::ConfigServerPort;
return ss.str();
}
-
+
return name;
}
- void ConfigServer::logChange( const string& what , const string& ns , const BSONObj& detail ){
- assert( _primary.ok() );
+ /* must never throw */
+ void ConfigServer::logChange( const string& what , const string& ns , const BSONObj& detail ) {
+ string changeID;
- static bool createdCapped = false;
- static AtomicUInt num;
-
- ScopedDbConnection conn( _primary );
-
- if ( ! createdCapped ){
- try {
- conn->createCollection( "config.changelog" , 1024 * 1024 * 10 , true );
- }
- catch ( UserException& e ){
- log(1) << "couldn't create changelog (like race condition): " << e << endl;
- // don't care
+ try {
+ // get this entry's ID so we can use on the exception code path too
+ stringstream id;
+ static AtomicUInt num;
+ id << getHostNameCached() << "-" << terseCurrentTime() << "-" << num++;
+ changeID = id.str();
+
+ // send a copy of the message to the log in case it doesn't manage to reach config.changelog
+ Client* c = currentClient.get();
+ BSONObj msg = BSON( "_id" << changeID << "server" << getHostNameCached() << "clientAddr" << (c ? c->clientAddress(true) : "N/A")
+ << "time" << DATENOW << "what" << what << "ns" << ns << "details" << detail );
+ log() << "about to log metadata event: " << msg << endl;
+
+ assert( _primary.ok() );
+
+ ScopedDbConnection conn( _primary );
+
+ static bool createdCapped = false;
+ if ( ! createdCapped ) {
+ try {
+ conn->createCollection( "config.changelog" , 1024 * 1024 * 10 , true );
+ }
+ catch ( UserException& e ) {
+ log(1) << "couldn't create changelog (like race condition): " << e << endl;
+ // don't care
+ }
+ createdCapped = true;
}
- createdCapped = true;
+
+ conn->insert( "config.changelog" , msg );
+
+ conn.done();
+
}
-
- stringstream id;
- id << getHostNameCached() << "-" << terseCurrentTime() << "-" << num++;
- BSONObj msg = BSON( "_id" << id.str() << "server" << getHostNameCached() << "time" << DATENOW <<
- "what" << what << "ns" << ns << "details" << detail );
- log() << "config change: " << msg << endl;
+ catch ( std::exception& e ) {
+ // if we got here, it means the config change is only in the log; it didn't make it to config.changelog
+ log() << "not logging config change: " << changeID << " " << e.what() << endl;
+ }
+ }
+ void ConfigServer::replicaSetChange( const ReplicaSetMonitor * monitor ) {
try {
- conn->insert( "config.changelog" , msg );
+ ScopedDbConnection conn( configServer.getConnectionString() );
+ conn->update( ShardNS::shard , BSON( "_id" << monitor->getName() ) , BSON( "$set" << BSON( "host" << monitor->getServerAddress() ) ) );
+ conn.done();
}
- catch ( std::exception& e ){
- log() << "not logging config change: " << e.what() << endl;
+ catch ( DBException & ) {
+ error() << "RSChangeWatcher: could not update config db for set: " << monitor->getName() << " to: " << monitor->getServerAddress() << endl;
}
-
- conn.done();
}
- DBConfigPtr configServerPtr (new ConfigServer());
- ConfigServer& configServer = dynamic_cast<ConfigServer&>(*configServerPtr);
+ DBConfigPtr configServerPtr (new ConfigServer());
+ ConfigServer& configServer = dynamic_cast<ConfigServer&>(*configServerPtr);
-}
+}
diff --git a/s/config.h b/s/config.h
index 5bff03f..0636835 100644
--- a/s/config.h
+++ b/s/config.h
@@ -26,14 +26,16 @@
#include "../db/namespace.h"
#include "../client/dbclient.h"
#include "../client/model.h"
-#include "shardkey.h"
+
+#include "chunk.h"
#include "shard.h"
+#include "shardkey.h"
namespace mongo {
struct ShardNS {
static string shard;
-
+
static string database;
static string collection;
static string chunk;
@@ -46,11 +48,10 @@ namespace mongo {
* Field names used in the 'shards' collection.
*/
struct ShardFields {
- static BSONField<bool> draining;
- static BSONField<long long> maxSize;
- static BSONField<long long> currSize;
+ static BSONField<bool> draining; // is it draining chunks?
+ static BSONField<long long> maxSize; // max allowed disk space usage
};
-
+
class ConfigServer;
class DBConfig;
@@ -59,93 +60,95 @@ namespace mongo {
extern DBConfigPtr configServerPtr;
extern ConfigServer& configServer;
- class ChunkManager;
- typedef shared_ptr<ChunkManager> ChunkManagerPtr;
-
/**
* top level configuration for a database
*/
class DBConfig {
struct CollectionInfo {
- CollectionInfo(){
+ CollectionInfo() {
_dirty = false;
_dropped = false;
}
-
- CollectionInfo( DBConfig * db , const BSONObj& in );
-
+
+ CollectionInfo( const BSONObj& in );
+
bool isSharded() const {
return _cm.get();
}
-
+
ChunkManagerPtr getCM() const {
return _cm;
}
- void shard( DBConfig * db , const string& ns , const ShardKeyPattern& key , bool unique );
+ void shard( const string& ns , const ShardKeyPattern& key , bool unique );
void unshard();
bool isDirty() const { return _dirty; }
bool wasDropped() const { return _dropped; }
-
+
void save( const string& ns , DBClientBase* conn );
-
+
private:
ChunkManagerPtr _cm;
bool _dirty;
bool _dropped;
};
-
+
typedef map<string,CollectionInfo> Collections;
-
+
public:
- DBConfig( string name )
- : _name( name ) ,
- _primary("config","") ,
- _shardingEnabled(false),
- _lock("DBConfig"){
+ DBConfig( string name )
+ : _name( name ) ,
+ _primary("config","") ,
+ _shardingEnabled(false),
+ _lock("DBConfig") {
assert( name.size() );
}
- virtual ~DBConfig(){}
-
- string getName(){ return _name; };
+ virtual ~DBConfig() {}
+
+ string getName() { return _name; };
/**
* @return if anything in this db is partitioned or not
*/
- bool isShardingEnabled(){
+ bool isShardingEnabled() {
return _shardingEnabled;
}
-
+
void enableSharding();
ChunkManagerPtr shardCollection( const string& ns , ShardKeyPattern fieldsAndOrder , bool unique );
-
+
+ /**
+ @return true if there was sharding info to remove
+ */
+ bool removeSharding( const string& ns );
+
/**
* @return whether or not the 'ns' collection is partitioned
*/
bool isSharded( const string& ns );
-
+
ChunkManagerPtr getChunkManager( const string& ns , bool reload = false );
-
+
/**
* @return the correct for shard for the ns
* if the namespace is sharded, will return NULL
*/
const Shard& getShard( const string& ns );
-
+
const Shard& getPrimary() const {
uassert( 8041 , (string)"no primary shard configured for db: " + _name , _primary.ok() );
return _primary;
}
-
+
void setPrimary( string s );
bool load();
bool reload();
-
+
bool dropDatabase( string& errmsg );
// model stuff
@@ -153,16 +156,13 @@ namespace mongo {
// lockless loading
void serialize(BSONObjBuilder& to);
- /**
- * if i need save in new format
- */
- bool unserialize(const BSONObj& from);
+ void unserialize(const BSONObj& from);
void getAllShards(set<Shard>& shards) const;
protected:
- /**
+ /**
lockless
*/
bool _isSharded( const string& ns );
@@ -173,24 +173,16 @@ namespace mongo {
bool _reload();
void _save();
-
- /**
- @return true if there was sharding info to remove
- */
- bool removeSharding( const string& ns );
-
string _name; // e.g. "alleyinsider"
Shard _primary; // e.g. localhost , mongo.foo.com:9999
bool _shardingEnabled;
-
+
//map<string,CollectionInfo> _sharded; // { "alleyinsider.blog.posts" : { ts : 1 } , ... ] - all ns that are sharded
//map<string,ChunkManagerPtr> _shards; // this will only have entries for things that have been looked at
Collections _collections;
mongo::mutex _lock; // TODO: change to r/w lock ??
-
- friend class ChunkManager;
};
class ConfigServer : public DBConfig {
@@ -198,38 +190,42 @@ namespace mongo {
ConfigServer();
~ConfigServer();
-
+
bool ok( bool checkConsistency = false );
-
- virtual string modelServer(){
+
+ virtual string modelServer() {
uassert( 10190 , "ConfigServer not setup" , _primary.ok() );
return _primary.getConnString();
}
-
+
/**
- call at startup, this will initiate connection to the grid db
+ call at startup, this will initiate connection to the grid db
*/
bool init( vector<string> configHosts );
-
+
bool init( string s );
bool allUp();
bool allUp( string& errmsg );
-
+
int dbConfigVersion();
int dbConfigVersion( DBClientBase& conn );
-
+
void reloadSettings();
/**
* @return 0 = ok, otherwise error #
*/
int checkConfigVersion( bool upgrade );
-
+
/**
- * log a change to config.changes
+ * Create a metadata change log entry in the config.changelog collection.
+ *
* @param what e.g. "split" , "migrate"
- * @param msg any more info
+ * @param ns to which collection the metadata change is being applied
+ * @param msg additional info about the metadata change
+ *
+ * This call is guaranteed never to throw.
*/
void logChange( const string& what , const string& ns , const BSONObj& detail = BSONObj() );
@@ -237,8 +233,10 @@ namespace mongo {
return ConnectionString( _primary.getConnString() , ConnectionString::SYNC );
}
+ void replicaSetChange( const ReplicaSetMonitor * monitor );
+
static int VERSION;
-
+
/**
* check to see if all config servers have the same state
diff --git a/s/config_migrate.cpp b/s/config_migrate.cpp
index 1a42144..57890a0 100644
--- a/s/config_migrate.cpp
+++ b/s/config_migrate.cpp
@@ -30,12 +30,12 @@
namespace mongo {
- int ConfigServer::checkConfigVersion( bool upgrade ){
+ int ConfigServer::checkConfigVersion( bool upgrade ) {
int cur = dbConfigVersion();
if ( cur == VERSION )
return 0;
-
- if ( cur == 0 ){
+
+ if ( cur == 0 ) {
ScopedDbConnection conn( _primary );
conn->insert( "config.version" , BSON( "_id" << 1 << "version" << VERSION ) );
pool.flush();
@@ -43,20 +43,20 @@ namespace mongo {
conn.done();
return 0;
}
-
- if ( cur == 2 ){
+
+ if ( cur == 2 ) {
// need to upgrade
assert( VERSION == 3 );
- if ( ! upgrade ){
+ if ( ! upgrade ) {
log() << "newer version of mongo meta data\n"
<< "need to --upgrade after shutting all mongos down"
<< endl;
return -9;
}
-
+
ScopedDbConnection conn( _primary );
-
+
// do a backup
string backupName;
{
@@ -67,20 +67,20 @@ namespace mongo {
log() << "backing up config to: " << backupName << endl;
conn->copyDatabase( "config" , backupName );
- map<string,string> hostToShard;
+ map<string,string> hostToShard;
set<string> shards;
// shards
{
unsigned n = 0;
auto_ptr<DBClientCursor> c = conn->query( ShardNS::shard , BSONObj() );
- while ( c->more() ){
+ while ( c->more() ) {
BSONObj o = c->next();
string host = o["host"].String();
string name = "";
-
+
BSONElement id = o["_id"];
- if ( id.type() == String ){
+ if ( id.type() == String ) {
name = id.String();
}
else {
@@ -88,18 +88,18 @@ namespace mongo {
ss << "shard" << hostToShard.size();
name = ss.str();
}
-
+
hostToShard[host] = name;
shards.insert( name );
n++;
}
-
+
assert( n == hostToShard.size() );
assert( n == shards.size() );
-
+
conn->remove( ShardNS::shard , BSONObj() );
-
- for ( map<string,string>::iterator i=hostToShard.begin(); i != hostToShard.end(); i++ ){
+
+ for ( map<string,string>::iterator i=hostToShard.begin(); i != hostToShard.end(); i++ ) {
conn->insert( ShardNS::shard , BSON( "_id" << i->second << "host" << i->first ) );
}
}
@@ -109,27 +109,27 @@ namespace mongo {
auto_ptr<DBClientCursor> c = conn->query( ShardNS::database , BSONObj() );
map<string,BSONObj> newDBs;
unsigned n = 0;
- while ( c->more() ){
+ while ( c->more() ) {
BSONObj old = c->next();
n++;
-
- if ( old["name"].eoo() ){
+
+ if ( old["name"].eoo() ) {
// already done
newDBs[old["_id"].String()] = old;
continue;
}
-
+
BSONObjBuilder b(old.objsize());
b.appendAs( old["name"] , "_id" );
-
+
BSONObjIterator i(old);
- while ( i.more() ){
+ while ( i.more() ) {
BSONElement e = i.next();
if ( strcmp( "_id" , e.fieldName() ) == 0 ||
- strcmp( "name" , e.fieldName() ) == 0 ){
+ strcmp( "name" , e.fieldName() ) == 0 ) {
continue;
}
-
+
b.append( e );
}
@@ -139,45 +139,45 @@ namespace mongo {
}
assert( n == newDBs.size() );
-
+
conn->remove( ShardNS::database , BSONObj() );
-
- for ( map<string,BSONObj>::iterator i=newDBs.begin(); i!=newDBs.end(); i++ ){
+
+ for ( map<string,BSONObj>::iterator i=newDBs.begin(); i!=newDBs.end(); i++ ) {
conn->insert( ShardNS::database , i->second );
}
-
+
}
-
+
// chunks
{
unsigned num = 0;
map<string,BSONObj> chunks;
auto_ptr<DBClientCursor> c = conn->query( ShardNS::chunk , BSONObj() );
- while ( c->more() ){
+ while ( c->more() ) {
BSONObj x = c->next();
BSONObjBuilder b;
string id = Chunk::genID( x["ns"].String() , x["min"].Obj() );
b.append( "_id" , id );
-
+
BSONObjIterator i(x);
- while ( i.more() ){
+ while ( i.more() ) {
BSONElement e = i.next();
if ( strcmp( e.fieldName() , "_id" ) == 0 )
continue;
b.append( e );
}
-
+
BSONObj n = b.obj();
log() << x << "\n\t" << n << endl;
chunks[id] = n;
num++;
}
-
+
assert( num == chunks.size() );
-
+
conn->remove( ShardNS::chunk , BSONObj() );
- for ( map<string,BSONObj>::iterator i=chunks.begin(); i!=chunks.end(); i++ ){
+ for ( map<string,BSONObj>::iterator i=chunks.begin(); i!=chunks.end(); i++ ) {
conn->insert( ShardNS::chunk , i->second );
}
@@ -188,7 +188,7 @@ namespace mongo {
pool.flush();
return 1;
}
-
+
log() << "don't know how to upgrade " << cur << " to " << VERSION << endl;
return -8;
}
diff --git a/s/cursors.cpp b/s/cursors.cpp
index 6dd7a20..cf2735b 100644
--- a/s/cursors.cpp
+++ b/s/cursors.cpp
@@ -21,90 +21,90 @@
#include "../client/connpool.h"
#include "../db/queryutil.h"
#include "../db/commands.h"
-#include "../util/background.h"
+#include "../util/concurrency/task.h"
namespace mongo {
-
+
// -------- ShardedCursor -----------
- ShardedClientCursor::ShardedClientCursor( QueryMessage& q , ClusteredCursor * cursor ){
+ ShardedClientCursor::ShardedClientCursor( QueryMessage& q , ClusteredCursor * cursor ) {
assert( cursor );
_cursor = cursor;
-
+
_skip = q.ntoskip;
_ntoreturn = q.ntoreturn;
-
+
_totalSent = 0;
_done = false;
_id = 0;
-
- if ( q.queryOptions & QueryOption_NoCursorTimeout ){
+
+ if ( q.queryOptions & QueryOption_NoCursorTimeout ) {
_lastAccessMillis = 0;
}
- else
+ else
_lastAccessMillis = Listener::getElapsedTimeMillis();
}
- ShardedClientCursor::~ShardedClientCursor(){
+ ShardedClientCursor::~ShardedClientCursor() {
assert( _cursor );
delete _cursor;
_cursor = 0;
}
- long long ShardedClientCursor::getId(){
- if ( _id <= 0 ){
+ long long ShardedClientCursor::getId() {
+ if ( _id <= 0 ) {
_id = cursorCache.genId();
assert( _id >= 0 );
}
return _id;
}
- void ShardedClientCursor::accessed(){
+ void ShardedClientCursor::accessed() {
if ( _lastAccessMillis > 0 )
_lastAccessMillis = Listener::getElapsedTimeMillis();
}
- long long ShardedClientCursor::idleTime( long long now ){
+ long long ShardedClientCursor::idleTime( long long now ) {
if ( _lastAccessMillis == 0 )
return 0;
return now - _lastAccessMillis;
}
- bool ShardedClientCursor::sendNextBatch( Request& r , int ntoreturn ){
+ bool ShardedClientCursor::sendNextBatch( Request& r , int ntoreturn ) {
uassert( 10191 , "cursor already done" , ! _done );
-
+
int maxSize = 1024 * 1024;
if ( _totalSent > 0 )
maxSize *= 3;
-
+
BufBuilder b(32768);
-
+
int num = 0;
bool sendMore = true;
- while ( _cursor->more() ){
+ while ( _cursor->more() ) {
BSONObj o = _cursor->next();
b.appendBuf( (void*)o.objdata() , o.objsize() );
num++;
-
- if ( b.len() > maxSize ){
+
+ if ( b.len() > maxSize ) {
break;
}
- if ( num == ntoreturn ){
+ if ( num == ntoreturn ) {
// soft limit aka batch size
break;
}
- if ( ntoreturn != 0 && ( -1 * num + _totalSent ) == ntoreturn ){
+ if ( ntoreturn != 0 && ( -1 * num + _totalSent ) == ntoreturn ) {
// hard limit - total to send
sendMore = false;
break;
}
- if ( ntoreturn == 0 && _totalSent == 0 && num > 100 ){
+ if ( ntoreturn == 0 && _totalSent == 0 && num > 100 ) {
// first batch should be max 100 unless batch size specified
break;
}
@@ -112,123 +112,141 @@ namespace mongo {
bool hasMore = sendMore && _cursor->more();
log(6) << "\t hasMore:" << hasMore << " wouldSendMoreIfHad: " << sendMore << " id:" << getId() << " totalSent: " << _totalSent << endl;
-
+
replyToQuery( 0 , r.p() , r.m() , b.buf() , b.len() , num , _totalSent , hasMore ? getId() : 0 );
_totalSent += num;
_done = ! hasMore;
-
+
return hasMore;
}
// ---- CursorCache -----
-
+
long long CursorCache::TIMEOUT = 600000;
CursorCache::CursorCache()
- :_mutex( "CursorCache" ), _shardedTotal(0){
+ :_mutex( "CursorCache" ), _shardedTotal(0) {
}
- CursorCache::~CursorCache(){
+ CursorCache::~CursorCache() {
// TODO: delete old cursors?
int logLevel = 1;
if ( _cursors.size() || _refs.size() )
logLevel = 0;
log( logLevel ) << " CursorCache at shutdown - "
- << " sharded: " << _cursors.size()
+ << " sharded: " << _cursors.size()
<< " passthrough: " << _refs.size()
<< endl;
}
- ShardedClientCursorPtr CursorCache::get( long long id ){
+ ShardedClientCursorPtr CursorCache::get( long long id ) const {
+ LOG(_myLogLevel) << "CursorCache::get id: " << id << endl;
scoped_lock lk( _mutex );
- MapSharded::iterator i = _cursors.find( id );
- if ( i == _cursors.end() ){
+ MapSharded::const_iterator i = _cursors.find( id );
+ if ( i == _cursors.end() ) {
OCCASIONALLY log() << "Sharded CursorCache missing cursor id: " << id << endl;
return ShardedClientCursorPtr();
}
i->second->accessed();
return i->second;
}
-
- void CursorCache::store( ShardedClientCursorPtr cursor ){
+
+ void CursorCache::store( ShardedClientCursorPtr cursor ) {
+ LOG(_myLogLevel) << "CursorCache::store cursor " << " id: " << cursor->getId() << endl;
assert( cursor->getId() );
scoped_lock lk( _mutex );
_cursors[cursor->getId()] = cursor;
_shardedTotal++;
}
- void CursorCache::remove( long long id ){
+ void CursorCache::remove( long long id ) {
assert( id );
scoped_lock lk( _mutex );
_cursors.erase( id );
}
-
- void CursorCache::storeRef( const string& server , long long id ){
+
+ void CursorCache::storeRef( const string& server , long long id ) {
+ LOG(_myLogLevel) << "CursorCache::storeRef server: " << server << " id: " << id << endl;
assert( id );
scoped_lock lk( _mutex );
_refs[id] = server;
}
-
- long long CursorCache::genId(){
- while ( true ){
+
+ string CursorCache::getRef( long long id ) const {
+ LOG(_myLogLevel) << "CursorCache::getRef id: " << id << endl;
+ assert( id );
+ scoped_lock lk( _mutex );
+ MapNormal::const_iterator i = _refs.find( id );
+ if ( i == _refs.end() )
+ return "";
+ return i->second;
+ }
+
+
+ long long CursorCache::genId() {
+ while ( true ) {
long long x = security.getNonce();
if ( x == 0 )
continue;
if ( x < 0 )
x *= -1;
-
+
scoped_lock lk( _mutex );
MapSharded::iterator i = _cursors.find( x );
if ( i != _cursors.end() )
continue;
-
+
MapNormal::iterator j = _refs.find( x );
if ( j != _refs.end() )
continue;
-
+
return x;
}
}
- void CursorCache::gotKillCursors(Message& m ){
+ void CursorCache::gotKillCursors(Message& m ) {
int *x = (int *) m.singleData()->_data;
x++; // reserved
int n = *x++;
- if ( n > 2000 ){
+ if ( n > 2000 ) {
log( n < 30000 ? LL_WARNING : LL_ERROR ) << "receivedKillCursors, n=" << n << endl;
}
uassert( 13286 , "sent 0 cursors to kill" , n >= 1 );
uassert( 13287 , "too many cursors to kill" , n < 30000 );
-
+
long long * cursors = (long long *)x;
- for ( int i=0; i<n; i++ ){
+ for ( int i=0; i<n; i++ ) {
long long id = cursors[i];
- if ( ! id ){
+ LOG(_myLogLevel) << "CursorCache::gotKillCursors id: " << id << endl;
+
+ if ( ! id ) {
log( LL_WARNING ) << " got cursor id of 0 to kill" << endl;
continue;
}
-
- string server;
+
+ string server;
{
scoped_lock lk( _mutex );
MapSharded::iterator i = _cursors.find( id );
- if ( i != _cursors.end() ){
+ if ( i != _cursors.end() ) {
_cursors.erase( i );
continue;
}
-
+
MapNormal::iterator j = _refs.find( id );
- if ( j == _refs.end() ){
+ if ( j == _refs.end() ) {
log( LL_WARNING ) << "can't find cursor: " << id << endl;
continue;
}
server = j->second;
_refs.erase( j );
}
-
+
+ LOG(_myLogLevel) << "CursorCache::found gotKillCursors id: " << id << " server: " << server << endl;
+
assert( server.size() );
ScopedDbConnection conn( server );
conn->killCursor( id );
@@ -236,7 +254,7 @@ namespace mongo {
}
}
- void CursorCache::appendInfo( BSONObjBuilder& result ){
+ void CursorCache::appendInfo( BSONObjBuilder& result ) const {
scoped_lock lk( _mutex );
result.append( "sharded" , (int)_cursors.size() );
result.appendNumber( "shardedEver" , _shardedTotal );
@@ -244,12 +262,12 @@ namespace mongo {
result.append( "totalOpen" , (int)(_cursors.size() + _refs.size() ) );
}
- void CursorCache::doTimeouts(){
+ void CursorCache::doTimeouts() {
long long now = Listener::getElapsedTimeMillis();
scoped_lock lk( _mutex );
- for ( MapSharded::iterator i=_cursors.begin(); i!=_cursors.end(); ++i ){
+ for ( MapSharded::iterator i=_cursors.begin(); i!=_cursors.end(); ++i ) {
long long idleFor = i->second->idleTime( now );
- if ( idleFor < TIMEOUT ){
+ if ( idleFor < TIMEOUT ) {
continue;
}
log() << "killing old cursor " << i->second->getId() << " idle for: " << idleFor << "ms" << endl; // TODO: make log(1)
@@ -258,18 +276,19 @@ namespace mongo {
}
CursorCache cursorCache;
-
- class CursorTimeoutThread : public PeriodicBackgroundJob {
+
+ int CursorCache::_myLogLevel = 3;
+
+ class CursorTimeoutTask : public task::Task {
public:
- CursorTimeoutThread() : PeriodicBackgroundJob( 4000 ){}
- virtual string name() { return "cursorTimeout"; }
- virtual void runLoop(){
+ virtual string name() const { return "cursorTimeout"; }
+ virtual void doWork() {
cursorCache.doTimeouts();
}
- } cursorTimeoutThread;
+ } cursorTimeoutTask;
- void CursorCache::startTimeoutThread(){
- cursorTimeoutThread.go();
+ void CursorCache::startTimeoutThread() {
+ task::repeat( &cursorTimeoutTask , 400 );
}
class CmdCursorInfo : public Command {
@@ -280,7 +299,7 @@ namespace mongo {
help << " example: { cursorInfo : 1 }";
}
virtual LockType locktype() const { return NONE; }
- bool run(const string&, BSONObj& jsobj, string& errmsg, BSONObjBuilder& result, bool fromRepl ){
+ bool run(const string&, BSONObj& jsobj, string& errmsg, BSONObjBuilder& result, bool fromRepl ) {
cursorCache.appendInfo( result );
if ( jsobj["setTimeout"].isNumber() )
CursorCache::TIMEOUT = jsobj["setTimeout"].numberLong();
diff --git a/s/cursors.h b/s/cursors.h
index 53c5b64..7b54af6 100644
--- a/s/cursors.h
+++ b/s/cursors.h
@@ -16,7 +16,7 @@
*/
-#pragma once
+#pragma once
#include "../pch.h"
@@ -35,21 +35,21 @@ namespace mongo {
virtual ~ShardedClientCursor();
long long getId();
-
+
/**
* @return whether there is more data left
*/
- bool sendNextBatch( Request& r ){ return sendNextBatch( r , _ntoreturn ); }
+ bool sendNextBatch( Request& r ) { return sendNextBatch( r , _ntoreturn ); }
bool sendNextBatch( Request& r , int ntoreturn );
-
+
void accessed();
/** @return idle time in ms */
long long idleTime( long long now );
protected:
-
+
ClusteredCursor * _cursor;
-
+
int _skip;
int _ntoreturn;
@@ -62,10 +62,10 @@ namespace mongo {
};
typedef boost::shared_ptr<ShardedClientCursor> ShardedClientCursorPtr;
-
+
class CursorCache {
public:
-
+
static long long TIMEOUT;
typedef map<long long,ShardedClientCursorPtr> MapSharded;
@@ -73,29 +73,34 @@ namespace mongo {
CursorCache();
~CursorCache();
-
- ShardedClientCursorPtr get( long long id );
+
+ ShardedClientCursorPtr get( long long id ) const;
void store( ShardedClientCursorPtr cursor );
void remove( long long id );
void storeRef( const string& server , long long id );
- void gotKillCursors(Message& m );
-
- void appendInfo( BSONObjBuilder& result );
+ /** @return the server for id or "" */
+ string getRef( long long id ) const ;
+ void gotKillCursors(Message& m );
+
+ void appendInfo( BSONObjBuilder& result ) const ;
+
long long genId();
void doTimeouts();
void startTimeoutThread();
private:
- mutex _mutex;
+ mutable mongo::mutex _mutex;
MapSharded _cursors;
MapNormal _refs;
-
+
long long _shardedTotal;
+
+ static int _myLogLevel;
};
-
+
extern CursorCache cursorCache;
}
diff --git a/s/d_chunk_manager.cpp b/s/d_chunk_manager.cpp
new file mode 100644
index 0000000..d4fea30
--- /dev/null
+++ b/s/d_chunk_manager.cpp
@@ -0,0 +1,328 @@
+// @file d_chunk_manager.cpp
+
+/**
+* Copyright (C) 2010 10gen Inc.
+*
+* This program is free software: you can redistribute it and/or modify
+* it under the terms of the GNU Affero General Public License, version 3,
+* as published by the Free Software Foundation.
+*
+* This program is distributed in the hope that it will be useful,
+* but WITHOUT ANY WARRANTY; without even the implied warranty of
+* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+* GNU Affero General Public License for more details.
+*
+* You should have received a copy of the GNU Affero General Public License
+* along with this program. If not, see <http://www.gnu.org/licenses/>.
+*/
+
+#include "pch.h"
+
+#include "../client/connpool.h"
+#include "../client/dbclientmockcursor.h"
+#include "../db/instance.h"
+
+#include "d_chunk_manager.h"
+
+namespace mongo {
+
+ ShardChunkManager::ShardChunkManager( const string& configServer , const string& ns , const string& shardName ) {
+
+ // have to get a connection to the config db
+ // special case if i'm the configdb since i'm locked and if i connect to myself
+ // its a deadlock
+ scoped_ptr<ScopedDbConnection> scoped;
+ scoped_ptr<DBDirectClient> direct;
+ DBClientBase * conn;
+ if ( configServer.empty() ) {
+ direct.reset( new DBDirectClient() );
+ conn = direct.get();
+ }
+ else {
+ scoped.reset( new ScopedDbConnection( configServer ) );
+ conn = scoped->get();
+ }
+
+ // get this collection's sharding key
+ BSONObj collectionDoc = conn->findOne( "config.collections", BSON( "_id" << ns ) );
+ uassert( 13539 , str::stream() << ns << " does not exist" , !collectionDoc.isEmpty() );
+ uassert( 13540 , str::stream() << ns << " collection config entry corrupted" , collectionDoc["dropped"].type() );
+ uassert( 13541 , str::stream() << ns << " dropped. Re-shard collection first." , !collectionDoc["dropped"].Bool() );
+ _fillCollectionKey( collectionDoc );
+
+ // query for all the chunks for 'ns' that live in this shard, sorting so we can efficiently bucket them
+ BSONObj q = BSON( "ns" << ns << "shard" << shardName );
+ auto_ptr<DBClientCursor> cursor = conn->query( "config.chunks" , Query(q).sort( "min" ) );
+ _fillChunks( cursor.get() );
+ _fillRanges();
+
+ if ( scoped.get() )
+ scoped->done();
+
+ if ( _chunksMap.empty() )
+ log() << "no chunk for collection " << ns << " on shard " << shardName << endl;
+ }
+
+ ShardChunkManager::ShardChunkManager( const BSONObj& collectionDoc , const BSONArray& chunksArr ) {
+ _fillCollectionKey( collectionDoc );
+
+ scoped_ptr<DBClientMockCursor> c ( new DBClientMockCursor( chunksArr ) );
+ _fillChunks( c.get() );
+ _fillRanges();
+ }
+
+ void ShardChunkManager::_fillCollectionKey( const BSONObj& collectionDoc ) {
+ BSONElement e = collectionDoc["key"];
+ uassert( 13542 , str::stream() << "collection doesn't have a key: " << collectionDoc , ! e.eoo() && e.isABSONObj() );
+
+ BSONObj keys = e.Obj().getOwned();
+ BSONObjBuilder b;
+ BSONForEach( key , keys ) {
+ b.append( key.fieldName() , 1 );
+ }
+ _key = b.obj();
+ }
+
+ void ShardChunkManager::_fillChunks( DBClientCursorInterface* cursor ) {
+ assert( cursor );
+
+ ShardChunkVersion version;
+ while ( cursor->more() ) {
+ BSONObj d = cursor->next();
+ _chunksMap.insert( make_pair( d["min"].Obj().getOwned() , d["max"].Obj().getOwned() ) );
+
+ ShardChunkVersion currVersion( d["lastmod"] );
+ if ( currVersion > version ) {
+ version = currVersion;
+ }
+ }
+ _version = version;
+ }
+
+ void ShardChunkManager::_fillRanges() {
+ if ( _chunksMap.empty() )
+ return;
+
+ // load the chunk information, coallesceing their ranges
+ // the version for this shard would be the highest version for any of the chunks
+ RangeMap::const_iterator it = _chunksMap.begin();
+ BSONObj min,max;
+ while ( it != _chunksMap.end() ) {
+ BSONObj currMin = it->first;
+ BSONObj currMax = it->second;
+ ++it;
+
+ // coallesce the chunk's bounds in ranges if they are adjacent chunks
+ if ( min.isEmpty() ) {
+ min = currMin;
+ max = currMax;
+ continue;
+ }
+ if ( max == currMin ) {
+ max = currMax;
+ continue;
+ }
+
+ _rangesMap.insert( make_pair( min , max ) );
+
+ min = currMin;
+ max = currMax;
+ }
+ assert( ! min.isEmpty() );
+
+ _rangesMap.insert( make_pair( min , max ) );
+ }
+
+ static bool contains( const BSONObj& min , const BSONObj& max , const BSONObj& point ) {
+ return point.woCompare( min ) >= 0 && point.woCompare( max ) < 0;
+ }
+
+ bool ShardChunkManager::belongsToMe( const BSONObj& obj ) const {
+ if ( _rangesMap.size() == 0 )
+ return false;
+
+ BSONObj x = obj.extractFields(_key);
+
+ RangeMap::const_iterator it = _rangesMap.upper_bound( x );
+ if ( it != _rangesMap.begin() )
+ it--;
+
+ bool good = contains( it->first , it->second , x );
+
+#if 0
+ if ( ! good ) {
+ log() << "bad: " << x << " " << it->first << " " << x.woCompare( it->first ) << " " << x.woCompare( it->second ) << endl;
+ for ( RangeMap::const_iterator i=_rangesMap.begin(); i!=_rangesMap.end(); ++i ) {
+ log() << "\t" << i->first << "\t" << i->second << "\t" << endl;
+ }
+ }
+#endif
+
+ return good;
+ }
+
+ bool ShardChunkManager::getNextChunk( const BSONObj& lookupKey, BSONObj* foundMin , BSONObj* foundMax ) const {
+ assert( foundMin );
+ assert( foundMax );
+ *foundMin = BSONObj();
+ *foundMax = BSONObj();
+
+ if ( _chunksMap.empty() ) {
+ return true;
+ }
+
+ RangeMap::const_iterator it;
+ if ( lookupKey.isEmpty() ) {
+ it = _chunksMap.begin();
+ *foundMin = it->first;
+ *foundMax = it->second;
+ return _chunksMap.size() == 1;
+ }
+
+ it = _chunksMap.upper_bound( lookupKey );
+ if ( it != _chunksMap.end() ) {
+ *foundMin = it->first;
+ *foundMax = it->second;
+ return false;
+ }
+
+ return true;
+ }
+
+ void ShardChunkManager::_assertChunkExists( const BSONObj& min , const BSONObj& max ) const {
+ RangeMap::const_iterator it = _chunksMap.find( min );
+ if ( it == _chunksMap.end() ) {
+ uasserted( 13586 , str::stream() << "couldn't find chunk " << min << "->" << max );
+ }
+
+ if ( it->second.woCompare( max ) != 0 ) {
+ ostringstream os;
+ os << "ranges differ, "
+ << "requested: " << min << " -> " << max << " "
+ << "existing: " << (it == _chunksMap.end()) ? "<empty>" : it->first.toString() + " -> " + it->second.toString();
+ uasserted( 13587 , os.str() );
+ }
+ }
+
+ ShardChunkManager* ShardChunkManager::cloneMinus( const BSONObj& min, const BSONObj& max, const ShardChunkVersion& version ) {
+
+ // check that we have the exact chunk that'll be subtracted
+ _assertChunkExists( min , max );
+
+ auto_ptr<ShardChunkManager> p( new ShardChunkManager );
+ p->_key = this->_key;
+
+ if ( _chunksMap.size() == 1 ) {
+ // if left with no chunks, just reset version
+ uassert( 13590 , str::stream() << "setting version to " << version << " on removing last chunk", version == 0 );
+
+ p->_version = 0;
+
+ }
+ else {
+ // can't move version backwards when subtracting chunks
+ // this is what guarantees that no read or write would be taken once we subtract data from the current shard
+ if ( version <= _version ) {
+ uasserted( 13585 , str::stream() << "version " << version.toString() << " not greater than " << _version.toString() );
+ }
+
+ p->_chunksMap = this->_chunksMap;
+ p->_chunksMap.erase( min );
+ p->_version = version;
+ p->_fillRanges();
+ }
+
+ return p.release();
+ }
+
+ static bool overlap( const BSONObj& l1 , const BSONObj& h1 , const BSONObj& l2 , const BSONObj& h2 ) {
+ return ! ( ( h1.woCompare( l2 ) <= 0 ) || ( h2.woCompare( l1 ) <= 0 ) );
+ }
+
+ ShardChunkManager* ShardChunkManager::clonePlus( const BSONObj& min , const BSONObj& max , const ShardChunkVersion& version ) {
+
+ // it is acceptable to move version backwards (e.g., undoing a migration that went bad during commit)
+ // but only cloning away the last chunk may reset the version to 0
+ uassert( 13591 , "version can't be set to zero" , version > 0 );
+
+ if ( ! _chunksMap.empty() ) {
+
+ // check that there isn't any chunk on the interval to be added
+ RangeMap::const_iterator it = _chunksMap.lower_bound( max );
+ if ( it != _chunksMap.begin() ) {
+ --it;
+ }
+ if ( overlap( min , max , it->first , it->second ) ) {
+ ostringstream os;
+ os << "ranges overlap, "
+ << "requested: " << min << " -> " << max << " "
+ << "existing: " << it->first.toString() + " -> " + it->second.toString();
+ uasserted( 13588 , os.str() );
+ }
+ }
+
+ auto_ptr<ShardChunkManager> p( new ShardChunkManager );
+
+ p->_key = this->_key;
+ p->_chunksMap = this->_chunksMap;
+ p->_chunksMap.insert( make_pair( min.getOwned() , max.getOwned() ) );
+ p->_version = version;
+ p->_fillRanges();
+
+ return p.release();
+ }
+
+ ShardChunkManager* ShardChunkManager::cloneSplit( const BSONObj& min , const BSONObj& max , const vector<BSONObj>& splitKeys ,
+ const ShardChunkVersion& version ) {
+
+ // the version required in both resulting chunks could be simply an increment in the minor portion of the current version
+ // however, we are enforcing uniqueness over the attributes <ns, lastmod> of the configdb collection 'chunks'
+ // so in practice, a migrate somewhere may force this split to pick up a version that has the major portion higher
+ // than the one that this shard has been using
+ //
+ // TODO drop the uniqueness constraint and tigthen the check below so that only the minor portion of version changes
+ if ( version <= _version ) {
+ uasserted( 13592 , str::stream() << "version " << version.toString() << " not greater than " << _version.toString() );
+ }
+
+ // check that we have the exact chunk that'll be split and that the split point is valid
+ _assertChunkExists( min , max );
+ for ( vector<BSONObj>::const_iterator it = splitKeys.begin() ; it != splitKeys.end() ; ++it ) {
+ if ( ! contains( min , max , *it ) ) {
+ uasserted( 13593 , str::stream() << "can split " << min << " -> " << max << " on " << *it );
+ }
+ }
+
+ auto_ptr<ShardChunkManager> p( new ShardChunkManager );
+
+ p->_key = this->_key;
+ p->_chunksMap = this->_chunksMap;
+ p->_version = version; // will increment second, third, ... chunks below
+
+ BSONObj startKey = min;
+ for ( vector<BSONObj>::const_iterator it = splitKeys.begin() ; it != splitKeys.end() ; ++it ) {
+ BSONObj split = *it;
+ p->_chunksMap[min] = split.getOwned();
+ p->_chunksMap.insert( make_pair( split.getOwned() , max.getOwned() ) );
+ p->_version.incMinor();
+ startKey = split;
+ }
+ p->_fillRanges();
+
+ return p.release();
+ }
+
+ string ShardChunkManager::toString() const {
+ StringBuilder ss;
+ ss << " ShardChunkManager version: " << _version << " key: " << _key;
+ bool first = true;
+ for ( RangeMap::const_iterator i=_rangesMap.begin(); i!=_rangesMap.end(); ++i ) {
+ if ( first ) first = false;
+ else ss << " , ";
+
+ ss << i->first << " -> " << i->second;
+ }
+ return ss.str();
+ }
+
+} // namespace mongo
diff --git a/s/d_chunk_manager.h b/s/d_chunk_manager.h
new file mode 100644
index 0000000..9fb95e7
--- /dev/null
+++ b/s/d_chunk_manager.h
@@ -0,0 +1,150 @@
+// @file d_chunk_manager.h
+
+/**
+* Copyright (C) 2008 10gen Inc.
+*
+* This program is free software: you can redistribute it and/or modify
+* it under the terms of the GNU Affero General Public License, version 3,
+* as published by the Free Software Foundation.
+*
+* This program is distributed in the hope that it will be useful,
+* but WITHOUT ANY WARRANTY; without even the implied warranty of
+* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+* GNU Affero General Public License for more details.
+*
+* You should have received a copy of the GNU Affero General Public License
+* along with this program. If not, see <http://www.gnu.org/licenses/>.
+*/
+
+#pragma once
+
+#include "../pch.h"
+
+#include "../db/jsobj.h"
+#include "util.h"
+
+namespace mongo {
+
+ /**
+ * Controls the boundaries of all the chunks for a given collection that live in this shard.
+ *
+ * ShardChunkManager instances never change after construction. There are methods provided that would generate a
+ * new manager if new chunks are added, subtracted, or split.
+ *
+ * TODO
+ * The responsibility of maintaining the version for a shard is still shared between this class and its caller. The
+ * manager does check corner cases (e.g. cloning out the last chunk generates a manager with version 0) but ultimately
+ * still cannot be responsible to set all versions. Currently, they are a function of the global state as opposed to
+ * the per-shard one.
+ */
+ class ShardChunkManager : public boost::noncopyable {
+ public:
+
+ /**
+ * Loads the ShardChunkManager with all boundaries for chunks of a given collection that live in an given
+ * shard.
+ *
+ * @param configServer name of the server where the configDB currently is. Can be empty to indicate
+ * that the configDB is running locally
+ * @param ns namespace for the collections whose chunks we're interested
+ * @param shardName name of the shard that this chunk matcher should track
+ *
+ * This constructor throws if collection is dropped/malformed and on connectivity errors
+ */
+ ShardChunkManager( const string& configServer , const string& ns , const string& shardName );
+
+ /**
+ * Same as the regular constructor but used in unittest (no access to configDB required).
+ *
+ * @param collectionDoc simulates config.collection's entry for one colleciton
+ * @param chunksDocs simulates config.chunks' entries for one collection's shard
+ */
+ ShardChunkManager( const BSONObj& collectionDoc , const BSONArray& chunksDoc );
+
+ ~ShardChunkManager() {}
+
+ /**
+ * Generates a new manager based on 'this's state minus a given chunk.
+ *
+ * @param min max chunk boundaries for the chunk to subtract
+ * @param version that the resulting manager should be at. The version has to be higher than the current one.
+ * When cloning away the last chunk, verstion must be 0.
+ * @return a new ShardChunkManager, to be owned by the caller
+ */
+ ShardChunkManager* cloneMinus( const BSONObj& min , const BSONObj& max , const ShardChunkVersion& version );
+
+ /**
+ * Generates a new manager based on 'this's state plus a given chunk.
+ *
+ * @param min max chunk boundaries for the chunk to add
+ * @param version that the resulting manager should be at. It can never be 0, though (see CloneMinus).
+ * @return a new ShardChunkManager, to be owned by the caller
+ */
+ ShardChunkManager* clonePlus( const BSONObj& min , const BSONObj& max , const ShardChunkVersion& version );
+
+ /**
+ * Generates a new manager by splitting an existing chunk at one or more points.
+ *
+ * @param min max boundaries of chunk to be split
+ * @param splitKeys points to split original chunk at
+ * @param version to be used in first chunk. The subsequent chunks would increment the minor version.
+ * @return a new ShardChunkManager with the chunk split, to be owned by the caller
+ */
+ ShardChunkManager* cloneSplit( const BSONObj& min , const BSONObj& max , const vector<BSONObj>& splitKeys ,
+ const ShardChunkVersion& version );
+
+ /**
+ * Checks whether a document belongs to this shard.
+ *
+ * @param obj document containing sharding keys (and, optionally, other attributes)
+ * @return true if shards hold the object
+ */
+ bool belongsToMe( const BSONObj& obj ) const;
+
+ /**
+ * Given a chunk's min key (or empty doc), gets the boundary of the chunk following that one (the first).
+ *
+ * @param lookupKey is the min key for a previously obtained chunk or the empty document
+ * @param foundMin IN/OUT min for chunk following the one starting at lookupKey
+ * @param foundMax IN/OUT max for the above chunk
+ * @return true if the chunk returned is the last one
+ */
+ bool getNextChunk( const BSONObj& lookupKey, BSONObj* foundMin , BSONObj* foundMax ) const;
+
+ // accessors
+
+ ShardChunkVersion getVersion() const { return _version; }
+ BSONObj getKey() const { return _key.getOwned(); }
+ unsigned getNumChunks() const { return _chunksMap.size(); }
+
+ string toString() const;
+ private:
+ // highest ShardChunkVersion for which this ShardChunkManager's information is accurate
+ ShardChunkVersion _version;
+
+ // key pattern for chunks under this range
+ BSONObj _key;
+
+ // a map from a min key into the chunk's (or range's) max boundary
+ typedef map< BSONObj, BSONObj , BSONObjCmp > RangeMap;
+ RangeMap _chunksMap;
+
+ // a map from a min key into a range or continguous chunks
+ // redundant but we expect high chunk continguity, expecially in small installations
+ RangeMap _rangesMap;
+
+ /** constructors helpers */
+ void _fillCollectionKey( const BSONObj& collectionDoc );
+ void _fillChunks( DBClientCursorInterface* cursor );
+ void _fillRanges();
+
+ /** throws if the exact chunk is not in the chunks' map */
+ void _assertChunkExists( const BSONObj& min , const BSONObj& max ) const;
+
+ /** can only be used in the cloning calls */
+ ShardChunkManager() {}
+ };
+
+ typedef shared_ptr<ShardChunkManager> ShardChunkManagerPtr;
+
+} // namespace mongo
diff --git a/s/d_logic.cpp b/s/d_logic.cpp
index 62288ed..c032883 100644
--- a/s/d_logic.cpp
+++ b/s/d_logic.cpp
@@ -1,4 +1,4 @@
-// d_logic.cpp
+// @file d_logic.cpp
/**
* Copyright (C) 2008 10gen Inc.
@@ -37,32 +37,32 @@
#include "shard.h"
#include "d_logic.h"
+#include "d_writeback.h"
using namespace std;
namespace mongo {
- bool handlePossibleShardedMessage( Message &m, DbResponse* dbresponse ){
- if ( ! shardingState.enabled() )
- return false;
+ bool _handlePossibleShardedMessage( Message &m, DbResponse* dbresponse ) {
+ DEV assert( shardingState.enabled() );
int op = m.operation();
- if ( op < 2000
- || op >= 3000
- || op == dbGetMore // cursors are weird
- )
+ if ( op < 2000
+ || op >= 3000
+ || op == dbGetMore // cursors are weird
+ )
return false;
-
- DbMessage d(m);
+
+ DbMessage d(m);
const char *ns = d.getns();
string errmsg;
- if ( shardVersionOk( ns , opIsWrite( op ) , errmsg ) ){
+ if ( shardVersionOk( ns , opIsWrite( op ) , errmsg ) ) {
return false;
}
log(1) << "connection meta data too old - will retry ns:(" << ns << ") op:(" << opToString(op) << ") " << errmsg << endl;
-
- if ( doesOpGetAResponse( op ) ){
+
+ if ( doesOpGetAResponse( op ) ) {
assert( dbresponse );
BufBuilder b( 32768 );
b.skip( sizeof( QueryResult ) );
@@ -70,7 +70,7 @@ namespace mongo {
BSONObj obj = BSON( "$err" << errmsg );
b.appendBuf( obj.objdata() , obj.objsize() );
}
-
+
QueryResult *qr = (QueryResult*)b.buf();
qr->_resultFlags() = ResultFlag_ErrSet | ResultFlag_ShardConfigStale;
qr->len = b.len();
@@ -82,19 +82,19 @@ namespace mongo {
Message * resp = new Message();
resp->setData( qr , true );
-
+
dbresponse->response = resp;
dbresponse->responseTo = m.header()->id;
return true;
}
-
+
OID writebackID;
writebackID.init();
lastError.getSafe()->writeback( writebackID );
const OID& clientID = ShardedConnectionInfo::get(false)->getID();
massert( 10422 , "write with bad shard config and no server id!" , clientID.isSet() );
-
+
log(1) << "got write with an old config - writing back ns: " << ns << endl;
if ( logLevel ) log(1) << debugString( m ) << endl;
@@ -102,11 +102,12 @@ namespace mongo {
b.appendBool( "writeBack" , true );
b.append( "ns" , ns );
b.append( "id" , writebackID );
+ b.append( "connectionId" , cc().getConnectionId() );
b.appendTimestamp( "version" , shardingState.getVersion( ns ) );
b.appendTimestamp( "yourVersion" , ShardedConnectionInfo::get( true )->getVersion( ns ) );
b.appendBinData( "msg" , m.header()->len , bdtCustom , (char*)(m.singleData()) );
log(2) << "writing back msg with len: " << m.header()->len << " op: " << m.operation() << endl;
- queueWriteBack( clientID.str() , b.obj() );
+ writeBackManager.queueWriteBack( clientID.str() , b.obj() );
return true;
}
diff --git a/s/d_logic.h b/s/d_logic.h
index a000f6b..718836c 100644
--- a/s/d_logic.h
+++ b/s/d_logic.h
@@ -1,4 +1,4 @@
-// d_logic.h
+// @file d_logic.h
/*
* Copyright (C) 2010 10gen Inc.
*
@@ -19,38 +19,20 @@
#pragma once
#include "../pch.h"
+
#include "../db/jsobj.h"
+
+#include "d_chunk_manager.h"
#include "util.h"
namespace mongo {
-
- class ShardingState;
-
- typedef ShardChunkVersion ConfigVersion;
- typedef map<string,ConfigVersion> NSVersionMap;
-
- // -----------
- class ChunkMatcher {
- typedef map<BSONObj,pair<BSONObj,BSONObj>,BSONObjCmp> MyMap;
- public:
-
- bool belongsToMe( const BSONObj& key , const DiskLoc& loc ) const;
+ class Database;
+ class DiskLoc;
- private:
- ChunkMatcher( ConfigVersion version );
-
- void gotRange( const BSONObj& min , const BSONObj& max );
-
- ConfigVersion _version;
- BSONObj _key;
- MyMap _map;
-
- friend class ShardingState;
- };
+ typedef ShardChunkVersion ConfigVersion;
+ typedef map<string,ConfigVersion> NSVersionMap;
- typedef shared_ptr<ChunkMatcher> ChunkMatcherPtr;
-
// --------------
// --- global state ---
// --------------
@@ -58,100 +40,182 @@ namespace mongo {
class ShardingState {
public:
ShardingState();
-
+
bool enabled() const { return _enabled; }
const string& getConfigServer() const { return _configServer; }
void enable( const string& server );
void gotShardName( const string& name );
- void gotShardHost( const string& host );
-
+ void gotShardHost( string host );
+
+ /** Reverts back to a state where this mongod is not sharded. */
+ void resetShardingState();
+
+ // versioning support
+
bool hasVersion( const string& ns );
bool hasVersion( const string& ns , ConfigVersion& version );
- ConfigVersion& getVersion( const string& ns ); // TODO: this is dangeroues
- void setVersion( const string& ns , const ConfigVersion& version );
-
+ const ConfigVersion getVersion( const string& ns ) const;
+
+ /**
+ * Uninstalls the manager for a given collection. This should be used when the collection is dropped.
+ *
+ * NOTE:
+ * An existing collection with no chunks on this shard will have a manager on version 0, which is different than a
+ * a dropped collection, which will not have a manager.
+ *
+ * TODO
+ * When sharding state is enabled, absolutely all collections should have a manager. (The non-sharded ones are
+ * a be degenerate case of one-chunk collections).
+ * For now, a dropped collection and an non-sharded one are indistinguishable (SERVER-1849)
+ *
+ * @param ns the collection to be dropped
+ */
+ void resetVersion( const string& ns );
+
+ /**
+ * Requests to access a collection at a certain version. If the collection's manager is not at that version it
+ * will try to update itself to the newest version. The request is only granted if the version is the current or
+ * the newest one.
+ *
+ * @param ns collection to be accessed
+ * @param version (IN) the client belive this collection is on and (OUT) the version the manager is actually in
+ * @return true if the access can be allowed at the provided version
+ */
+ bool trySetVersion( const string& ns , ConfigVersion& version );
+
void appendInfo( BSONObjBuilder& b );
-
- ChunkMatcherPtr getChunkMatcher( const string& ns );
-
+
+ // querying support
+
+ bool needShardChunkManager( const string& ns ) const;
+ ShardChunkManagerPtr getShardChunkManager( const string& ns );
+
+ // chunk migrate and split support
+
+ /**
+ * Creates and installs a new chunk manager for a given collection by "forgetting" about one of its chunks.
+ * The new manager uses the provided version, which has to be higher than the current manager's.
+ * One exception: if the forgotten chunk is the last one in this shard for the collection, version has to be 0.
+ *
+ * If it runs successfully, clients need to grab the new version to access the collection.
+ *
+ * @param ns the collection
+ * @param min max the chunk to eliminate from the current manager
+ * @param version at which the new manager should be at
+ */
+ void donateChunk( const string& ns , const BSONObj& min , const BSONObj& max , ShardChunkVersion version );
+
+ /**
+ * Creates and installs a new chunk manager for a given collection by reclaiming a previously donated chunk.
+ * The previous manager's version has to be provided.
+ *
+ * If it runs successfully, clients that became stale by the previous donateChunk will be able to access the
+ * collection again.
+ *
+ * @param ns the collection
+ * @param min max the chunk to reclaim and add to the current manager
+ * @param version at which the new manager should be at
+ */
+ void undoDonateChunk( const string& ns , const BSONObj& min , const BSONObj& max , ShardChunkVersion version );
+
+ /**
+ * Creates and installs a new chunk manager for a given collection by splitting one of its chunks in two or more.
+ * The version for the first split chunk should be provided. The subsequent chunks' version would be the latter with the
+ * minor portion incremented.
+ *
+ * The effect on clients will depend on the version used. If the major portion is the same as the current shards,
+ * clients shouldn't perceive the split.
+ *
+ * @param ns the collection
+ * @param min max the chunk that should be split
+ * @param splitKeys point in which to split
+ * @param version at which the new manager should be at
+ */
+ void splitChunk( const string& ns , const BSONObj& min , const BSONObj& max , const vector<BSONObj>& splitKeys ,
+ ShardChunkVersion version );
+
bool inCriticalMigrateSection();
+
private:
-
bool _enabled;
-
+
string _configServer;
-
+
string _shardName;
string _shardHost;
- mongo::mutex _mutex;
- NSVersionMap _versions;
- map<string,ChunkMatcherPtr> _chunks;
+ // protects state below
+ mutable mongo::mutex _mutex;
+
+ // map from a namespace into the ensemble of chunk ranges that are stored in this mongod
+ // a ShardChunkManager carries all state we need for a collection at this shard, including its version information
+ typedef map<string,ShardChunkManagerPtr> ChunkManagersMap;
+ ChunkManagersMap _chunks;
};
-
+
extern ShardingState shardingState;
- // --------------
- // --- per connection ---
- // --------------
-
+ /**
+ * one per connection from mongos
+ * holds version state for each namesapce
+ */
class ShardedConnectionInfo {
public:
ShardedConnectionInfo();
-
+
const OID& getID() const { return _id; }
bool hasID() const { return _id.isSet(); }
void setID( const OID& id );
-
- ConfigVersion& getVersion( const string& ns ); // TODO: this is dangeroues
+
+ const ConfigVersion getVersion( const string& ns ) const;
void setVersion( const string& ns , const ConfigVersion& version );
-
+
static ShardedConnectionInfo* get( bool create );
static void reset();
-
- bool inForceMode() const {
- return _forceMode;
+
+ bool inForceVersionOkMode() const {
+ return _forceVersionOk;
}
-
- void enterForceMode(){ _forceMode = true; }
- void leaveForceMode(){ _forceMode = false; }
+
+ void enterForceVersionOkMode() { _forceVersionOk = true; }
+ void leaveForceVersionOkMode() { _forceVersionOk = false; }
private:
-
+
OID _id;
NSVersionMap _versions;
- bool _forceMode;
+ bool _forceVersionOk; // if this is true, then chunk version #s aren't check, and all ops are allowed
static boost::thread_specific_ptr<ShardedConnectionInfo> _tl;
};
- struct ShardForceModeBlock {
- ShardForceModeBlock(){
+ struct ShardForceVersionOkModeBlock {
+ ShardForceVersionOkModeBlock() {
info = ShardedConnectionInfo::get( false );
if ( info )
- info->enterForceMode();
+ info->enterForceVersionOkMode();
}
- ~ShardForceModeBlock(){
+ ~ShardForceVersionOkModeBlock() {
if ( info )
- info->leaveForceMode();
+ info->leaveForceVersionOkMode();
}
ShardedConnectionInfo * info;
};
-
+
// -----------------
// --- core ---
// -----------------
unsigned long long extractVersion( BSONElement e , string& errmsg );
-
+
/**
* @return true if we have any shard info for the ns
*/
bool haveLocalShardingInfo( const string& ns );
-
+
/**
* @return true if the current threads shard version is ok, or not in sharded version
*/
@@ -160,15 +224,18 @@ namespace mongo {
/**
* @return true if we took care of the message and nothing else should be done
*/
- bool handlePossibleShardedMessage( Message &m, DbResponse * dbresponse );
+ struct DbResponse;
- void logOpForSharding( const char * opstr , const char * ns , const BSONObj& obj , BSONObj * patt );
+ bool _handlePossibleShardedMessage( Message &m, DbResponse * dbresponse );
- // -----------------
- // --- writeback ---
- // -----------------
+ /** What does this do? document please? */
+ inline bool handlePossibleShardedMessage( Message &m, DbResponse * dbresponse ) {
+ if( !shardingState.enabled() )
+ return false;
+ return _handlePossibleShardedMessage(m, dbresponse);
+ }
- /* queue a write back on a remote server for a failed write */
- void queueWriteBack( const string& remote , const BSONObj& o );
+ void logOpForSharding( const char * opstr , const char * ns , const BSONObj& obj , BSONObj * patt );
+ void aboutToDeleteForSharding( const Database* db , const DiskLoc& dl );
}
diff --git a/s/d_migrate.cpp b/s/d_migrate.cpp
index 8e9584c..2878276 100644
--- a/s/d_migrate.cpp
+++ b/s/d_migrate.cpp
@@ -25,18 +25,24 @@
#include "pch.h"
#include <map>
#include <string>
+#include <algorithm>
#include "../db/commands.h"
#include "../db/jsobj.h"
#include "../db/dbmessage.h"
#include "../db/query.h"
#include "../db/cmdline.h"
+#include "../db/queryoptimizer.h"
+#include "../db/btree.h"
+#include "../db/repl_block.h"
+#include "../db/dur.h"
#include "../client/connpool.h"
#include "../client/distlock.h"
#include "../util/queue.h"
#include "../util/unittest.h"
+#include "../util/processinfo.h"
#include "shard.h"
#include "d_logic.h"
@@ -49,131 +55,185 @@ namespace mongo {
class MoveTimingHelper {
public:
- MoveTimingHelper( const string& where , const string& ns , BSONObj min , BSONObj max )
- : _where( where ) , _ns( ns ){
- _next = 1;
+ MoveTimingHelper( const string& where , const string& ns , BSONObj min , BSONObj max , int total )
+ : _where( where ) , _ns( ns ) , _next( 0 ) , _total( total ) {
+ _nextNote = 0;
_b.append( "min" , min );
_b.append( "max" , max );
}
- ~MoveTimingHelper(){
- configServer.logChange( (string)"moveChunk." + _where , _ns, _b.obj() );
+ ~MoveTimingHelper() {
+ // even if logChange doesn't throw, bson does
+ // sigh
+ try {
+ if ( _next != _total ) {
+ note( "aborted" );
+ }
+ configServer.logChange( (string)"moveChunk." + _where , _ns, _b.obj() );
+ }
+ catch ( const std::exception& e ) {
+ log( LL_WARNING ) << "couldn't record timing for moveChunk '" << _where << "': " << e.what() << endl;
+ }
}
-
- void done( int step ){
- assert( step == _next++ );
-
+
+ void done( int step ) {
+ assert( step == ++_next );
+ assert( step <= _total );
+
stringstream ss;
ss << "step" << step;
string s = ss.str();
-
+
CurOp * op = cc().curop();
if ( op )
op->setMessage( s.c_str() );
- else
+ else
log( LL_WARNING ) << "op is null in MoveTimingHelper::done" << endl;
-
+
_b.appendNumber( s , _t.millis() );
_t.reset();
+
+#if 0
+ // debugging for memory leak?
+ ProcessInfo pi;
+ ss << " v:" << pi.getVirtualMemorySize()
+ << " r:" << pi.getResidentSize();
+ log() << ss.str() << endl;
+#endif
}
-
-
+
+
+ void note( const string& s ) {
+ string field = "note";
+ if ( _nextNote > 0 ) {
+ StringBuilder buf;
+ buf << "note" << _nextNote;
+ field = buf.str();
+ }
+ _nextNote++;
+
+ _b.append( field , s );
+ }
+
private:
Timer _t;
string _where;
string _ns;
-
+
int _next;
-
+ int _total; // expected # of steps
+ int _nextNote;
+
BSONObjBuilder _b;
+
};
struct OldDataCleanup {
+ static AtomicUInt _numThreads; // how many threads are doing async cleanusp
+
string ns;
BSONObj min;
BSONObj max;
set<CursorId> initial;
- void doRemove(){
- ShardForceModeBlock sf;
+
+ OldDataCleanup(){
+ _numThreads++;
+ }
+ OldDataCleanup( const OldDataCleanup& other ) {
+ ns = other.ns;
+ min = other.min.getOwned();
+ max = other.max.getOwned();
+ initial = other.initial;
+ _numThreads++;
+ }
+ ~OldDataCleanup(){
+ _numThreads--;
+ }
+
+ void doRemove() {
+ ShardForceVersionOkModeBlock sf;
writelock lk(ns);
RemoveSaver rs("moveChunk",ns,"post-cleanup");
long long num = Helpers::removeRange( ns , min , max , true , false , cmdLine.moveParanoia ? &rs : 0 );
log() << "moveChunk deleted: " << num << endl;
}
+
};
+ AtomicUInt OldDataCleanup::_numThreads = 0;
+
static const char * const cleanUpThreadName = "cleanupOldData";
-
- void _cleanupOldData( OldDataCleanup cleanup ){
+
+ void _cleanupOldData( OldDataCleanup cleanup ) {
Client::initThread( cleanUpThreadName );
log() << " (start) waiting to cleanup " << cleanup.ns << " from " << cleanup.min << " -> " << cleanup.max << " # cursors:" << cleanup.initial.size() << endl;
int loops = 0;
Timer t;
- while ( t.seconds() < 900 ){ // 15 minutes
+ while ( t.seconds() < 900 ) { // 15 minutes
assert( dbMutex.getState() == 0 );
sleepmillis( 20 );
-
+
set<CursorId> now;
- ClientCursor::find( cleanup.ns , now );
-
+ ClientCursor::find( cleanup.ns , now );
+
set<CursorId> left;
- for ( set<CursorId>::iterator i=cleanup.initial.begin(); i!=cleanup.initial.end(); ++i ){
+ for ( set<CursorId>::iterator i=cleanup.initial.begin(); i!=cleanup.initial.end(); ++i ) {
CursorId id = *i;
if ( now.count(id) )
left.insert( id );
}
-
+
if ( left.size() == 0 )
break;
cleanup.initial = left;
-
- if ( ( loops++ % 200 ) == 0 ){
+
+ if ( ( loops++ % 200 ) == 0 ) {
log() << " (looping " << loops << ") waiting to cleanup " << cleanup.ns << " from " << cleanup.min << " -> " << cleanup.max << " # cursors:" << cleanup.initial.size() << endl;
-
+
stringstream ss;
- for ( set<CursorId>::iterator i=cleanup.initial.begin(); i!=cleanup.initial.end(); ++i ){
+ for ( set<CursorId>::iterator i=cleanup.initial.begin(); i!=cleanup.initial.end(); ++i ) {
CursorId id = *i;
ss << id << " ";
}
log() << " cursors: " << ss.str() << endl;
}
}
-
+
cleanup.doRemove();
cc().shutdown();
}
- void cleanupOldData( OldDataCleanup cleanup ){
+ void cleanupOldData( OldDataCleanup cleanup ) {
try {
_cleanupOldData( cleanup );
}
- catch ( std::exception& e ){
+ catch ( std::exception& e ) {
log() << " error cleaning old data:" << e.what() << endl;
}
- catch ( ... ){
+ catch ( ... ) {
log() << " unknown error cleaning old data" << endl;
}
}
class ChunkCommandHelper : public Command {
public:
- ChunkCommandHelper( const char * name )
- : Command( name ){
+ ChunkCommandHelper( const char * name )
+ : Command( name ) {
}
-
+
virtual void help( stringstream& help ) const {
- help << "internal should not be calling this directly" << endl;
+ help << "internal - should not be called directly" << endl;
}
virtual bool slaveOk() const { return false; }
virtual bool adminOnly() const { return true; }
- virtual LockType locktype() const { return NONE; }
+ virtual LockType locktype() const { return NONE; }
};
- bool isInRange( const BSONObj& obj , const BSONObj& min , const BSONObj& max ){
+ bool isInRange( const BSONObj& obj , const BSONObj& min , const BSONObj& max ) {
BSONObj k = obj.extractFields( min, true );
return k.woCompare( min ) >= 0 && k.woCompare( max ) < 0;
@@ -182,48 +242,57 @@ namespace mongo {
class MigrateFromStatus {
public:
-
- MigrateFromStatus()
- : _mutex( "MigrateFromStatus" ){
+
+ MigrateFromStatus() : _m("MigrateFromStatus") {
_active = false;
_inCriticalSection = false;
+ _memoryUsed = 0;
}
- void start( string ns , const BSONObj& min , const BSONObj& max ){
+ void start( string ns , const BSONObj& min , const BSONObj& max ) {
+ scoped_lock l(_m); // reads and writes _active
+
assert( ! _active );
-
+
assert( ! min.isEmpty() );
assert( ! max.isEmpty() );
assert( ns.size() );
-
+
_ns = ns;
_min = min;
_max = max;
-
- _deleted.clear();
- _reload.clear();
-
+
+ assert( _cloneLocs.size() == 0 );
+ assert( _deleted.size() == 0 );
+ assert( _reload.size() == 0 );
+ assert( _memoryUsed == 0 );
+
_active = true;
}
-
- void done(){
- if ( ! _active )
- return;
- _active = false;
- _inCriticalSection = false;
- scoped_lock lk( _mutex );
+ void done() {
+ readlock lk( _ns );
+
_deleted.clear();
_reload.clear();
+ _cloneLocs.clear();
+ _memoryUsed = 0;
+
+ scoped_lock l(_m);
+ _active = false;
+ _inCriticalSection = false;
}
-
- void logOp( const char * opstr , const char * ns , const BSONObj& obj , BSONObj * patt ){
- if ( ! _active )
+
+ void logOp( const char * opstr , const char * ns , const BSONObj& obj , BSONObj * patt ) {
+ if ( ! _getActive() )
return;
if ( _ns != ns )
return;
-
+
+ // no need to log if this is not an insertion, an update, or an actual deletion
+ // note: opstr 'db' isn't a deletion but a mention that a database exists (for replication
+ // machinery mostly)
char op = opstr[0];
if ( op == 'n' || op =='c' || ( op == 'd' && opstr[1] == 'b' ) )
return;
@@ -231,68 +300,68 @@ namespace mongo {
BSONElement ide;
if ( patt )
ide = patt->getField( "_id" );
- else
+ else
ide = obj["_id"];
-
- if ( ide.eoo() ){
+
+ if ( ide.eoo() ) {
log( LL_WARNING ) << "logOpForSharding got mod with no _id, ignoring obj: " << obj << endl;
return;
}
-
+
BSONObj it;
- switch ( opstr[0] ){
-
+ switch ( opstr[0] ) {
+
case 'd': {
-
- if ( getThreadName() == cleanUpThreadName ){
+
+ if ( getThreadName() == cleanUpThreadName ) {
// we don't want to xfer things we're cleaning
// as then they'll be deleted on TO
// which is bad
return;
}
-
+
// can't filter deletes :(
- scoped_lock lk( _mutex );
_deleted.push_back( ide.wrap() );
+ _memoryUsed += ide.size() + 5;
return;
}
-
- case 'i':
+
+ case 'i':
it = obj;
break;
-
- case 'u':
- if ( ! Helpers::findById( cc() , _ns.c_str() , ide.wrap() , it ) ){
+
+ case 'u':
+ if ( ! Helpers::findById( cc() , _ns.c_str() , ide.wrap() , it ) ) {
log( LL_WARNING ) << "logOpForSharding couldn't find: " << ide << " even though should have" << endl;
return;
}
break;
-
+
}
-
+
if ( ! isInRange( it , _min , _max ) )
return;
-
- scoped_lock lk( _mutex );
+
_reload.push_back( ide.wrap() );
+ _memoryUsed += ide.size() + 5;
}
- void xfer( list<BSONObj> * l , BSONObjBuilder& b , const char * name , long long& size , bool explode ){
+ void xfer( list<BSONObj> * l , BSONObjBuilder& b , const char * name , long long& size , bool explode ) {
const long long maxSize = 1024 * 1024;
-
+
if ( l->size() == 0 || size > maxSize )
return;
-
+
BSONArrayBuilder arr(b.subarrayStart(name));
-
- list<BSONObj>::iterator i = l->begin();
-
- while ( i != l->end() && size < maxSize ){
+
+ list<BSONObj>::iterator i = l->begin();
+
+ while ( i != l->end() && size < maxSize ) {
BSONObj t = *i;
- if ( explode ){
+ if ( explode ) {
BSONObj it;
- if ( Helpers::findById( cc() , _ns.c_str() , t, it ) ){
+ if ( Helpers::findById( cc() , _ns.c_str() , t, it ) ) {
arr.append( it );
size += it.objsize();
}
@@ -303,12 +372,16 @@ namespace mongo {
i = l->erase( i );
size += t.objsize();
}
-
+
arr.done();
}
- bool transferMods( string& errmsg , BSONObjBuilder& b ){
- if ( ! _active ){
+ /**
+ * called from the dest of a migrate
+ * transfers mods from src to dest
+ */
+ bool transferMods( string& errmsg , BSONObjBuilder& b ) {
+ if ( ! _getActive() ) {
errmsg = "no active migration!";
return false;
}
@@ -318,8 +391,7 @@ namespace mongo {
{
readlock rl( _ns );
Client::Context cx( _ns );
-
- scoped_lock lk( _mutex );
+
xfer( &_deleted , b , "deleted" , size , false );
xfer( &_reload , b , "reload" , size , true );
}
@@ -329,45 +401,201 @@ namespace mongo {
return true;
}
- bool _inCriticalSection;
+ /**
+ * Get the disklocs that belong to the chunk migrated and sort them in _cloneLocs (to avoid seeking disk later)
+ *
+ * @param maxChunkSize number of bytes beyond which a chunk's base data (no indices) is considered too large to move
+ * @param errmsg filled with textual description of error if this call return false
+ * @return false if approximate chunk size is too big to move or true otherwise
+ */
+ bool storeCurrentLocs( long long maxChunkSize , string& errmsg , BSONObjBuilder& result ) {
+ readlock l( _ns );
+ Client::Context ctx( _ns );
+ NamespaceDetails *d = nsdetails( _ns.c_str() );
+ if ( ! d ) {
+ errmsg = "ns not found, should be impossible";
+ return false;
+ }
+
+ BSONObj keyPattern;
+ // the copies are needed because the indexDetailsForRange destroys the input
+ BSONObj min = _min.copy();
+ BSONObj max = _max.copy();
+ IndexDetails *idx = indexDetailsForRange( _ns.c_str() , errmsg , min , max , keyPattern );
+ if ( idx == NULL ) {
+ errmsg = "can't find index in storeCurrentLocs";
+ return false;
+ }
+
+ scoped_ptr<ClientCursor> cc( new ClientCursor( QueryOption_NoCursorTimeout ,
+ shared_ptr<Cursor>( new BtreeCursor( d , d->idxNo(*idx) , *idx , min , max , false , 1 ) ) ,
+ _ns ) );
+
+ // use the average object size to estimate how many objects a full chunk would carry
+ // do that while traversing the chunk's range using the sharding index, below
+ // there's a fair amout of slack before we determine a chunk is too large because object sizes will vary
+ unsigned long long maxRecsWhenFull;
+ long long avgRecSize;
+ const long long totalRecs = d->stats.nrecords;
+ if ( totalRecs > 0 ) {
+ avgRecSize = d->stats.datasize / totalRecs;
+ maxRecsWhenFull = maxChunkSize / avgRecSize;
+ maxRecsWhenFull = 130 * maxRecsWhenFull / 100; // slack
+ }
+ else {
+ avgRecSize = 0;
+ maxRecsWhenFull = numeric_limits<long long>::max();
+ }
+
+ // do a full traversal of the chunk and don't stop even if we think it is a large chunk
+ // we want the number of records to better report, in that case
+ bool isLargeChunk = false;
+ unsigned long long recCount = 0;;
+ while ( cc->ok() ) {
+ DiskLoc dl = cc->currLoc();
+ if ( ! isLargeChunk ) {
+ _cloneLocs.insert( dl );
+ }
+ cc->advance();
+
+ // we can afford to yield here because any change to the base data that we might miss is already being
+ // queued and will be migrated in the 'transferMods' stage
+ if ( ! cc->yieldSometimes() ) {
+ break;
+ }
+
+ if ( ++recCount > maxRecsWhenFull ) {
+ isLargeChunk = true;
+ }
+ }
+
+ if ( isLargeChunk ) {
+ warning() << "can't move chunk of size (aprox) " << recCount * avgRecSize
+ << " because maximum size allowed to move is " << maxChunkSize
+ << " ns: " << _ns << " " << _min << " -> " << _max
+ << endl;
+ result.appendBool( "chunkTooBig" , true );
+ result.appendNumber( "chunkSize" , (long long)(recCount * avgRecSize) );
+ errmsg = "chunk too big to move";
+ return false;
+ }
+
+ log() << "moveChunk number of documents: " << _cloneLocs.size() << endl;
+ return true;
+ }
+
+ bool clone( string& errmsg , BSONObjBuilder& result ) {
+ if ( ! _getActive() ) {
+ errmsg = "not active";
+ return false;
+ }
+
+ readlock l( _ns );
+ Client::Context ctx( _ns );
+
+ NamespaceDetails *d = nsdetails( _ns.c_str() );
+ assert( d );
+
+ BSONArrayBuilder a( std::min( BSONObjMaxUserSize , (int)( ( 12 + d->averageObjectSize() )* _cloneLocs.size() ) ) );
+
+ set<DiskLoc>::iterator i = _cloneLocs.begin();
+ for ( ; i!=_cloneLocs.end(); ++i ) {
+ DiskLoc dl = *i;
+ BSONObj o = dl.obj();
+
+ // use the builder size instead of accumulating 'o's size so that we take into consideration
+ // the overhead of BSONArray indices
+ if ( a.len() + o.objsize() + 1024 > BSONObjMaxUserSize ) {
+ break;
+ }
+ a.append( o );
+ }
+
+ result.appendArray( "objects" , a.arr() );
+ _cloneLocs.erase( _cloneLocs.begin() , i );
+ return true;
+ }
+
+ void aboutToDelete( const Database* db , const DiskLoc& dl ) {
+ dbMutex.assertWriteLocked();
+
+ if ( ! _getActive() )
+ return;
+
+ if ( ! db->ownsNS( _ns ) )
+ return;
+
+ _cloneLocs.erase( dl );
+ }
+
+ long long mbUsed() const { return _memoryUsed / ( 1024 * 1024 ); }
+
+ bool getInCriticalSection() const { scoped_lock l(_m); return _inCriticalSection; }
+ void setInCriticalSection( bool b ) { scoped_lock l(_m); _inCriticalSection = b; }
+
+ bool isActive() const { return _getActive(); }
private:
-
+ mutable mongo::mutex _m; // protect _inCriticalSection and _active
+ bool _inCriticalSection;
bool _active;
string _ns;
BSONObj _min;
BSONObj _max;
- list<BSONObj> _reload;
- list<BSONObj> _deleted;
+ // disk locs yet to be transferred from here to the other side
+ // no locking needed because build by 1 thread in a read lock
+ // depleted by 1 thread in a read lock
+ // updates applied by 1 thread in a write lock
+ set<DiskLoc> _cloneLocs;
+
+ list<BSONObj> _reload; // objects that were modified that must be recloned
+ list<BSONObj> _deleted; // objects deleted during clone that should be deleted later
+ long long _memoryUsed; // bytes in _reload + _deleted
+
+ bool _getActive() const { scoped_lock l(_m); return _active; }
+ void _setActive( bool b ) { scoped_lock l(_m); _active = b; }
- mongo::mutex _mutex;
-
} migrateFromStatus;
-
+
struct MigrateStatusHolder {
- MigrateStatusHolder( string ns , const BSONObj& min , const BSONObj& max ){
+ MigrateStatusHolder( string ns , const BSONObj& min , const BSONObj& max ) {
migrateFromStatus.start( ns , min , max );
}
- ~MigrateStatusHolder(){
+ ~MigrateStatusHolder() {
migrateFromStatus.done();
}
};
- void logOpForSharding( const char * opstr , const char * ns , const BSONObj& obj , BSONObj * patt ){
+ void logOpForSharding( const char * opstr , const char * ns , const BSONObj& obj , BSONObj * patt ) {
migrateFromStatus.logOp( opstr , ns , obj , patt );
}
- class TransferModsCommand : public ChunkCommandHelper{
+ void aboutToDeleteForSharding( const Database* db , const DiskLoc& dl ) {
+ migrateFromStatus.aboutToDelete( db , dl );
+ }
+
+ class TransferModsCommand : public ChunkCommandHelper {
public:
- TransferModsCommand() : ChunkCommandHelper( "_transferMods" ){}
+ TransferModsCommand() : ChunkCommandHelper( "_transferMods" ) {}
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
return migrateFromStatus.transferMods( errmsg, result );
}
} transferModsCommand;
+
+ class InitialCloneCommand : public ChunkCommandHelper {
+ public:
+ InitialCloneCommand() : ChunkCommandHelper( "_migrateClone" ) {}
+
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
+ return migrateFromStatus.clone( errmsg, result );
+ }
+ } initialCloneCommand;
+
+
/**
* this is the main entry for moveChunk
* called to initial a move
@@ -376,20 +604,22 @@ namespace mongo {
*/
class MoveChunkCommand : public Command {
public:
- MoveChunkCommand() : Command( "moveChunk" ){}
+ MoveChunkCommand() : Command( "moveChunk" ) {}
virtual void help( stringstream& help ) const {
help << "should not be calling this directly" << endl;
}
virtual bool slaveOk() const { return false; }
virtual bool adminOnly() const { return true; }
- virtual LockType locktype() const { return NONE; }
-
-
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ virtual LockType locktype() const { return NONE; }
+
+
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
// 1. parse options
// 2. make sure my view is complete and lock
// 3. start migrate
+ // in a read lock, get all DiskLoc and sort so we can do as little seeking as possible
+ // tell to start transferring
// 4. pause till migrate caught up
// 5. LOCK
// a) update my config, essentially locking
@@ -398,10 +628,9 @@ namespace mongo {
// d) logChange to config server
// 6. wait for all current cursors to expire
// 7. remove data locally
-
+
// -------------------------------
-
-
+
// 1.
string ns = cmdObj.firstElement().str();
string to = cmdObj["to"].str();
@@ -409,38 +638,45 @@ namespace mongo {
BSONObj min = cmdObj["min"].Obj();
BSONObj max = cmdObj["max"].Obj();
BSONElement shardId = cmdObj["shardId"];
-
- if ( ns.empty() ){
+ BSONElement maxSizeElem = cmdObj["maxChunkSizeBytes"];
+
+ if ( ns.empty() ) {
errmsg = "need to specify namespace in command";
return false;
}
-
- if ( to.empty() ){
- errmsg = "need to specify server to move shard to";
+
+ if ( to.empty() ) {
+ errmsg = "need to specify server to move chunk to";
return false;
}
- if ( from.empty() ){
- errmsg = "need to specify server to move shard from (redundat i know)";
+ if ( from.empty() ) {
+ errmsg = "need to specify server to move chunk from";
return false;
}
-
- if ( min.isEmpty() ){
+
+ if ( min.isEmpty() ) {
errmsg = "need to specify a min";
return false;
}
- if ( max.isEmpty() ){
+ if ( max.isEmpty() ) {
errmsg = "need to specify a max";
return false;
}
-
- if ( shardId.eoo() ){
+
+ if ( shardId.eoo() ) {
errmsg = "need shardId";
return false;
}
-
- if ( ! shardingState.enabled() ){
- if ( cmdObj["configdb"].type() != String ){
+
+ if ( maxSizeElem.eoo() || ! maxSizeElem.isNumber() ) {
+ errmsg = "need to specify maxChunkSizeBytes";
+ return false;
+ }
+ const long long maxChunkSize = maxSizeElem.numberLong(); // in bytes
+
+ if ( ! shardingState.enabled() ) {
+ if ( cmdObj["configdb"].type() != String ) {
errmsg = "sharding not enabled";
return false;
}
@@ -449,78 +685,107 @@ namespace mongo {
configServer.init( configdb );
}
- MoveTimingHelper timing( "from" , ns , min , max );
+ MoveTimingHelper timing( "from" , ns , min , max , 6 /* steps */);
Shard fromShard( from );
Shard toShard( to );
-
- log() << "got movechunk: " << cmdObj << endl;
+
+ log() << "received moveChunk request: " << cmdObj << endl;
timing.done(1);
- // 2.
-
+
+ // 2.
DistributedLock lockSetup( ConnectionString( shardingState.getConfigServer() , ConnectionString::SYNC ) , ns );
dist_lock_try dlk( &lockSetup , (string)"migrate-" + min.toString() );
- if ( ! dlk.got() ){
- errmsg = "someone else has the lock";
+ if ( ! dlk.got() ) {
+ errmsg = "the collection's metadata lock is taken";
result.append( "who" , dlk.other() );
return false;
}
+ BSONObj chunkInfo = BSON("min" << min << "max" << max << "from" << fromShard.getName() << "to" << toShard.getName());
+ configServer.logChange( "moveChunk.start" , ns , chunkInfo );
+
ShardChunkVersion maxVersion;
string myOldShard;
{
ScopedDbConnection conn( shardingState.getConfigServer() );
-
+
BSONObj x = conn->findOne( ShardNS::chunk , Query( BSON( "ns" << ns ) ).sort( BSON( "lastmod" << -1 ) ) );
maxVersion = x["lastmod"];
- x = conn->findOne( ShardNS::chunk , shardId.wrap( "_id" ) );
- assert( x["shard"].type() );
- myOldShard = x["shard"].String();
-
- if ( myOldShard != fromShard.getName() ){
- errmsg = "i'm out of date";
+ BSONObj currChunk = conn->findOne( ShardNS::chunk , shardId.wrap( "_id" ) );
+ assert( currChunk["shard"].type() );
+ assert( currChunk["min"].type() );
+ assert( currChunk["max"].type() );
+ myOldShard = currChunk["shard"].String();
+ conn.done();
+
+ BSONObj currMin = currChunk["min"].Obj();
+ BSONObj currMax = currChunk["max"].Obj();
+ if ( currMin.woCompare( min ) || currMax.woCompare( max ) ) {
+ errmsg = "boundaries are outdated (likely a split occurred)";
+ result.append( "currMin" , currMin );
+ result.append( "currMax" , currMax );
+ result.append( "requestedMin" , min );
+ result.append( "requestedMax" , max );
+
+ log( LL_WARNING ) << "aborted moveChunk because" << errmsg << ": " << min << "->" << max
+ << " is now " << currMin << "->" << currMax << endl;
+ return false;
+ }
+
+ if ( myOldShard != fromShard.getName() ) {
+ errmsg = "location is outdated (likely balance or migrate occurred)";
result.append( "from" , fromShard.getName() );
result.append( "official" , myOldShard );
+
+ log( LL_WARNING ) << "aborted moveChunk because " << errmsg << ": chunk is at " << myOldShard
+ << " and not at " << fromShard.getName() << endl;
return false;
}
-
- if ( maxVersion < shardingState.getVersion( ns ) ){
- errmsg = "official version less than mine?";;
+
+ if ( maxVersion < shardingState.getVersion( ns ) ) {
+ errmsg = "official version less than mine?";
result.appendTimestamp( "officialVersion" , maxVersion );
result.appendTimestamp( "myVersion" , shardingState.getVersion( ns ) );
+
+ log( LL_WARNING ) << "aborted moveChunk because " << errmsg << ": official " << maxVersion
+ << " mine: " << shardingState.getVersion(ns) << endl;
return false;
}
- conn.done();
+ // since this could be the first call that enable sharding we also make sure to have the chunk manager up to date
+ shardingState.gotShardName( myOldShard );
+ ShardChunkVersion shardVersion;
+ shardingState.trySetVersion( ns , shardVersion /* will return updated */ );
+
+ log() << "moveChunk request accepted at version " << shardVersion << endl;
}
-
+
timing.done(2);
-
+
// 3.
MigrateStatusHolder statusHolder( ns , min , max );
{
- dblock lk;
- // this makes sure there wasn't a write inside the .cpp code we can miss
- }
-
- {
-
- ScopedDbConnection conn( to );
- BSONObj res;
- bool ok = conn->runCommand( "admin" ,
- BSON( "_recvChunkStart" << ns <<
- "from" << from <<
- "min" << min <<
- "max" << max <<
- "configServer" << configServer.modelServer()
- ) ,
- res );
- conn.done();
+ // this gets a read lock, so we know we have a checkpoint for mods
+ if ( ! migrateFromStatus.storeCurrentLocs( maxChunkSize , errmsg , result ) )
+ return false;
- if ( ! ok ){
- errmsg = "_recvChunkStart failed: ";
+ ScopedDbConnection connTo( to );
+ BSONObj res;
+ bool ok = connTo->runCommand( "admin" ,
+ BSON( "_recvChunkStart" << ns <<
+ "from" << from <<
+ "min" << min <<
+ "max" << max <<
+ "configServer" << configServer.modelServer()
+ ) ,
+ res );
+ connTo.done();
+
+ if ( ! ok ) {
+ errmsg = "moveChunk failed to engage TO-shard in the data transfer: ";
assert( res["errmsg"].type() );
errmsg += res["errmsg"].String();
result.append( "cause" , res );
@@ -529,118 +794,275 @@ namespace mongo {
}
timing.done( 3 );
-
- // 4.
- for ( int i=0; i<86400; i++ ){ // don't want a single chunk move to take more than a day
+
+ // 4.
+ for ( int i=0; i<86400; i++ ) { // don't want a single chunk move to take more than a day
assert( dbMutex.getState() == 0 );
- sleepsecs( 1 );
+ sleepsecs( 1 );
ScopedDbConnection conn( to );
BSONObj res;
bool ok = conn->runCommand( "admin" , BSON( "_recvChunkStatus" << 1 ) , res );
res = res.getOwned();
conn.done();
-
- log(0) << "_recvChunkStatus : " << res << endl;
-
- if ( ! ok || res["state"].String() == "fail" ){
- log( LL_ERROR ) << "_recvChunkStatus error : " << res << endl;
- errmsg = "_recvChunkStatus error";
- result.append( "cause" ,res );
+
+ log(0) << "moveChunk data transfer progress: " << res << " my mem used: " << migrateFromStatus.mbUsed() << endl;
+
+ if ( ! ok || res["state"].String() == "fail" ) {
+ log( LL_WARNING ) << "moveChunk error transfering data caused migration abort: " << res << endl;
+ errmsg = "data transfer error";
+ result.append( "cause" , res );
return false;
}
if ( res["state"].String() == "steady" )
break;
+ if ( migrateFromStatus.mbUsed() > (500 * 1024 * 1024) ) {
+ // this is too much memory for us to use for this
+ // so we're going to abort the migrate
+ ScopedDbConnection conn( to );
+ BSONObj res;
+ conn->runCommand( "admin" , BSON( "_recvChunkAbort" << 1 ) , res );
+ res = res.getOwned();
+ conn.done();
+ error() << "aborting migrate because too much memory used res: " << res << endl;
+ errmsg = "aborting migrate because too much memory used";
+ result.appendBool( "split" , true );
+ return false;
+ }
+
killCurrentOp.checkForInterrupt();
}
timing.done(4);
// 5.
- {
+ {
// 5.a
- migrateFromStatus._inCriticalSection = true;
- ShardChunkVersion myVersion = maxVersion;
+ // we're under the collection lock here, so no other migrate can change maxVersion or ShardChunkManager state
+ migrateFromStatus.setInCriticalSection( true );
+ ShardChunkVersion currVersion = maxVersion;
+ ShardChunkVersion myVersion = currVersion;
myVersion.incMajor();
-
+
{
- dblock lk;
+ writelock lk( ns );
assert( myVersion > shardingState.getVersion( ns ) );
- shardingState.setVersion( ns , myVersion );
- assert( myVersion == shardingState.getVersion( ns ) );
- log() << "moveChunk locking myself to: " << myVersion << endl;
+
+ // bump the chunks manager's version up and "forget" about the chunk being moved
+ // this is not the commit point but in practice the state in this shard won't until the commit it done
+ shardingState.donateChunk( ns , min , max , myVersion );
}
-
+ log() << "moveChunk setting version to: " << myVersion << endl;
+
// 5.b
+ // we're under the collection lock here, too, so we can undo the chunk donation because no other state change
+ // could be ongoing
{
BSONObj res;
- ScopedDbConnection conn( to );
- bool ok = conn->runCommand( "admin" ,
- BSON( "_recvChunkCommit" << 1 ) ,
- res );
- conn.done();
- log() << "moveChunk commit result: " << res << endl;
- if ( ! ok ){
- log() << "_recvChunkCommit failed: " << res << endl;
+ ScopedDbConnection connTo( to );
+ bool ok = connTo->runCommand( "admin" ,
+ BSON( "_recvChunkCommit" << 1 ) ,
+ res );
+ connTo.done();
+
+ if ( ! ok ) {
+ {
+ writelock lk( ns );
+
+ // revert the chunk manager back to the state before "forgetting" about the chunk
+ shardingState.undoDonateChunk( ns , min , max , currVersion );
+ }
+
+ log() << "movChunk migrate commit not accepted by TO-shard: " << res
+ << " resetting shard version to: " << currVersion << endl;
+
errmsg = "_recvChunkCommit failed!";
result.append( "cause" , res );
return false;
}
+
+ log() << "moveChunk migrate commit accepted by TO-shard: " << res << endl;
}
-
+
// 5.c
- ScopedDbConnection conn( shardingState.getConfigServer() );
-
- BSONObjBuilder temp;
- temp.append( "shard" , toShard.getName() );
- temp.appendTimestamp( "lastmod" , myVersion );
-
- conn->update( ShardNS::chunk , shardId.wrap( "_id" ) , BSON( "$set" << temp.obj() ) );
-
- {
- // update another random chunk
- BSONObj x = conn->findOne( ShardNS::chunk , Query( BSON( "ns" << ns << "shard" << myOldShard ) ).sort( BSON( "lastmod" << -1 ) ) );
- if ( ! x.isEmpty() ){
-
- BSONObjBuilder temp2;
- myVersion.incMinor();
-
- temp2.appendTimestamp( "lastmod" , myVersion );
-
- shardingState.setVersion( ns , myVersion );
-
- conn->update( ShardNS::chunk , x["_id"].wrap() , BSON( "$set" << temp2.obj() ) );
-
- log() << "moveChunk updating self to: " << myVersion << endl;
+
+ // version at which the next highest lastmod will be set
+ // if the chunk being moved is the last in the shard, nextVersion is that chunk's lastmod
+ // otherwise the highest version is from the chunk being bumped on the FROM-shard
+ ShardChunkVersion nextVersion;
+
+ // we want to go only once to the configDB but perhaps change two chunks, the one being migrated and another
+ // local one (so to bump version for the entire shard)
+ // we use the 'applyOps' mechanism to group the two updates and make them safer
+ // TODO pull config update code to a module
+
+ BSONObjBuilder cmdBuilder;
+
+ BSONArrayBuilder updates( cmdBuilder.subarrayStart( "applyOps" ) );
+ {
+ // update for the chunk being moved
+ BSONObjBuilder op;
+ op.append( "op" , "u" );
+ op.appendBool( "b" , false /* no upserting */ );
+ op.append( "ns" , ShardNS::chunk );
+
+ BSONObjBuilder n( op.subobjStart( "o" ) );
+ n.append( "_id" , Chunk::genID( ns , min ) );
+ n.appendTimestamp( "lastmod" , myVersion /* same as used on donateChunk */ );
+ n.append( "ns" , ns );
+ n.append( "min" , min );
+ n.append( "max" , max );
+ n.append( "shard" , toShard.getName() );
+ n.done();
+
+ BSONObjBuilder q( op.subobjStart( "o2" ) );
+ q.append( "_id" , Chunk::genID( ns , min ) );
+ q.done();
+
+ updates.append( op.obj() );
+ }
+
+ nextVersion = myVersion;
+
+ // if we have chunks left on the FROM shard, update the version of one of them as well
+ // we can figure that out by grabbing the chunkManager installed on 5.a
+ // TODO expose that manager when installing it
+
+ ShardChunkManagerPtr chunkManager = shardingState.getShardChunkManager( ns );
+ if( chunkManager->getNumChunks() > 0 ) {
+
+ // get another chunk on that shard
+ BSONObj lookupKey;
+ BSONObj bumpMin, bumpMax;
+ do {
+ chunkManager->getNextChunk( lookupKey , &bumpMin , &bumpMax );
+ lookupKey = bumpMin;
+ }
+ while( bumpMin == min );
+
+ BSONObjBuilder op;
+ op.append( "op" , "u" );
+ op.appendBool( "b" , false );
+ op.append( "ns" , ShardNS::chunk );
+
+ nextVersion.incMinor(); // same as used on donateChunk
+ BSONObjBuilder n( op.subobjStart( "o" ) );
+ n.append( "_id" , Chunk::genID( ns , bumpMin ) );
+ n.appendTimestamp( "lastmod" , nextVersion );
+ n.append( "ns" , ns );
+ n.append( "min" , bumpMin );
+ n.append( "max" , bumpMax );
+ n.append( "shard" , fromShard.getName() );
+ n.done();
+
+ BSONObjBuilder q( op.subobjStart( "o2" ) );
+ q.append( "_id" , Chunk::genID( ns , bumpMin ) );
+ q.done();
+
+ updates.append( op.obj() );
+
+ log() << "moveChunk updating self version to: " << nextVersion << " through "
+ << bumpMin << " -> " << bumpMax << " for collection '" << ns << "'" << endl;
+
+ }
+ else {
+
+ log() << "moveChunk moved last chunk out for collection '" << ns << "'" << endl;
+ }
+
+ updates.done();
+
+ BSONArrayBuilder preCond( cmdBuilder.subarrayStart( "preCondition" ) );
+ {
+ BSONObjBuilder b;
+ b.append( "ns" , ShardNS::chunk );
+ b.append( "q" , BSON( "query" << BSON( "ns" << ns ) << "orderby" << BSON( "lastmod" << -1 ) ) );
+ {
+ BSONObjBuilder bb( b.subobjStart( "res" ) );
+ bb.appendTimestamp( "lastmod" , maxVersion );
+ bb.done();
}
- else {
- //++myVersion;
- shardingState.setVersion( ns , 0 );
+ preCond.append( b.obj() );
+ }
+
+ preCond.done();
+
+ BSONObj cmd = cmdBuilder.obj();
+ log(7) << "moveChunk update: " << cmd << endl;
+
+ bool ok = false;
+ BSONObj cmdResult;
+ try {
+ ScopedDbConnection conn( shardingState.getConfigServer() );
+ ok = conn->runCommand( "config" , cmd , cmdResult );
+ conn.done();
+ }
+ catch ( DBException& e ) {
+ ok = false;
+ BSONObjBuilder b;
+ e.getInfo().append( b );
+ cmdResult = b.obj();
+ }
+
+ if ( ! ok ) {
+
+ // this could be a blip in the connectivity
+ // wait out a few seconds and check if the commit request made it
+ //
+ // if the commit made it to the config, we'll see the chunk in the new shard and there's no action
+ // if the commit did not make it, currently the only way to fix this state is to bounce the mongod so
+ // that the old state (before migrating) be brought in
+
+ warning() << "moveChunk commit outcome ongoing: " << cmd << " for command :" << cmdResult << endl;
+ sleepsecs( 10 );
+
+ try {
+ ScopedDbConnection conn( shardingState.getConfigServer() );
+
+ // look for the chunk in this shard whose version got bumped
+ // we assume that if that mod made it to the config, the applyOps was successful
+ BSONObj doc = conn->findOne( ShardNS::chunk , Query(BSON( "ns" << ns )).sort( BSON("lastmod" << -1)));
+ ShardChunkVersion checkVersion = doc["lastmod"];
+
+ if ( checkVersion == nextVersion ) {
+ log() << "moveChunk commit confirmed" << endl;
+
+ }
+ else {
+ error() << "moveChunk commit failed: version is at"
+ << checkVersion << " instead of " << nextVersion << endl;
+ error() << "TERMINATING" << endl;
+ dbexit( EXIT_SHARDING_ERROR );
+ }
+
+ conn.done();
- log() << "moveChunk now i'm empty" << endl;
+ }
+ catch ( ... ) {
+ error() << "moveChunk failed to get confirmation of commit" << endl;
+ error() << "TERMINATING" << endl;
+ dbexit( EXIT_SHARDING_ERROR );
}
}
- conn.done();
- migrateFromStatus._inCriticalSection = false;
+ migrateFromStatus.setInCriticalSection( false );
+
// 5.d
- configServer.logChange( "moveChunk" , ns , BSON( "min" << min << "max" << max <<
- "from" << fromShard.getName() <<
- "to" << toShard.getName() ) );
+ configServer.logChange( "moveChunk.commit" , ns , chunkInfo );
}
-
+
migrateFromStatus.done();
timing.done(5);
-
- { // 6.
+ {
+ // 6.
OldDataCleanup c;
c.ns = ns;
c.min = min.getOwned();
c.max = max.getOwned();
ClientCursor::find( ns , c.initial );
- if ( c.initial.size() ){
+ if ( c.initial.size() ) {
log() << "forking for cleaning up chunk data" << endl;
boost::thread t( boost::bind( &cleanupOldData , c ) );
}
@@ -649,24 +1071,24 @@ namespace mongo {
// 7.
c.doRemove();
}
-
-
+
+
}
- timing.done(6);
+ timing.done(6);
return true;
-
+
}
-
+
} moveChunkCmd;
- bool ShardingState::inCriticalMigrateSection(){
- return migrateFromStatus._inCriticalSection;
+ bool ShardingState::inCriticalMigrateSection() {
+ return migrateFromStatus.getInCriticalSection();
}
/* -----
below this are the "to" side commands
-
+
command to initiate
worker thread
does initial clone
@@ -679,71 +1101,74 @@ namespace mongo {
class MigrateStatus {
public:
-
- MigrateStatus(){
- active = false;
- }
- void prepare(){
+ MigrateStatus() : m_active("MigrateStatus") { active = false; }
+
+ void prepare() {
+ scoped_lock l(m_active); // reading and writing 'active'
+
assert( ! active );
state = READY;
errmsg = "";
numCloned = 0;
+ clonedBytes = 0;
numCatchup = 0;
numSteady = 0;
active = true;
}
- void go(){
+ void go() {
try {
_go();
}
- catch ( std::exception& e ){
+ catch ( std::exception& e ) {
state = FAIL;
errmsg = e.what();
log( LL_ERROR ) << "migrate failed: " << e.what() << endl;
}
- catch ( ... ){
+ catch ( ... ) {
state = FAIL;
errmsg = "UNKNOWN ERROR";
log( LL_ERROR ) << "migrate failed with unknown exception" << endl;
}
- active = false;
+ setActive( false );
}
-
- void _go(){
- assert( active );
+
+ void _go() {
+ assert( getActive() );
assert( state == READY );
assert( ! min.isEmpty() );
assert( ! max.isEmpty() );
-
- MoveTimingHelper timing( "to" , ns , min , max );
-
+
+ MoveTimingHelper timing( "to" , ns , min , max , 5 /* steps */ );
+
ScopedDbConnection conn( from );
conn->getLastError(); // just test connection
- { // 1. copy indexes
+ {
+ // 1. copy indexes
auto_ptr<DBClientCursor> indexes = conn->getIndexes( ns );
vector<BSONObj> all;
- while ( indexes->more() ){
+ while ( indexes->more() ) {
all.push_back( indexes->next().getOwned() );
}
-
+
writelock lk( ns );
Client::Context ct( ns );
-
+
string system_indexes = cc().database()->name + ".system.indexes";
- for ( unsigned i=0; i<all.size(); i++ ){
+ for ( unsigned i=0; i<all.size(); i++ ) {
BSONObj idx = all[i];
- theDataFileMgr.insert( system_indexes.c_str() , idx.objdata() , idx.objsize() );
+ theDataFileMgr.insertAndLog( system_indexes.c_str() , idx );
}
-
+
timing.done(1);
}
-
- { // 2. delete any data already in range
+
+ {
+ // 2. delete any data already in range
writelock lk( ns );
RemoveSaver rs( "moveChunk" , ns , "preCleanup" );
long long num = Helpers::removeRange( ns , min , max , true , false , cmdLine.moveParanoia ? &rs : 0 );
@@ -752,29 +1177,54 @@ namespace mongo {
timing.done(2);
}
-
-
- { // 3. initial bulk clone
+
+
+ {
+ // 3. initial bulk clone
state = CLONE;
- auto_ptr<DBClientCursor> cursor = conn->query( ns , Query().minKey( min ).maxKey( max ) , /* QueryOption_Exhaust */ 0 );
- assert( cursor.get() );
- while ( cursor->more() ){
- BSONObj o = cursor->next().getOwned();
- {
- writelock lk( ns );
- Helpers::upsert( ns , o );
+
+ while ( true ) {
+ BSONObj res;
+ if ( ! conn->runCommand( "admin" , BSON( "_migrateClone" << 1 ) , res ) ) {
+ state = FAIL;
+ errmsg = "_migrateClone failed: ";
+ errmsg += res.toString();
+ error() << errmsg << endl;
+ conn.done();
+ return;
+ }
+
+ BSONObj arr = res["objects"].Obj();
+ int thisTime = 0;
+
+ BSONObjIterator i( arr );
+ while( i.more() ) {
+ BSONObj o = i.next().Obj();
+ {
+ writelock lk( ns );
+ Helpers::upsert( ns , o );
+ }
+ thisTime++;
+ numCloned++;
+ clonedBytes += o.objsize();
}
- numCloned++;
+
+ if ( thisTime == 0 )
+ break;
}
timing.done(3);
}
-
- { // 4. do bulk of mods
+
+ // if running on a replicated system, we'll need to flush the docs we cloned to the secondaries
+ ReplTime lastOpApplied;
+
+ {
+ // 4. do bulk of mods
state = CATCHUP;
- while ( true ){
+ while ( true ) {
BSONObj res;
- if ( ! conn->runCommand( "admin" , BSON( "_transferMods" << 1 ) , res ) ){
+ if ( ! conn->runCommand( "admin" , BSON( "_transferMods" << 1 ) , res ) ) {
state = FAIL;
errmsg = "_transferMods failed: ";
errmsg += res.toString();
@@ -784,18 +1234,26 @@ namespace mongo {
}
if ( res["size"].number() == 0 )
break;
-
- apply( res );
+
+ apply( res , &lastOpApplied );
+
+ if ( state == ABORT ) {
+ timing.note( "aborted" );
+ return;
+ }
}
timing.done(4);
}
-
- { // 5. wait for commit
+
+ {
+ // 5. wait for commit
+ Timer timeWaitingForCommit;
+
state = STEADY;
- while ( state == STEADY || state == COMMIT_START ){
+ while ( state == STEADY || state == COMMIT_START ) {
BSONObj res;
- if ( ! conn->runCommand( "admin" , BSON( "_transferMods" << 1 ) , res ) ){
+ if ( ! conn->runCommand( "admin" , BSON( "_transferMods" << 1 ) , res ) ) {
log() << "_transferMods failed in STEADY state: " << res << endl;
errmsg = res.toString();
state = FAIL;
@@ -803,36 +1261,48 @@ namespace mongo {
return;
}
- if ( res["size"].number() > 0 && apply( res ) )
+ if ( res["size"].number() > 0 && apply( res , &lastOpApplied ) )
continue;
-
- if ( state == COMMIT_START )
+
+ if ( state == COMMIT_START && flushPendingWrites( lastOpApplied ) )
break;
sleepmillis( 10 );
}
-
+
+ if ( state == ABORT ) {
+ timing.note( "aborted" );
+ return;
+ }
+
+ if ( timeWaitingForCommit.seconds() > 86400 ) {
+ state = FAIL;
+ errmsg = "timed out waiting for commit";
+ return;
+ }
+
timing.done(5);
}
-
+
state = DONE;
conn.done();
}
- void status( BSONObjBuilder& b ){
- b.appendBool( "active" , active );
+ void status( BSONObjBuilder& b ) {
+ b.appendBool( "active" , getActive() );
b.append( "ns" , ns );
b.append( "from" , from );
b.append( "min" , min );
b.append( "max" , max );
-
+
b.append( "state" , stateString() );
if ( state == FAIL )
b.append( "errmsg" , errmsg );
{
BSONObjBuilder bb( b.subobjStart( "counts" ) );
bb.append( "cloned" , numCloned );
+ bb.append( "clonedBytes" , clonedBytes );
bb.append( "catchup" , numCatchup );
bb.append( "steady" , numSteady );
bb.done();
@@ -841,17 +1311,22 @@ namespace mongo {
}
- bool apply( const BSONObj& xfer ){
+ bool apply( const BSONObj& xfer , ReplTime* lastOpApplied ) {
+ ReplTime dummy;
+ if ( lastOpApplied == NULL ) {
+ lastOpApplied = &dummy;
+ }
+
bool didAnything = false;
-
- if ( xfer["deleted"].isABSONObj() ){
+
+ if ( xfer["deleted"].isABSONObj() ) {
writelock lk(ns);
Client::Context cx(ns);
-
+
RemoveSaver rs( "moveChunk" , ns , "removedDuring" );
BSONObjIterator i( xfer["deleted"].Obj() );
- while ( i.more() ){
+ while ( i.more() ) {
BSONObj id = i.next().Obj();
// do not apply deletes if they do not belong to the chunk being migrated
@@ -865,27 +1340,56 @@ namespace mongo {
}
Helpers::removeRange( ns , id , id, false , true , cmdLine.moveParanoia ? &rs : 0 );
+
+ *lastOpApplied = cx.getClient()->getLastOp();
didAnything = true;
}
}
-
- if ( xfer["reload"].isABSONObj() ){
+
+ if ( xfer["reload"].isABSONObj() ) {
writelock lk(ns);
Client::Context cx(ns);
BSONObjIterator i( xfer["reload"].Obj() );
- while ( i.more() ){
+ while ( i.more() ) {
BSONObj it = i.next().Obj();
+
Helpers::upsert( ns , it );
+
+ *lastOpApplied = cx.getClient()->getLastOp();
didAnything = true;
}
}
return didAnything;
}
-
- string stateString(){
- switch ( state ){
+
+ bool flushPendingWrites( const ReplTime& lastOpApplied ) {
+ // if replication is on, try to force enough secondaries to catch up
+ // TODO opReplicatedEnough should eventually honor priorities and geo-awareness
+ // for now, we try to replicate to a sensible number of secondaries
+ const int slaveCount = getSlaveCount() / 2 + 1;
+ if ( ! opReplicatedEnough( lastOpApplied , slaveCount ) ) {
+ log( LL_WARNING ) << "migrate commit attempt timed out contacting " << slaveCount
+ << " slaves for '" << ns << "' " << min << " -> " << max << endl;
+ return false;
+ }
+ log() << "migrate commit succeeded flushing to secondaries for '" << ns << "' " << min << " -> " << max << endl;
+
+ {
+ readlock lk(ns); // commitNow() currently requires it
+
+ // if durability is on, force a write to journal
+ if ( getDur().commitNow() ) {
+ log() << "migrate commit flushed to journal for '" << ns << "' " << min << " -> " << max << endl;
+ }
+ }
+
+ return true;
+ }
+
+ string stateString() {
+ switch ( state ) {
case READY: return "ready";
case CLONE: return "clone";
case CATCHUP: return "catchup";
@@ -893,17 +1397,18 @@ namespace mongo {
case COMMIT_START: return "commitStart";
case DONE: return "done";
case FAIL: return "fail";
+ case ABORT: return "abort";
}
assert(0);
return "";
}
- bool startCommit(){
+ bool startCommit() {
if ( state != STEADY )
return false;
state = COMMIT_START;
-
- for ( int i=0; i<86400; i++ ){
+
+ for ( int i=0; i<86400; i++ ) {
sleepmillis(1);
if ( state == DONE )
return true;
@@ -912,42 +1417,60 @@ namespace mongo {
return false;
}
+ void abort() {
+ state = ABORT;
+ errmsg = "aborted";
+ }
+
+ bool getActive() const { scoped_lock l(m_active); return active; }
+ void setActive( bool b ) { scoped_lock l(m_active); active = b; }
+
+ mutable mongo::mutex m_active;
bool active;
-
+
string ns;
string from;
-
+
BSONObj min;
BSONObj max;
-
+
long long numCloned;
+ long long clonedBytes;
long long numCatchup;
long long numSteady;
- enum State { READY , CLONE , CATCHUP , STEADY , COMMIT_START , DONE , FAIL } state;
+ enum State { READY , CLONE , CATCHUP , STEADY , COMMIT_START , DONE , FAIL , ABORT } state;
string errmsg;
-
+
} migrateStatus;
-
- void migrateThread(){
+
+ void migrateThread() {
Client::initThread( "migrateThread" );
migrateStatus.go();
cc().shutdown();
}
-
+
class RecvChunkStartCommand : public ChunkCommandHelper {
public:
- RecvChunkStartCommand() : ChunkCommandHelper( "_recvChunkStart" ){}
+ RecvChunkStartCommand() : ChunkCommandHelper( "_recvChunkStart" ) {}
virtual LockType locktype() const { return WRITE; } // this is so don't have to do locking internally
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
-
- if ( migrateStatus.active ){
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
+
+ if ( migrateStatus.getActive() ) {
errmsg = "migrate already in progress";
return false;
}
+ if ( OldDataCleanup::_numThreads > 0 ) {
+ errmsg =
+ str::stream()
+ << "still waiting for a previous migrates data to get cleaned, can't accept new chunks, num threads: "
+ << OldDataCleanup::_numThreads;
+ return false;
+ }
+
if ( ! configServer.ok() )
configServer.init( cmdObj["configServer"].String() );
@@ -957,9 +1480,9 @@ namespace mongo {
migrateStatus.from = cmdObj["from"].String();
migrateStatus.min = cmdObj["min"].Obj().getOwned();
migrateStatus.max = cmdObj["max"].Obj().getOwned();
-
+
boost::thread m( migrateThread );
-
+
result.appendBool( "started" , true );
return true;
}
@@ -968,20 +1491,20 @@ namespace mongo {
class RecvChunkStatusCommand : public ChunkCommandHelper {
public:
- RecvChunkStatusCommand() : ChunkCommandHelper( "_recvChunkStatus" ){}
+ RecvChunkStatusCommand() : ChunkCommandHelper( "_recvChunkStatus" ) {}
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
migrateStatus.status( result );
return 1;
}
-
+
} recvChunkStatusCommand;
class RecvChunkCommitCommand : public ChunkCommandHelper {
public:
- RecvChunkCommitCommand() : ChunkCommandHelper( "_recvChunkCommit" ){}
-
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ RecvChunkCommitCommand() : ChunkCommandHelper( "_recvChunkCommit" ) {}
+
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
bool ok = migrateStatus.startCommit();
migrateStatus.status( result );
return ok;
@@ -989,10 +1512,22 @@ namespace mongo {
} recvChunkCommitCommand;
+ class RecvChunkAbortCommand : public ChunkCommandHelper {
+ public:
+ RecvChunkAbortCommand() : ChunkCommandHelper( "_recvChunkAbort" ) {}
+
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
+ migrateStatus.abort();
+ migrateStatus.status( result );
+ return true;
+ }
+
+ } recvChunkAboortCommand;
+
class IsInRangeTest : public UnitTest {
public:
- void run(){
+ void run() {
BSONObj min = BSON( "x" << 1 );
BSONObj max = BSON( "x" << 5 );
@@ -1002,6 +1537,8 @@ namespace mongo {
assert( isInRange( BSON( "x" << 4 ) , min , max ) );
assert( ! isInRange( BSON( "x" << 5 ) , min , max ) );
assert( ! isInRange( BSON( "x" << 6 ) , min , max ) );
+
+ log(1) << "isInRangeTest passed" << endl;
}
} isInRangeTest;
}
diff --git a/s/d_split.cpp b/s/d_split.cpp
index fdefc7e..0896803 100644
--- a/s/d_split.cpp
+++ b/s/d_split.cpp
@@ -1,4 +1,4 @@
-// d_split.cpp
+// @file d_split.cpp
/**
* Copyright (C) 2008 10gen Inc.
@@ -27,6 +27,13 @@
#include "../db/query.h"
#include "../db/queryoptimizer.h"
+#include "../client/connpool.h"
+#include "../client/distlock.h"
+
+#include "chunk.h" // for static genID only
+#include "config.h"
+#include "d_logic.h"
+
namespace mongo {
// TODO: Fold these checks into each command.
@@ -43,19 +50,19 @@ namespace mongo {
public:
CmdMedianKey() : Command( "medianKey" ) {}
virtual bool slaveOk() const { return true; }
- virtual LockType locktype() const { return READ; }
+ virtual LockType locktype() const { return READ; }
virtual void help( stringstream &help ) const {
- help <<
- "Internal command.\n"
- "example: { medianKey:\"blog.posts\", keyPattern:{x:1}, min:{x:10}, max:{x:55} }\n"
- "NOTE: This command may take a while to run";
+ help <<
+ "Internal command.\n"
+ "example: { medianKey:\"blog.posts\", keyPattern:{x:1}, min:{x:10}, max:{x:55} }\n"
+ "NOTE: This command may take a while to run";
}
- bool run(const string& dbname, BSONObj& jsobj, string& errmsg, BSONObjBuilder& result, bool fromRepl ){
+ bool run(const string& dbname, BSONObj& jsobj, string& errmsg, BSONObjBuilder& result, bool fromRepl ) {
const char *ns = jsobj.getStringField( "medianKey" );
BSONObj min = jsobj.getObjectField( "min" );
BSONObj max = jsobj.getObjectField( "max" );
BSONObj keyPattern = jsobj.getObjectField( "keyPattern" );
-
+
Client::Context ctx( ns );
IndexDetails *id = cmdIndexDetailsForRange( ns, errmsg, min, max, keyPattern );
@@ -66,22 +73,22 @@ namespace mongo {
int num = 0;
NamespaceDetails *d = nsdetails(ns);
int idxNo = d->idxNo(*id);
-
+
// only yielding on firt half for now
// after this it should be in ram, so 2nd should be fast
{
shared_ptr<Cursor> c( new BtreeCursor( d, idxNo, *id, min, max, false, 1 ) );
scoped_ptr<ClientCursor> cc( new ClientCursor( QueryOption_NoCursorTimeout , c , ns ) );
- while ( c->ok() ){
+ while ( c->ok() ) {
num++;
c->advance();
if ( ! cc->yieldSometimes() )
break;
}
}
-
+
num /= 2;
-
+
BtreeCursor c( d, idxNo, *id, min, max, false, 1 );
for( ; num; c.advance(), --num );
@@ -99,15 +106,15 @@ namespace mongo {
int x = median.woCompare( min , BSONObj() , false );
int y = median.woCompare( max , BSONObj() , false );
- if ( x == 0 || y == 0 ){
+ if ( x == 0 || y == 0 ) {
// its on an edge, ok
}
- else if ( x < 0 && y < 0 ){
+ else if ( x < 0 && y < 0 ) {
log( LL_ERROR ) << "median error (1) min: " << min << " max: " << max << " median: " << median << endl;
errmsg = "median error 1";
return false;
}
- else if ( x > 0 && y > 0 ){
+ else if ( x > 0 && y > 0 ) {
log( LL_ERROR ) << "median error (2) min: " << min << " max: " << max << " median: " << median << endl;
errmsg = "median error 2";
return false;
@@ -117,95 +124,662 @@ namespace mongo {
}
} cmdMedianKey;
- class SplitVector : public Command {
- public:
- SplitVector() : Command( "splitVector" , false ){}
+ class CheckShardingIndex : public Command {
+ public:
+ CheckShardingIndex() : Command( "checkShardingIndex" , false ) {}
virtual bool slaveOk() const { return false; }
virtual LockType locktype() const { return READ; }
virtual void help( stringstream &help ) const {
- help <<
- "Internal command.\n"
- "example: { splitVector : \"myLargeCollection\" , keyPattern : {x:1} , maxChunkSize : 200 }\n"
- "maxChunkSize unit in MBs\n"
- "NOTE: This command may take a while to run";
+ help << "Internal command.\n";
}
- bool run(const string& dbname, BSONObj& jsobj, string& errmsg, BSONObjBuilder& result, bool fromRepl ){
- const char* ns = jsobj.getStringField( "splitVector" );
+
+ bool run(const string& dbname, BSONObj& jsobj, string& errmsg, BSONObjBuilder& result, bool fromRepl ) {
+
+ const char* ns = jsobj.getStringField( "checkShardingIndex" );
BSONObj keyPattern = jsobj.getObjectField( "keyPattern" );
- long long maxChunkSize = 0;
- BSONElement maxSizeElem = jsobj[ "maxChunkSize" ];
- if ( ! maxSizeElem.eoo() ){
- maxChunkSize = maxSizeElem.numberLong() * 1<<20;
- } else {
- errmsg = "need to specify the desired max chunk size";
+ // If min and max are not provided use the "minKey" and "maxKey" for the sharding key pattern.
+ BSONObj min = jsobj.getObjectField( "min" );
+ BSONObj max = jsobj.getObjectField( "max" );
+ if ( min.isEmpty() && max.isEmpty() ) {
+ BSONObjBuilder minBuilder;
+ BSONObjBuilder maxBuilder;
+ BSONForEach(key, keyPattern) {
+ minBuilder.appendMinKey( key.fieldName() );
+ maxBuilder.appendMaxKey( key.fieldName() );
+ }
+ min = minBuilder.obj();
+ max = maxBuilder.obj();
+ }
+ else if ( min.isEmpty() || max.isEmpty() ) {
+ errmsg = "either provide both min and max or leave both empty";
return false;
}
-
- Client::Context ctx( ns );
- BSONObjBuilder minBuilder;
- BSONObjBuilder maxBuilder;
- BSONForEach(key, keyPattern){
- minBuilder.appendMinKey( key.fieldName() );
- maxBuilder.appendMaxKey( key.fieldName() );
+ Client::Context ctx( ns );
+ NamespaceDetails *d = nsdetails( ns );
+ if ( ! d ) {
+ errmsg = "ns not found";
+ return false;
}
- BSONObj min = minBuilder.obj();
- BSONObj max = maxBuilder.obj();
IndexDetails *idx = cmdIndexDetailsForRange( ns , errmsg , min , max , keyPattern );
- if ( idx == NULL ){
+ if ( idx == NULL ) {
errmsg = "couldn't find index over splitting key";
return false;
}
- NamespaceDetails *d = nsdetails( ns );
- BtreeCursor c( d , d->idxNo(*idx) , *idx , min , max , false , 1 );
+ BtreeCursor * bc = new BtreeCursor( d , d->idxNo(*idx) , *idx , min , max , false , 1 );
+ shared_ptr<Cursor> c( bc );
+ scoped_ptr<ClientCursor> cc( new ClientCursor( QueryOption_NoCursorTimeout , c , ns ) );
+ if ( ! cc->ok() ) {
+ // range is empty
+ return true;
+ }
- // We'll use the average object size and number of object to find approximately how many keys
- // each chunk should have. We'll split a little smaller than the specificied by 'maxSize'
- // assuming a recently sharded collectio is still going to grow.
+ // for now, the only check is that all shard keys are filled
+ // null is ok,
+ // TODO if $exist for nulls were picking the index, it could be used instead efficiently
+ while ( cc->ok() ) {
+ BSONObj currKey = c->currKey();
+
+ BSONObjIterator i( currKey );
+ int n = 0;
+ while ( i.more() ) {
+ BSONElement key = i.next();
+ n++;
- const long long dataSize = d->datasize;
- const long long recCount = d->nrecords;
- long long keyCount = 0;
- if (( dataSize > 0 ) && ( recCount > 0 )){
- const long long avgRecSize = dataSize / recCount;
- keyCount = 90 * maxChunkSize / (100 * avgRecSize);
+ if ( key.type() && key.type() != jstNULL )
+ continue;
+
+ BSONObj obj = c->current();
+ BSONObjIterator j( keyPattern );
+ BSONElement real;
+ for ( int x=0; x<n; x++ )
+ real = j.next();
+
+ real = obj.getFieldDotted( real.fieldName() );
+
+ if ( real.type() )
+ continue;
+
+ ostringstream os;
+ os << "found null value in key " << bc->prettyKey( currKey ) << " for doc: " << real["_id"];
+ log() << "checkShardingIndex for '" << ns << "' failed: " << os.str() << endl;
+
+ errmsg = os.str();
+ return false;
+ }
+ cc->advance();
}
- // We traverse the index and add the keyCount-th key to the result vector. If that key
- // appeared in the vector before, we omit it. The assumption here is that all the
- // instances of a key value live in the same chunk.
+ return true;
+ }
+ } cmdCheckShardingIndex;
- Timer timer;
- long long currCount = 0;
- vector<BSONObj> splitKeys;
- BSONObj currKey;
- while ( c.ok() ){
- currCount++;
- if ( currCount > keyCount ){
- if ( ! currKey.isEmpty() && (currKey.woCompare( c.currKey() ) == 0 ) )
- continue;
-
- currKey = c.currKey();
- splitKeys.push_back( c.prettyKey( currKey ) );
- currCount = 0;
+ class SplitVector : public Command {
+ public:
+ SplitVector() : Command( "splitVector" , false ) {}
+ virtual bool slaveOk() const { return false; }
+ virtual LockType locktype() const { return READ; }
+ virtual void help( stringstream &help ) const {
+ help <<
+ "Internal command.\n"
+ "examples:\n"
+ " { splitVector : \"blog.post\" , keyPattern:{x:1} , min:{x:10} , max:{x:20}, maxChunkSize:200 }\n"
+ " maxChunkSize unit in MBs\n"
+ " May optionally specify 'maxSplitPoints' and 'maxChunkObjects' to avoid traversing the whole chunk\n"
+ " \n"
+ " { splitVector : \"blog.post\" , keyPattern:{x:1} , min:{x:10} , max:{x:20}, force: true }\n"
+ " 'force' will produce one split point even if data is small; defaults to false\n"
+ "NOTE: This command may take a while to run";
+ }
+
+ bool run(const string& dbname, BSONObj& jsobj, string& errmsg, BSONObjBuilder& result, bool fromRepl ) {
+
+ //
+ // 1.a We'll parse the parameters in two steps. First, make sure the we can use the split index to get
+ // a good approximation of the size of the chunk -- without needing to access the actual data.
+ //
+
+ const char* ns = jsobj.getStringField( "splitVector" );
+ BSONObj keyPattern = jsobj.getObjectField( "keyPattern" );
+
+ // If min and max are not provided use the "minKey" and "maxKey" for the sharding key pattern.
+ BSONObj min = jsobj.getObjectField( "min" );
+ BSONObj max = jsobj.getObjectField( "max" );
+ if ( min.isEmpty() && max.isEmpty() ) {
+ BSONObjBuilder minBuilder;
+ BSONObjBuilder maxBuilder;
+ BSONForEach(key, keyPattern) {
+ minBuilder.appendMinKey( key.fieldName() );
+ maxBuilder.appendMaxKey( key.fieldName() );
}
- c.advance();
+ min = minBuilder.obj();
+ max = maxBuilder.obj();
+ }
+ else if ( min.isEmpty() || max.isEmpty() ) {
+ errmsg = "either provide both min and max or leave both empty";
+ return false;
}
- ostringstream os;
- os << "Finding the split vector for " << ns << " over "<< keyPattern;
- logIfSlow( timer , os.str() );
+ long long maxSplitPoints = 0;
+ BSONElement maxSplitPointsElem = jsobj[ "maxSplitPoints" ];
+ if ( maxSplitPointsElem.isNumber() ) {
+ maxSplitPoints = maxSplitPointsElem.numberLong();
+ }
- // Warning: we are sending back an array of keys but are currently limited to
- // 4MB work of 'result' size. This should be okay for now.
+ long long maxChunkObjects = 0;
+ BSONElement MaxChunkObjectsElem = jsobj[ "maxChunkObjects" ];
+ if ( MaxChunkObjectsElem.isNumber() ) {
+ maxChunkObjects = MaxChunkObjectsElem.numberLong();
+ }
+
+ vector<BSONObj> splitKeys;
+
+ {
+ // Get the size estimate for this namespace
+ Client::Context ctx( ns );
+ NamespaceDetails *d = nsdetails( ns );
+ if ( ! d ) {
+ errmsg = "ns not found";
+ return false;
+ }
+
+ IndexDetails *idx = cmdIndexDetailsForRange( ns , errmsg , min , max , keyPattern );
+ if ( idx == NULL ) {
+ errmsg = "couldn't find index over splitting key";
+ return false;
+ }
+
+ const long long recCount = d->stats.nrecords;
+ const long long dataSize = d->stats.datasize;
+
+ //
+ // 1.b Now that we have the size estimate, go over the remaining parameters and apply any maximum size
+ // restrictions specified there.
+ //
+
+ // 'force'-ing a split is equivalent to having maxChunkSize be the size of the current chunk, i.e., the
+ // logic below will split that chunk in half
+ long long maxChunkSize = 0;
+ bool force = false;
+ {
+ BSONElement maxSizeElem = jsobj[ "maxChunkSize" ];
+ BSONElement forceElem = jsobj[ "force" ];
+
+ if ( forceElem.trueValue() ) {
+ force = true;
+ maxChunkSize = dataSize;
+
+ }
+ else if ( maxSizeElem.isNumber() ) {
+ maxChunkSize = maxSizeElem.numberLong() * 1<<20;
+
+ }
+ else {
+ maxSizeElem = jsobj["maxChunkSizeBytes"];
+ if ( maxSizeElem.isNumber() ) {
+ maxChunkSize = maxSizeElem.numberLong();
+ }
+ }
+
+ if ( maxChunkSize <= 0 ) {
+ errmsg = "need to specify the desired max chunk size (maxChunkSize or maxChunkSizeBytes)";
+ return false;
+ }
+ }
+
+
+ // If there's not enough data for more than one chunk, no point continuing.
+ if ( dataSize < maxChunkSize || recCount == 0 ) {
+ vector<BSONObj> emptyVector;
+ result.append( "splitKeys" , emptyVector );
+ return true;
+ }
+
+ log() << "request split points lookup for chunk " << ns << " " << min << " -->> " << max << endl;
+
+ // We'll use the average object size and number of object to find approximately how many keys
+ // each chunk should have. We'll split at half the maxChunkSize or maxChunkObjects, if
+ // provided.
+ const long long avgRecSize = dataSize / recCount;
+ long long keyCount = maxChunkSize / (2 * avgRecSize);
+ if ( maxChunkObjects && ( maxChunkObjects < keyCount ) ) {
+ log() << "limiting split vector to " << maxChunkObjects << " (from " << keyCount << ") objects " << endl;
+ keyCount = maxChunkObjects;
+ }
+
+ //
+ // 2. Traverse the index and add the keyCount-th key to the result vector. If that key
+ // appeared in the vector before, we omit it. The invariant here is that all the
+ // instances of a given key value live in the same chunk.
+ //
+
+ Timer timer;
+ long long currCount = 0;
+ long long numChunks = 0;
+
+ BtreeCursor * bc = new BtreeCursor( d , d->idxNo(*idx) , *idx , min , max , false , 1 );
+ shared_ptr<Cursor> c( bc );
+ scoped_ptr<ClientCursor> cc( new ClientCursor( QueryOption_NoCursorTimeout , c , ns ) );
+ if ( ! cc->ok() ) {
+ errmsg = "can't open a cursor for splitting (desired range is possibly empty)";
+ return false;
+ }
+
+ // Use every 'keyCount'-th key as a split point. We add the initial key as a sentinel, to be removed
+ // at the end. If a key appears more times than entries allowed on a chunk, we issue a warning and
+ // split on the following key.
+ set<BSONObj> tooFrequentKeys;
+ splitKeys.push_back( c->currKey().getOwned() );
+ while ( 1 ) {
+ while ( cc->ok() ) {
+ currCount++;
+ BSONObj currKey = c->currKey();
+
+ DEV assert( currKey.woCompare( max ) <= 0 );
+
+ if ( currCount > keyCount ) {
+ // Do not use this split key if it is the same used in the previous split point.
+ if ( currKey.woCompare( splitKeys.back() ) == 0 ) {
+ tooFrequentKeys.insert( currKey.getOwned() );
+
+ }
+ else {
+ splitKeys.push_back( currKey.getOwned() );
+ currCount = 0;
+ numChunks++;
+
+ LOG(4) << "picked a split key: " << bc->prettyKey( currKey ) << endl;
+ }
+
+ }
+
+ cc->advance();
+
+ // Stop if we have enough split points.
+ if ( maxSplitPoints && ( numChunks >= maxSplitPoints ) ) {
+ log() << "max number of requested split points reached (" << numChunks
+ << ") before the end of chunk " << ns << " " << min << " -->> " << max
+ << endl;
+ break;
+ }
+
+ if ( ! cc->yieldSometimes() ) {
+ // we were near and and got pushed to the end
+ // i think returning the splits we've already found is fine
+
+ // don't use the btree cursor pointer to acces keys beyond this point but ok
+ // to use it for format the keys we've got already
+
+ break;
+ }
+ }
+
+ if ( splitKeys.size() > 1 || ! force )
+ break;
+
+ force = false;
+ keyCount = currCount / 2;
+ currCount = 0;
+ log() << "splitVector doing another cycle because of force, keyCount now: " << keyCount << endl;
+
+ c.reset( new BtreeCursor( d , d->idxNo(*idx) , *idx , min , max , false , 1 ) );
+ cc.reset( new ClientCursor( QueryOption_NoCursorTimeout , c , ns ) );
+ }
+
+ //
+ // 3. Format the result and issue any warnings about the data we gathered while traversing the
+ // index
+ //
+
+ // Warn for keys that are more numerous than maxChunkSize allows.
+ for ( set<BSONObj>::const_iterator it = tooFrequentKeys.begin(); it != tooFrequentKeys.end(); ++it ) {
+ warning() << "chunk is larger than " << maxChunkSize
+ << " bytes because of key " << bc->prettyKey( *it ) << endl;
+ }
+
+ // Remove the sentinel at the beginning before returning and add fieldnames.
+ splitKeys.erase( splitKeys.begin() );
+ for ( vector<BSONObj>::iterator it = splitKeys.begin(); it != splitKeys.end() ; ++it ) {
+ *it = bc->prettyKey( *it );
+ }
+
+ if ( timer.millis() > cmdLine.slowMS ) {
+ warning() << "Finding the split vector for " << ns << " over "<< keyPattern
+ << " keyCount: " << keyCount << " numSplits: " << splitKeys.size()
+ << " lookedAt: " << currCount << " took " << timer.millis() << "ms"
+ << endl;
+ }
+
+ // Warning: we are sending back an array of keys but are currently limited to
+ // 4MB work of 'result' size. This should be okay for now.
+
+ }
result.append( "splitKeys" , splitKeys );
+
return true;
}
} cmdSplitVector;
+ // ** temporary ** 2010-10-22
+ // chunkInfo is a helper to collect and log information about the chunks generated in splitChunk.
+ // It should hold the chunk state for this module only, while we don't have min/max key info per chunk on the
+ // mongod side. Do not build on this; it will go away.
+ struct ChunkInfo {
+ BSONObj min;
+ BSONObj max;
+ ShardChunkVersion lastmod;
+
+ ChunkInfo() { }
+ ChunkInfo( BSONObj aMin , BSONObj aMax , ShardChunkVersion aVersion ) : min(aMin) , max(aMax) , lastmod(aVersion) {}
+ void appendShortVersion( const char* name, BSONObjBuilder& b ) const;
+ string toString() const;
+ };
+
+ void ChunkInfo::appendShortVersion( const char * name , BSONObjBuilder& b ) const {
+ BSONObjBuilder bb( b.subobjStart( name ) );
+ bb.append( "min" , min );
+ bb.append( "max" , max );
+ bb.appendTimestamp( "lastmod" , lastmod );
+ bb.done();
+ }
+
+ string ChunkInfo::toString() const {
+ ostringstream os;
+ os << "lastmod: " << lastmod.toString() << " min: " << min << " max: " << endl;
+ return os.str();
+ }
+ // ** end temporary **
+
+ class SplitChunkCommand : public Command {
+ public:
+ SplitChunkCommand() : Command( "splitChunk" ) {}
+ virtual void help( stringstream& help ) const {
+ help <<
+ "internal command usage only\n"
+ "example:\n"
+ " { splitChunk:\"db.foo\" , keyPattern: {a:1} , min : {a:100} , max: {a:200} { splitKeys : [ {a:150} , ... ]}";
+ }
+
+ virtual bool slaveOk() const { return false; }
+ virtual bool adminOnly() const { return true; }
+ virtual LockType locktype() const { return NONE; }
+
+ bool run(const string& dbname, BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool fromRepl ) {
+
+ //
+ // 1. check whether parameters passed to splitChunk are sound
+ //
+
+ const string ns = cmdObj.firstElement().str();
+ if ( ns.empty() ) {
+ errmsg = "need to specify namespace in command";
+ return false;
+ }
+
+ BSONObj keyPattern = cmdObj["keyPattern"].Obj();
+ if ( keyPattern.isEmpty() ) {
+ errmsg = "need to specify the key pattern the collection is sharded over";
+ return false;
+ }
+
+ BSONObj min = cmdObj["min"].Obj();
+ if ( min.isEmpty() ) {
+ errmsg = "neet to specify the min key for the chunk";
+ return false;
+ }
+
+ BSONObj max = cmdObj["max"].Obj();
+ if ( max.isEmpty() ) {
+ errmsg = "neet to specify the max key for the chunk";
+ return false;
+ }
+
+ string from = cmdObj["from"].str();
+ if ( from.empty() ) {
+ errmsg = "need specify server to split chunk at";
+ return false;
+ }
+
+ BSONObj splitKeysElem = cmdObj["splitKeys"].Obj();
+ if ( splitKeysElem.isEmpty() ) {
+ errmsg = "need to provide the split points to chunk over";
+ return false;
+ }
+ vector<BSONObj> splitKeys;
+ BSONObjIterator it( splitKeysElem );
+ while ( it.more() ) {
+ splitKeys.push_back( it.next().Obj().getOwned() );
+ }
+
+ BSONElement shardId = cmdObj["shardId"];
+ if ( shardId.eoo() ) {
+ errmsg = "need to provide shardId";
+ return false;
+ }
+
+ // It is possible that this is the first sharded command this mongod is asked to perform. If so,
+ // start sharding apparatus. We'd still be missing some more shard-related info but we'll get it
+ // in step 2. below.
+ if ( ! shardingState.enabled() ) {
+ if ( cmdObj["configdb"].type() != String ) {
+ errmsg = "sharding not enabled";
+ return false;
+ }
+ string configdb = cmdObj["configdb"].String();
+ shardingState.enable( configdb );
+ configServer.init( configdb );
+ }
+
+ Shard myShard( from );
+
+ log() << "received splitChunk request: " << cmdObj << endl;
+
+ //
+ // 2. lock the collection's metadata and get highest version for the current shard
+ //
+
+ DistributedLock lockSetup( ConnectionString( shardingState.getConfigServer() , ConnectionString::SYNC) , ns );
+ dist_lock_try dlk( &lockSetup, string("split-") + min.toString() );
+ if ( ! dlk.got() ) {
+ errmsg = "the collection's metadata lock is taken";
+ result.append( "who" , dlk.other() );
+ return false;
+ }
+
+ // TODO This is a check migrate does to the letter. Factor it out and share. 2010-10-22
+
+ ShardChunkVersion maxVersion;
+ string shard;
+ ChunkInfo origChunk;
+ {
+ ScopedDbConnection conn( shardingState.getConfigServer() );
+
+ BSONObj x = conn->findOne( ShardNS::chunk , Query( BSON( "ns" << ns ) ).sort( BSON( "lastmod" << -1 ) ) );
+ maxVersion = x["lastmod"];
+
+ BSONObj currChunk = conn->findOne( ShardNS::chunk , shardId.wrap( "_id" ) ).getOwned();
+ assert( currChunk["shard"].type() );
+ assert( currChunk["min"].type() );
+ assert( currChunk["max"].type() );
+ shard = currChunk["shard"].String();
+ conn.done();
+
+ BSONObj currMin = currChunk["min"].Obj();
+ BSONObj currMax = currChunk["max"].Obj();
+ if ( currMin.woCompare( min ) || currMax.woCompare( max ) ) {
+ errmsg = "chunk boundaries are outdated (likely a split occurred)";
+ result.append( "currMin" , currMin );
+ result.append( "currMax" , currMax );
+ result.append( "requestedMin" , min );
+ result.append( "requestedMax" , max );
+
+ log( LL_WARNING ) << "aborted split because " << errmsg << ": " << min << "->" << max
+ << " is now " << currMin << "->" << currMax << endl;
+ return false;
+ }
+
+ if ( shard != myShard.getName() ) {
+ errmsg = "location is outdated (likely balance or migrate occurred)";
+ result.append( "from" , myShard.getName() );
+ result.append( "official" , shard );
+
+ log( LL_WARNING ) << "aborted split because " << errmsg << ": chunk is at " << shard
+ << " and not at " << myShard.getName() << endl;
+ return false;
+ }
+
+ if ( maxVersion < shardingState.getVersion( ns ) ) {
+ errmsg = "official version less than mine?";
+ result.appendTimestamp( "officialVersion" , maxVersion );
+ result.appendTimestamp( "myVersion" , shardingState.getVersion( ns ) );
+
+ log( LL_WARNING ) << "aborted split because " << errmsg << ": official " << maxVersion
+ << " mine: " << shardingState.getVersion(ns) << endl;
+ return false;
+ }
+
+ origChunk.min = currMin.getOwned();
+ origChunk.max = currMax.getOwned();
+ origChunk.lastmod = currChunk["lastmod"];
+
+ // since this could be the first call that enable sharding we also make sure to have the chunk manager up to date
+ shardingState.gotShardName( shard );
+ ShardChunkVersion shardVersion;
+ shardingState.trySetVersion( ns , shardVersion /* will return updated */ );
+
+ log() << "splitChunk accepted at version " << shardVersion << endl;
+
+ }
+
+ //
+ // 3. create the batch of updates to metadata ( the new chunks ) to be applied via 'applyOps' command
+ //
+
+ BSONObjBuilder logDetail;
+ origChunk.appendShortVersion( "before" , logDetail );
+ log(1) << "before split on " << origChunk << endl;
+ vector<ChunkInfo> newChunks;
+
+ ShardChunkVersion myVersion = maxVersion;
+ BSONObj startKey = min;
+ splitKeys.push_back( max ); // makes it easier to have 'max' in the next loop. remove later.
+
+ BSONObjBuilder cmdBuilder;
+ BSONArrayBuilder updates( cmdBuilder.subarrayStart( "applyOps" ) );
+
+ for ( vector<BSONObj>::const_iterator it = splitKeys.begin(); it != splitKeys.end(); ++it ) {
+ BSONObj endKey = *it;
+
+ // splits only update the 'minor' portion of version
+ myVersion.incMinor();
+
+ // build an update operation against the chunks collection of the config database with
+ // upsert true
+ BSONObjBuilder op;
+ op.append( "op" , "u" );
+ op.appendBool( "b" , true );
+ op.append( "ns" , ShardNS::chunk );
+
+ // add the modified (new) chunk infomation as the update object
+ BSONObjBuilder n( op.subobjStart( "o" ) );
+ n.append( "_id" , Chunk::genID( ns , startKey ) );
+ n.appendTimestamp( "lastmod" , myVersion );
+ n.append( "ns" , ns );
+ n.append( "min" , startKey );
+ n.append( "max" , endKey );
+ n.append( "shard" , shard );
+ n.done();
+
+ // add the chunk's _id as the query part of the update statement
+ BSONObjBuilder q( op.subobjStart( "o2" ) );
+ q.append( "_id" , Chunk::genID( ns , startKey ) );
+ q.done();
+
+ updates.append( op.obj() );
+
+ // remember this chunk info for logging later
+ newChunks.push_back( ChunkInfo( startKey , endKey, myVersion ) );
+
+ startKey = endKey;
+ }
+
+ updates.done();
+
+ {
+ BSONArrayBuilder preCond( cmdBuilder.subarrayStart( "preCondition" ) );
+ BSONObjBuilder b;
+ b.append( "ns" , ShardNS::chunk );
+ b.append( "q" , BSON( "query" << BSON( "ns" << ns ) << "orderby" << BSON( "lastmod" << -1 ) ) );
+ {
+ BSONObjBuilder bb( b.subobjStart( "res" ) );
+ bb.appendTimestamp( "lastmod" , maxVersion );
+ bb.done();
+ }
+ preCond.append( b.obj() );
+ preCond.done();
+ }
+
+ //
+ // 4. apply the batch of updates to metadata and to the chunk manager
+ //
+
+ BSONObj cmd = cmdBuilder.obj();
+
+ LOG(1) << "splitChunk update: " << cmd << endl;
+
+ bool ok;
+ BSONObj cmdResult;
+ {
+ ScopedDbConnection conn( shardingState.getConfigServer() );
+ ok = conn->runCommand( "config" , cmd , cmdResult );
+ conn.done();
+ }
+
+ if ( ! ok ) {
+ stringstream ss;
+ ss << "saving chunks failed. cmd: " << cmd << " result: " << cmdResult;
+ error() << ss.str() << endl;
+ msgasserted( 13593 , ss.str() ); // assert(13593)
+ }
+
+ // install a chunk manager with knowledge about newly split chunks in this shard's state
+ splitKeys.pop_back(); // 'max' was used as sentinel
+ maxVersion.incMinor();
+ shardingState.splitChunk( ns , min , max , splitKeys , maxVersion );
+
+ //
+ // 5. logChanges
+ //
+
+ // single splits are logged different than multisplits
+ if ( newChunks.size() == 2 ) {
+ newChunks[0].appendShortVersion( "left" , logDetail );
+ newChunks[1].appendShortVersion( "right" , logDetail );
+ configServer.logChange( "split" , ns , logDetail.obj() );
+
+ }
+ else {
+ BSONObj beforeDetailObj = logDetail.obj();
+ BSONObj firstDetailObj = beforeDetailObj.getOwned();
+ const int newChunksSize = newChunks.size();
+
+ for ( int i=0; i < newChunksSize; i++ ) {
+ BSONObjBuilder chunkDetail;
+ chunkDetail.appendElements( beforeDetailObj );
+ chunkDetail.append( "number", i );
+ chunkDetail.append( "of" , newChunksSize );
+ newChunks[i].appendShortVersion( "chunk" , chunkDetail );
+ configServer.logChange( "multi-split" , ns , chunkDetail.obj() );
+ }
+ }
+
+ return true;
+ }
+ } cmdSplitChunk;
+
} // namespace mongo
diff --git a/s/d_state.cpp b/s/d_state.cpp
index 3f13b79..11fbcef 100644
--- a/s/d_state.cpp
+++ b/s/d_state.cpp
@@ -1,4 +1,4 @@
-// d_state.cpp
+// @file d_state.cpp
/**
* Copyright (C) 2008 10gen Inc.
@@ -44,12 +44,12 @@ using namespace std;
namespace mongo {
// -----ShardingState START ----
-
+
ShardingState::ShardingState()
- : _enabled(false) , _mutex( "ShardingState" ){
+ : _enabled(false) , _mutex( "ShardingState" ) {
}
-
- void ShardingState::enable( const string& server ){
+
+ void ShardingState::enable( const string& server ) {
_enabled = true;
assert( server.size() );
if ( _configServer.size() == 0 )
@@ -58,69 +58,177 @@ namespace mongo {
assert( server == _configServer );
}
}
-
- void ShardingState::gotShardName( const string& name ){
- if ( _shardName.size() == 0 ){
+
+ void ShardingState::gotShardName( const string& name ) {
+ scoped_lock lk(_mutex);
+ if ( _shardName.size() == 0 ) {
+ // TODO SERVER-2299 verify the name is sound w.r.t IPs
_shardName = name;
return;
}
-
+
if ( _shardName == name )
return;
stringstream ss;
- ss << "gotShardName different than what i had before "
- << " before [" << _shardName << "] "
- << " got [" << name << "] "
- ;
+ ss << "gotShardName different than what i had before "
+ << " before [" << _shardName << "] "
+ << " got [" << name << "] "
+ ;
uasserted( 13298 , ss.str() );
}
-
- void ShardingState::gotShardHost( const string& host ){
- if ( _shardHost.size() == 0 ){
+
+ void ShardingState::gotShardHost( string host ) {
+ scoped_lock lk(_mutex);
+ size_t slash = host.find( '/' );
+ if ( slash != string::npos )
+ host = host.substr( 0 , slash );
+
+ if ( _shardHost.size() == 0 ) {
_shardHost = host;
return;
}
-
+
if ( _shardHost == host )
return;
stringstream ss;
- ss << "gotShardHost different than what i had before "
- << " before [" << _shardHost << "] "
- << " got [" << host << "] "
- ;
+ ss << "gotShardHost different than what i had before "
+ << " before [" << _shardHost << "] "
+ << " got [" << host << "] "
+ ;
uasserted( 13299 , ss.str() );
}
-
- bool ShardingState::hasVersion( const string& ns ){
+
+ void ShardingState::resetShardingState() {
scoped_lock lk(_mutex);
- NSVersionMap::const_iterator i = _versions.find(ns);
- return i != _versions.end();
+
+ _enabled = false;
+ _configServer.clear();
+ _shardName.clear();
+ _shardHost.clear();
+ _chunks.clear();
}
-
- bool ShardingState::hasVersion( const string& ns , ConfigVersion& version ){
+
+ // TODO we shouldn't need three ways for checking the version. Fix this.
+ bool ShardingState::hasVersion( const string& ns ) {
scoped_lock lk(_mutex);
- NSVersionMap::const_iterator i = _versions.find(ns);
- if ( i == _versions.end() )
+
+ ChunkManagersMap::const_iterator it = _chunks.find(ns);
+ return it != _chunks.end();
+ }
+
+ bool ShardingState::hasVersion( const string& ns , ConfigVersion& version ) {
+ scoped_lock lk(_mutex);
+
+ ChunkManagersMap::const_iterator it = _chunks.find(ns);
+ if ( it == _chunks.end() )
return false;
- version = i->second;
+
+ ShardChunkManagerPtr p = it->second;
+ version = p->getVersion();
return true;
}
-
- ConfigVersion& ShardingState::getVersion( const string& ns ){
+
+ const ConfigVersion ShardingState::getVersion( const string& ns ) const {
scoped_lock lk(_mutex);
- return _versions[ns];
+
+ ChunkManagersMap::const_iterator it = _chunks.find( ns );
+ if ( it != _chunks.end() ) {
+ ShardChunkManagerPtr p = it->second;
+ return p->getVersion();
+ }
+ else {
+ return 0;
+ }
}
-
- void ShardingState::setVersion( const string& ns , const ConfigVersion& version ){
- scoped_lock lk(_mutex);
- ConfigVersion& me = _versions[ns];
- assert( version == 0 || version > me );
- me = version;
+
+ void ShardingState::donateChunk( const string& ns , const BSONObj& min , const BSONObj& max , ShardChunkVersion version ) {
+ scoped_lock lk( _mutex );
+
+ ChunkManagersMap::const_iterator it = _chunks.find( ns );
+ assert( it != _chunks.end() ) ;
+ ShardChunkManagerPtr p = it->second;
+
+ // empty shards should have version 0
+ version = ( p->getNumChunks() > 1 ) ? version : ShardChunkVersion( 0 , 0 );
+
+ ShardChunkManagerPtr cloned( p->cloneMinus( min , max , version ) );
+ _chunks[ns] = cloned;
+ }
+
+ void ShardingState::undoDonateChunk( const string& ns , const BSONObj& min , const BSONObj& max , ShardChunkVersion version ) {
+ scoped_lock lk( _mutex );
+
+ ChunkManagersMap::const_iterator it = _chunks.find( ns );
+ assert( it != _chunks.end() ) ;
+ ShardChunkManagerPtr p( it->second->clonePlus( min , max , version ) );
+ _chunks[ns] = p;
+ }
+
+ void ShardingState::splitChunk( const string& ns , const BSONObj& min , const BSONObj& max , const vector<BSONObj>& splitKeys ,
+ ShardChunkVersion version ) {
+ scoped_lock lk( _mutex );
+
+ ChunkManagersMap::const_iterator it = _chunks.find( ns );
+ assert( it != _chunks.end() ) ;
+ ShardChunkManagerPtr p( it->second->cloneSplit( min , max , splitKeys , version ) );
+ _chunks[ns] = p;
}
- void ShardingState::appendInfo( BSONObjBuilder& b ){
+ void ShardingState::resetVersion( const string& ns ) {
+ scoped_lock lk( _mutex );
+
+ _chunks.erase( ns );
+ }
+
+ bool ShardingState::trySetVersion( const string& ns , ConfigVersion& version /* IN-OUT */ ) {
+
+ // fast path - requested version is at the same version as this chunk manager
+ //
+ // cases:
+ // + this shard updated the version for a migrate's commit (FROM side)
+ // a client reloaded chunk state from config and picked the newest version
+ // + two clients reloaded
+ // one triggered the 'slow path' (below)
+ // when the second's request gets here, the version is already current
+ {
+ scoped_lock lk( _mutex );
+ ChunkManagersMap::const_iterator it = _chunks.find( ns );
+ if ( it != _chunks.end() && it->second->getVersion() == version )
+ return true;
+ }
+
+ // slow path - requested version is different than the current chunk manager's, if one exists, so must check for
+ // newest version in the config server
+ //
+ // cases:
+ // + a chunk moved TO here
+ // (we don't bump up the version on the TO side but the commit to config does use higher version)
+ // a client reloads from config an issued the request
+ // + there was a take over from a secondary
+ // the secondary had no state (managers) at all, so every client request will fall here
+ // + a stale client request a version that's not current anymore
+
+ const string c = (_configServer == _shardHost) ? "" /* local */ : _configServer;
+ ShardChunkManagerPtr p( new ShardChunkManager( c , ns , _shardName ) );
+ {
+ scoped_lock lk( _mutex );
+
+ // since we loaded the chunk manager unlocked, other thread may have done the same
+ // make sure we keep the freshest config info only
+ ChunkManagersMap::const_iterator it = _chunks.find( ns );
+ if ( it == _chunks.end() || p->getVersion() >= it->second->getVersion() ) {
+ _chunks[ns] = p;
+ }
+
+ ShardChunkVersion oldVersion = version;
+ version = p->getVersion();
+ return oldVersion == version;
+ }
+ }
+
+ void ShardingState::appendInfo( BSONObjBuilder& b ) {
b.appendBool( "enabled" , _enabled );
if ( ! _enabled )
return;
@@ -131,117 +239,56 @@ namespace mongo {
{
BSONObjBuilder bb( b.subobjStart( "versions" ) );
-
+
scoped_lock lk(_mutex);
- for ( NSVersionMap::iterator i=_versions.begin(); i!=_versions.end(); ++i ){
- bb.appendTimestamp( i->first.c_str() , i->second );
+
+ for ( ChunkManagersMap::iterator it = _chunks.begin(); it != _chunks.end(); ++it ) {
+ ShardChunkManagerPtr p = it->second;
+ bb.appendTimestamp( it->first , p->getVersion() );
}
bb.done();
}
}
- ChunkMatcherPtr ShardingState::getChunkMatcher( const string& ns ){
+ bool ShardingState::needShardChunkManager( const string& ns ) const {
if ( ! _enabled )
- return ChunkMatcherPtr();
-
- if ( ! ShardedConnectionInfo::get( false ) )
- return ChunkMatcherPtr();
+ return false;
- ConfigVersion version;
- {
- scoped_lock lk( _mutex );
- version = _versions[ns];
-
- if ( ! version )
- return ChunkMatcherPtr();
-
- ChunkMatcherPtr p = _chunks[ns];
- if ( p && p->_version >= version )
- return p;
- }
+ if ( ! ShardedConnectionInfo::get( false ) )
+ return false;
- BSONObj q;
- {
- BSONObjBuilder b;
- b.append( "ns" , ns.c_str() );
- b.append( "shard" , BSON( "$in" << BSON_ARRAY( _shardHost << _shardName ) ) );
- q = b.obj();
- }
+ return true;
+ }
- auto_ptr<ScopedDbConnection> scoped;
- auto_ptr<DBDirectClient> direct;
-
- DBClientBase * conn;
+ ShardChunkManagerPtr ShardingState::getShardChunkManager( const string& ns ) {
+ scoped_lock lk( _mutex );
- if ( _configServer == _shardHost ){
- direct.reset( new DBDirectClient() );
- conn = direct.get();
+ ChunkManagersMap::const_iterator it = _chunks.find( ns );
+ if ( it == _chunks.end() ) {
+ return ShardChunkManagerPtr();
}
else {
- scoped.reset( new ScopedDbConnection( _configServer ) );
- conn = scoped->get();
+ return it->second;
}
-
- auto_ptr<DBClientCursor> cursor = conn->query( "config.chunks" , Query(q).sort( "min" ) );
- assert( cursor.get() );
- if ( ! cursor->more() ){
- if ( scoped.get() )
- scoped->done();
- return ChunkMatcherPtr();
- }
-
- ChunkMatcherPtr p( new ChunkMatcher( version ) );
-
- BSONObj min,max;
- while ( cursor->more() ){
- BSONObj d = cursor->next();
-
- if ( min.isEmpty() ){
- min = d["min"].Obj().getOwned();
- max = d["max"].Obj().getOwned();
- continue;
- }
-
- if ( max == d["min"].Obj() ){
- max = d["max"].Obj().getOwned();
- continue;
- }
-
- p->gotRange( min.getOwned() , max.getOwned() );
- min = d["min"].Obj().getOwned();
- max = d["max"].Obj().getOwned();
- }
- assert( ! min.isEmpty() );
- p->gotRange( min.getOwned() , max.getOwned() );
-
- if ( scoped.get() )
- scoped->done();
-
- {
- scoped_lock lk( _mutex );
- _chunks[ns] = p;
- }
-
- return p;
}
ShardingState shardingState;
// -----ShardingState END ----
-
+
// -----ShardedConnectionInfo START ----
boost::thread_specific_ptr<ShardedConnectionInfo> ShardedConnectionInfo::_tl;
- ShardedConnectionInfo::ShardedConnectionInfo(){
- _forceMode = false;
+ ShardedConnectionInfo::ShardedConnectionInfo() {
+ _forceVersionOk = false;
_id.clear();
}
-
- ShardedConnectionInfo* ShardedConnectionInfo::get( bool create ){
+
+ ShardedConnectionInfo* ShardedConnectionInfo::get( bool create ) {
ShardedConnectionInfo* info = _tl.get();
- if ( ! info && create ){
+ if ( ! info && create ) {
log(1) << "entering shard mode for connection" << endl;
info = new ShardedConnectionInfo();
_tl.reset( info );
@@ -249,44 +296,50 @@ namespace mongo {
return info;
}
- void ShardedConnectionInfo::reset(){
+ void ShardedConnectionInfo::reset() {
_tl.reset();
}
- ConfigVersion& ShardedConnectionInfo::getVersion( const string& ns ){
- return _versions[ns];
+ const ConfigVersion ShardedConnectionInfo::getVersion( const string& ns ) const {
+ NSVersionMap::const_iterator it = _versions.find( ns );
+ if ( it != _versions.end() ) {
+ return it->second;
+ }
+ else {
+ return 0;
+ }
}
-
- void ShardedConnectionInfo::setVersion( const string& ns , const ConfigVersion& version ){
+
+ void ShardedConnectionInfo::setVersion( const string& ns , const ConfigVersion& version ) {
_versions[ns] = version;
}
- void ShardedConnectionInfo::setID( const OID& id ){
+ void ShardedConnectionInfo::setID( const OID& id ) {
_id = id;
}
// -----ShardedConnectionInfo END ----
- unsigned long long extractVersion( BSONElement e , string& errmsg ){
- if ( e.eoo() ){
+ unsigned long long extractVersion( BSONElement e , string& errmsg ) {
+ if ( e.eoo() ) {
errmsg = "no version";
return 0;
}
-
+
if ( e.isNumber() )
return (unsigned long long)e.number();
-
+
if ( e.type() == Date || e.type() == Timestamp )
return e._numberLong();
-
+
errmsg = "version is not a numeric type";
return 0;
}
class MongodShardCommand : public Command {
public:
- MongodShardCommand( const char * n ) : Command( n ){
+ MongodShardCommand( const char * n ) : Command( n ) {
}
virtual bool slaveOk() const {
return false;
@@ -295,12 +348,12 @@ namespace mongo {
return true;
}
};
-
-
- bool haveLocalShardingInfo( const string& ns ){
+
+
+ bool haveLocalShardingInfo( const string& ns ) {
if ( ! shardingState.enabled() )
return false;
-
+
if ( ! shardingState.hasVersion( ns ) )
return false;
@@ -309,266 +362,332 @@ namespace mongo {
class UnsetShardingCommand : public MongodShardCommand {
public:
- UnsetShardingCommand() : MongodShardCommand("unsetSharding"){}
+ UnsetShardingCommand() : MongodShardCommand("unsetSharding") {}
virtual void help( stringstream& help ) const {
help << " example: { unsetSharding : 1 } ";
}
-
- virtual LockType locktype() const { return NONE; }
-
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+
+ virtual LockType locktype() const { return NONE; }
+
+ virtual bool slaveOk() const { return true; }
+
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
ShardedConnectionInfo::reset();
return true;
- }
-
+ }
+
} unsetShardingCommand;
-
class SetShardVersion : public MongodShardCommand {
public:
- SetShardVersion() : MongodShardCommand("setShardVersion"){}
+ SetShardVersion() : MongodShardCommand("setShardVersion") {}
virtual void help( stringstream& help ) const {
help << " example: { setShardVersion : 'alleyinsider.foo' , version : 1 , configdb : '' } ";
}
-
- virtual LockType locktype() const { return WRITE; } // TODO: figure out how to make this not need to lock
-
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
- lastError.disableForCommand();
- ShardedConnectionInfo* info = ShardedConnectionInfo::get( true );
-
- bool authoritative = cmdObj.getBoolField( "authoritative" );
- string configdb = cmdObj["configdb"].valuestrsafe();
- { // configdb checking
- if ( configdb.size() == 0 ){
- errmsg = "no configdb";
- return false;
- }
+ virtual LockType locktype() const { return NONE; }
+
+ bool checkConfigOrInit( const string& configdb , bool authoritative , string& errmsg , BSONObjBuilder& result , bool locked=false ) const {
+ if ( configdb.size() == 0 ) {
+ errmsg = "no configdb";
+ return false;
+ }
+
+ if ( shardingState.enabled() ) {
+ if ( configdb == shardingState.getConfigServer() )
+ return true;
- if ( shardingState.enabled() ){
- if ( configdb != shardingState.getConfigServer() ){
- errmsg = "specified a different configdb!";
- return false;
- }
- }
- else {
- if ( ! authoritative ){
- result.appendBool( "need_authoritative" , true );
- errmsg = "first setShardVersion";
- return false;
- }
- shardingState.enable( configdb );
- configServer.init( configdb );
- }
+ result.append( "configdb" , BSON( "stored" << shardingState.getConfigServer() <<
+ "given" << configdb ) );
+ errmsg = "specified a different configdb!";
+ return false;
}
- if ( cmdObj["shard"].type() == String ){
- shardingState.gotShardName( cmdObj["shard"].String() );
- shardingState.gotShardHost( cmdObj["shardHost"].String() );
+ if ( ! authoritative ) {
+ result.appendBool( "need_authoritative" , true );
+ errmsg = "first setShardVersion";
+ return false;
+ }
+
+ if ( locked ) {
+ shardingState.enable( configdb );
+ configServer.init( configdb );
+ return true;
}
- { // setting up ids
- if ( cmdObj["serverID"].type() != jstOID ){
- // TODO: fix this
- //errmsg = "need serverID to be an OID";
- //return 0;
- }
- else {
- OID clientId = cmdObj["serverID"].__oid();
- if ( ! info->hasID() ){
- info->setID( clientId );
- }
- else if ( clientId != info->getID() ){
- errmsg = "server id has changed!";
- return 0;
- }
- }
+ dblock lk;
+ return checkConfigOrInit( configdb , authoritative , errmsg , result , true );
+ }
+
+ bool checkMongosID( ShardedConnectionInfo* info, const BSONElement& id, string errmsg ) {
+ if ( id.type() != jstOID ) {
+ // TODO: fix this
+ //errmsg = "need serverID to be an OID";
+ //return 0;
+ return true;
+ }
+
+ OID clientId = id.__oid();
+ if ( ! info->hasID() ) {
+ info->setID( clientId );
+ return true;
}
- unsigned long long version = extractVersion( cmdObj["version"] , errmsg );
+ if ( clientId != info->getID() ) {
+ errmsg = "server id has changed!";
+ return false;
+ }
+
+ return true;
+ }
+
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
+
+ // Steps
+ // 1. check basic config
+ // 2. extract params from command
+ // 3. fast check
+ // 4. slow check (LOCKS)
+
+ // step 1
- if ( errmsg.size() ){
+ lastError.disableForCommand();
+ ShardedConnectionInfo* info = ShardedConnectionInfo::get( true );
+
+ bool authoritative = cmdObj.getBoolField( "authoritative" );
+
+ // check config server is ok or enable sharding
+ if ( ! checkConfigOrInit( cmdObj["configdb"].valuestrsafe() , authoritative , errmsg , result ) )
return false;
+
+ // check shard name/hosts are correct
+ if ( cmdObj["shard"].type() == String ) {
+ shardingState.gotShardName( cmdObj["shard"].String() );
+ shardingState.gotShardHost( cmdObj["shardHost"].String() );
}
+ // make sure we have the mongos id for writebacks
+ if ( ! checkMongosID( info , cmdObj["serverID"] , errmsg ) )
+ return false;
+
+ // step 2
+
string ns = cmdObj["setShardVersion"].valuestrsafe();
- if ( ns.size() == 0 ){
- errmsg = "need to speciy fully namespace";
+ if ( ns.size() == 0 ) {
+ errmsg = "need to speciy namespace";
return false;
}
+
+ const ConfigVersion version = extractVersion( cmdObj["version"] , errmsg );
+ if ( errmsg.size() )
+ return false;
+
+ // step 3
+
+ const ConfigVersion oldVersion = info->getVersion(ns);
+ const ConfigVersion globalVersion = shardingState.getVersion(ns);
+
+ result.appendTimestamp( "oldVersion" , oldVersion );
- ConfigVersion& oldVersion = info->getVersion(ns);
- ConfigVersion& globalVersion = shardingState.getVersion(ns);
+ if ( globalVersion > 0 && version > 0 ) {
+ // this means there is no reset going on an either side
+ // so its safe to make some assuptions
+
+ if ( version == globalVersion ) {
+ // mongos and mongod agree!
+ if ( oldVersion != version ) {
+ assert( oldVersion < globalVersion );
+ info->setVersion( ns , version );
+ }
+ return true;
+ }
+
+ }
+
+ // step 4
+ dblock setShardVersionLock; // TODO: can we get rid of this??
- if ( oldVersion > 0 && globalVersion == 0 ){
+ if ( oldVersion > 0 && globalVersion == 0 ) {
// this had been reset
- oldVersion = 0;
+ info->setVersion( ns , 0 );
}
- if ( version == 0 && globalVersion == 0 ){
+ if ( version == 0 && globalVersion == 0 ) {
// this connection is cleaning itself
- oldVersion = 0;
- return 1;
+ info->setVersion( ns , 0 );
+ return true;
}
- if ( version == 0 && globalVersion > 0 ){
- if ( ! authoritative ){
+ if ( version == 0 && globalVersion > 0 ) {
+ if ( ! authoritative ) {
result.appendBool( "need_authoritative" , true );
+ result.append( "ns" , ns );
result.appendTimestamp( "globalVersion" , globalVersion );
- result.appendTimestamp( "oldVersion" , oldVersion );
errmsg = "dropping needs to be authoritative";
- return 0;
+ return false;
}
log() << "wiping data for: " << ns << endl;
result.appendTimestamp( "beforeDrop" , globalVersion );
// only setting global version on purpose
// need clients to re-find meta-data
- globalVersion = 0;
- oldVersion = 0;
- return 1;
+ shardingState.resetVersion( ns );
+ info->setVersion( ns , 0 );
+ return true;
}
- if ( version < oldVersion ){
- errmsg = "you already have a newer version";
- result.appendTimestamp( "oldVersion" , oldVersion );
+ if ( version < oldVersion ) {
+ errmsg = "you already have a newer version of collection '" + ns + "'";
+ result.append( "ns" , ns );
result.appendTimestamp( "newVersion" , version );
result.appendTimestamp( "globalVersion" , globalVersion );
return false;
}
-
- if ( version < globalVersion ){
- while ( shardingState.inCriticalMigrateSection() ){
+
+ if ( version < globalVersion ) {
+ while ( shardingState.inCriticalMigrateSection() ) {
dbtemprelease r;
sleepmillis(2);
- log() << "waiting till out of critical section" << endl;
+ OCCASIONALLY log() << "waiting till out of critical section" << endl;
}
- errmsg = "going to older version for global";
+ errmsg = "going to older version for global for collection '" + ns + "'";
+ result.append( "ns" , ns );
result.appendTimestamp( "version" , version );
result.appendTimestamp( "globalVersion" , globalVersion );
return false;
}
-
- if ( globalVersion == 0 && ! cmdObj.getBoolField( "authoritative" ) ){
+
+ if ( globalVersion == 0 && ! authoritative ) {
// need authoritative for first look
- result.appendBool( "need_authoritative" , true );
result.append( "ns" , ns );
- errmsg = "first time for this ns";
+ result.appendBool( "need_authoritative" , true );
+ errmsg = "first time for collection '" + ns + "'";
return false;
}
+ Timer relockTime;
{
dbtemprelease unlock;
- shardingState.getChunkMatcher( ns );
+
+ ShardChunkVersion currVersion = version;
+ if ( ! shardingState.trySetVersion( ns , currVersion ) ) {
+ errmsg = str::stream() << "client version differs from config's for colleciton '" << ns << "'";
+ result.append( "ns" , ns );
+ result.appendTimestamp( "version" , version );
+ result.appendTimestamp( "globalVersion" , currVersion );
+ return false;
+ }
}
+ if ( relockTime.millis() >= ( cmdLine.slowMS - 10 ) ) {
+ log() << "setShardVersion - relocking slow: " << relockTime.millis() << endl;
+ }
+
+ info->setVersion( ns , version );
+ return true;
+ }
- result.appendTimestamp( "oldVersion" , oldVersion );
- oldVersion = version;
- globalVersion = version;
+ } setShardVersionCmd;
- result.append( "ok" , 1 );
- return 1;
- }
-
- } setShardVersion;
-
class GetShardVersion : public MongodShardCommand {
public:
- GetShardVersion() : MongodShardCommand("getShardVersion"){}
+ GetShardVersion() : MongodShardCommand("getShardVersion") {}
virtual void help( stringstream& help ) const {
help << " example: { getShardVersion : 'alleyinsider.foo' } ";
}
-
- virtual LockType locktype() const { return NONE; }
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ virtual LockType locktype() const { return NONE; }
+
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
string ns = cmdObj["getShardVersion"].valuestrsafe();
- if ( ns.size() == 0 ){
+ if ( ns.size() == 0 ) {
errmsg = "need to speciy fully namespace";
return false;
}
-
+
result.append( "configServer" , shardingState.getConfigServer() );
result.appendTimestamp( "global" , shardingState.getVersion(ns) );
-
+
ShardedConnectionInfo* info = ShardedConnectionInfo::get( false );
if ( info )
result.appendTimestamp( "mine" , info->getVersion(ns) );
- else
+ else
result.appendTimestamp( "mine" , 0 );
-
+
return true;
}
-
+
} getShardVersion;
class ShardingStateCmd : public MongodShardCommand {
public:
- ShardingStateCmd() : MongodShardCommand( "shardingState" ){}
+ ShardingStateCmd() : MongodShardCommand( "shardingState" ) {}
virtual LockType locktype() const { return WRITE; } // TODO: figure out how to make this not need to lock
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
shardingState.appendInfo( result );
return true;
}
-
+
} shardingStateCmd;
/**
* @ return true if not in sharded mode
or if version for this client is ok
*/
- bool shardVersionOk( const string& ns , bool isWriteOp , string& errmsg ){
+ bool shardVersionOk( const string& ns , bool isWriteOp , string& errmsg ) {
if ( ! shardingState.enabled() )
return true;
ShardedConnectionInfo* info = ShardedConnectionInfo::get( false );
- if ( ! info ){
+ if ( ! info ) {
// this means the client has nothing sharded
// so this allows direct connections to do whatever they want
// which i think is the correct behavior
return true;
}
-
- if ( info->inForceMode() ){
+
+ if ( info->inForceVersionOkMode() ) {
return true;
}
- ConfigVersion version;
- if ( ! shardingState.hasVersion( ns , version ) ){
+ // TODO
+ // all collections at some point, be sharded or not, will have a version (and a ShardChunkManager)
+ // for now, we remove the sharding state of dropped collection
+ // so delayed request may come in. This has to be fixed.
+ ConfigVersion clientVersion = info->getVersion(ns);
+ ConfigVersion version;
+ if ( ! shardingState.hasVersion( ns , version ) && clientVersion == 0 ) {
return true;
}
- ConfigVersion clientVersion = info->getVersion(ns);
- if ( version == 0 && clientVersion > 0 ){
+ if ( version == 0 && clientVersion > 0 ) {
stringstream ss;
ss << "collection was dropped or this shard no longer valied version: " << version << " clientVersion: " << clientVersion;
errmsg = ss.str();
return false;
}
-
+
if ( clientVersion >= version )
return true;
-
- if ( clientVersion == 0 ){
+
+ if ( clientVersion == 0 ) {
stringstream ss;
ss << "client in sharded mode, but doesn't have version set for this collection: " << ns << " myVersion: " << version;
errmsg = ss.str();
return false;
}
- if ( isWriteOp && version.majorVersion() == clientVersion.majorVersion() ){
- // this means there was just a split
+ if ( version.majorVersion() == clientVersion.majorVersion() ) {
+ // this means there was just a split
// since on a split w/o a migrate this server is ok
- // going to accept write
+ // going to accept
return true;
}
@@ -578,51 +697,4 @@ namespace mongo {
return false;
}
- // --- ChunkMatcher ---
-
- ChunkMatcher::ChunkMatcher( ConfigVersion version )
- : _version( version ){
-
- }
-
- void ChunkMatcher::gotRange( const BSONObj& min , const BSONObj& max ){
- if (_key.isEmpty()){
- BSONObjBuilder b;
-
- BSONForEach(e, min) {
- b.append(e.fieldName(), 1);
- }
-
- _key = b.obj();
- }
-
- //TODO debug mode only?
- assert(min.nFields() == _key.nFields());
- assert(max.nFields() == _key.nFields());
-
- _map[min] = make_pair(min,max);
- }
-
- bool ChunkMatcher::belongsToMe( const BSONObj& key , const DiskLoc& loc ) const {
- if ( _map.size() == 0 )
- return false;
-
- BSONObj x = loc.obj().extractFields(_key);
-
- MyMap::const_iterator a = _map.upper_bound( x );
- if ( a != _map.begin() )
- a--;
-
- bool good = x.woCompare( a->second.first ) >= 0 && x.woCompare( a->second.second ) < 0;
-#if 0
- if ( ! good ){
- cout << "bad: " << x << "\t" << a->second.first << "\t" << x.woCompare( a->second.first ) << "\t" << x.woCompare( a->second.second ) << endl;
- for ( MyMap::const_iterator i=_map.begin(); i!=_map.end(); ++i ){
- cout << "\t" << i->first << "\t" << i->second.first << "\t" << i->second.second << endl;
- }
- }
-#endif
- return good;
- }
-
}
diff --git a/s/d_writeback.cpp b/s/d_writeback.cpp
index a18e5d5..401e0aa 100644
--- a/s/d_writeback.cpp
+++ b/s/d_writeback.cpp
@@ -19,62 +19,105 @@
#include "pch.h"
#include "../db/commands.h"
-#include "../db/jsobj.h"
-#include "../db/dbmessage.h"
-#include "../db/query.h"
-
-#include "../client/connpool.h"
-
#include "../util/queue.h"
-#include "shard.h"
+#include "d_writeback.h"
using namespace std;
namespace mongo {
- map< string , BlockingQueue<BSONObj>* > writebackQueue;
- mongo::mutex writebackQueueLock("sharding:writebackQueueLock");
+ // ---------- WriteBackManager class ----------
+
+ // TODO init at mongod startup
+ WriteBackManager writeBackManager;
+
+ WriteBackManager::WriteBackManager() : _writebackQueueLock("sharding:writebackQueueLock") {
+ }
+
+ WriteBackManager::~WriteBackManager() {
+ }
+
+ void WriteBackManager::queueWriteBack( const string& remote , const BSONObj& o ) {
+ getWritebackQueue( remote )->push( o );
+ }
- BlockingQueue<BSONObj>* getWritebackQueue( const string& remote ){
- scoped_lock lk (writebackQueueLock );
- BlockingQueue<BSONObj>*& q = writebackQueue[remote];
+ BlockingQueue<BSONObj>* WriteBackManager::getWritebackQueue( const string& remote ) {
+ scoped_lock lk ( _writebackQueueLock );
+ BlockingQueue<BSONObj>*& q = _writebackQueues[remote];
if ( ! q )
q = new BlockingQueue<BSONObj>();
return q;
}
-
- void queueWriteBack( const string& remote , const BSONObj& o ){
- getWritebackQueue( remote )->push( o );
+
+ bool WriteBackManager::queuesEmpty() const {
+ scoped_lock lk( _writebackQueueLock );
+ for ( WriteBackQueuesMap::const_iterator it = _writebackQueues.begin(); it != _writebackQueues.end(); ++it ) {
+ const BlockingQueue<BSONObj>* queue = it->second;
+ if (! queue->empty() ) {
+ return false;
+ }
+ }
+ return true;
}
+ // ---------- admin commands ----------
+
// Note, this command will block until there is something to WriteBack
class WriteBackCommand : public Command {
public:
- virtual LockType locktype() const { return NONE; }
+ virtual LockType locktype() const { return NONE; }
virtual bool slaveOk() const { return true; }
virtual bool adminOnly() const { return true; }
-
- WriteBackCommand() : Command( "writebacklisten" ){}
+
+ WriteBackCommand() : Command( "writebacklisten" ) {}
void help(stringstream& h) const { h<<"internal"; }
- bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool){
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
BSONElement e = cmdObj.firstElement();
- if ( e.type() != jstOID ){
+ if ( e.type() != jstOID ) {
errmsg = "need oid as first value";
return 0;
}
-
+
+ // get the command issuer's (a mongos) serverID
const OID id = e.__oid();
- BSONObj z = getWritebackQueue(id.str())->blockingPop();
- log(1) << "WriteBackCommand got : " << z << endl;
-
- result.append( "data" , z );
-
+
+ // the command issuer is blocked awaiting a response
+ // we want to do return at least at every 5 minutes so sockets don't timeout
+ BSONObj z;
+ if ( writeBackManager.getWritebackQueue(id.str())->blockingPop( z, 5 * 60 /* 5 minutes */ ) ) {
+ log(1) << "WriteBackCommand got : " << z << endl;
+ result.append( "data" , z );
+ }
+ else {
+ result.appendBool( "noop" , true );
+ }
+
return true;
}
} writeBackCommand;
-}
+ class WriteBacksQueuedCommand : public Command {
+ public:
+ virtual LockType locktype() const { return NONE; }
+ virtual bool slaveOk() const { return true; }
+ virtual bool adminOnly() const { return true; }
+
+ WriteBacksQueuedCommand() : Command( "writeBacksQueued" ) {}
+
+ void help(stringstream& help) const {
+ help << "Returns whether there are operations in the writeback queue at the time the command was called. "
+ << "This is an internal comand";
+ }
+
+ bool run(const string& , BSONObj& cmdObj, string& errmsg, BSONObjBuilder& result, bool) {
+ result.appendBool( "hasOpsQueued" , ! writeBackManager.queuesEmpty() );
+ return true;
+ }
+
+ } writeBacksQueuedCommand;
+
+} // namespace mongo
diff --git a/s/d_writeback.h b/s/d_writeback.h
new file mode 100644
index 0000000..32f5b1c
--- /dev/null
+++ b/s/d_writeback.h
@@ -0,0 +1,75 @@
+// @file d_writeback.h
+
+/**
+* Copyright (C) 2010 10gen Inc.
+*
+* This program is free software: you can redistribute it and/or modify
+* it under the terms of the GNU Affero General Public License, version 3,
+* as published by the Free Software Foundation.
+*
+* This program is distributed in the hope that it will be useful,
+* but WITHOUT ANY WARRANTY; without even the implied warranty of
+* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+* GNU Affero General Public License for more details.
+*
+* You should have received a copy of the GNU Affero General Public License
+* along with this program. If not, see <http://www.gnu.org/licenses/>.
+*/
+
+#pragma once
+
+#include "../pch.h"
+
+#include "../util/queue.h"
+
+namespace mongo {
+
+ /*
+ * The WriteBackManager keeps one queue of pending operations per mongos. The operations get here
+ * if they were directed to a chunk that is no longer in this mongod server. The operations are
+ * "written back" to the mongos server per its request (command 'writebacklisten').
+ *
+ * The class is thread safe.
+ */
+ class WriteBackManager {
+ public:
+ WriteBackManager();
+ ~WriteBackManager();
+
+ /*
+ * @param remote server ID this operation came from
+ * @param op the operation itself
+ *
+ * Enqueues opeartion 'op' in server 'remote's queue. The operation will be written back to
+ * remote at a later stager.
+ */
+ void queueWriteBack( const string& remote , const BSONObj& op );
+
+ /*
+ * @param remote server ID
+ * @return the queue for operations that came from 'remote'
+ *
+ * Gets access to server 'remote's queue, which is synchronized.
+ */
+ BlockingQueue<BSONObj>* getWritebackQueue( const string& remote );
+
+ /*
+ * @return true if there is no operation queued for write back
+ */
+ bool queuesEmpty() const;
+
+ private:
+ // a map from mongos's serverIDs to queues of "rejected" operations
+ // an operation is rejected if it targets data that does not live on this shard anymore
+ typedef map< string , BlockingQueue<BSONObj>* > WriteBackQueuesMap;
+
+ // '_writebackQueueLock' protects only the map itself, since each queue is syncrhonized.
+ mutable mongo::mutex _writebackQueueLock;
+ WriteBackQueuesMap _writebackQueues;
+
+ };
+
+ // TODO collect global state in a central place and init during startup
+ extern WriteBackManager writeBackManager;
+
+} // namespace mongo
diff --git a/s/dbgrid.vcproj b/s/dbgrid.vcproj
deleted file mode 100644
index 745d84e..0000000
--- a/s/dbgrid.vcproj
+++ /dev/null
@@ -1,1048 +0,0 @@
-<?xml version="1.0" encoding="Windows-1252"?>
-<VisualStudioProject
- ProjectType="Visual C++"
- Version="9.00"
- Name="mongos"
- ProjectGUID="{E03717ED-69B4-4D21-BC55-DF6690B585C6}"
- RootNamespace="dbgrid"
- Keyword="Win32Proj"
- TargetFrameworkVersion="196613"
- >
- <Platforms>
- <Platform
- Name="Win32"
- />
- </Platforms>
- <ToolFiles>
- </ToolFiles>
- <Configurations>
- <Configuration
- Name="Debug|Win32"
- OutputDirectory="$(SolutionDir)$(ConfigurationName)"
- IntermediateDirectory="$(ConfigurationName)"
- ConfigurationType="1"
- CharacterSet="1"
- >
- <Tool
- Name="VCPreBuildEventTool"
- />
- <Tool
- Name="VCCustomBuildTool"
- />
- <Tool
- Name="VCXMLDataGeneratorTool"
- />
- <Tool
- Name="VCWebServiceProxyGeneratorTool"
- />
- <Tool
- Name="VCMIDLTool"
- />
- <Tool
- Name="VCCLCompilerTool"
- Optimization="0"
- AdditionalIncludeDirectories="&quot;..\..\js\src&quot;;&quot;..\pcre-7.4&quot;;&quot;C:\Program Files\boost\latest&quot;;c:\boost;\boost"
- PreprocessorDefinitions="MONGO_EXPOSE_MACROS;XP_WIN;OLDJS;STATIC_JS_API;WIN32;_DEBUG;_CONSOLE;_CRT_SECURE_NO_WARNINGS;HAVE_CONFIG_H;PCRE_STATIC"
- MinimalRebuild="true"
- BasicRuntimeChecks="3"
- RuntimeLibrary="3"
- UsePrecompiledHeader="2"
- PrecompiledHeaderThrough="pch.h"
- WarningLevel="3"
- DebugInformationFormat="4"
- DisableSpecificWarnings="4355;4800"
- />
- <Tool
- Name="VCManagedResourceCompilerTool"
- />
- <Tool
- Name="VCResourceCompilerTool"
- />
- <Tool
- Name="VCPreLinkEventTool"
- />
- <Tool
- Name="VCLinkerTool"
- AdditionalDependencies="ws2_32.lib Psapi.lib"
- LinkIncremental="2"
- AdditionalLibraryDirectories="&quot;c:\program files\boost\latest\lib&quot;;c:\boost\lib;\boost\lib"
- GenerateDebugInformation="true"
- SubSystem="1"
- TargetMachine="1"
- />
- <Tool
- Name="VCALinkTool"
- />
- <Tool
- Name="VCManifestTool"
- />
- <Tool
- Name="VCXDCMakeTool"
- />
- <Tool
- Name="VCBscMakeTool"
- />
- <Tool
- Name="VCFxCopTool"
- />
- <Tool
- Name="VCAppVerifierTool"
- />
- <Tool
- Name="VCPostBuildEventTool"
- />
- </Configuration>
- <Configuration
- Name="Release|Win32"
- OutputDirectory="$(SolutionDir)$(ConfigurationName)"
- IntermediateDirectory="$(ConfigurationName)"
- ConfigurationType="1"
- CharacterSet="1"
- WholeProgramOptimization="1"
- >
- <Tool
- Name="VCPreBuildEventTool"
- />
- <Tool
- Name="VCCustomBuildTool"
- />
- <Tool
- Name="VCXMLDataGeneratorTool"
- />
- <Tool
- Name="VCWebServiceProxyGeneratorTool"
- />
- <Tool
- Name="VCMIDLTool"
- />
- <Tool
- Name="VCCLCompilerTool"
- Optimization="2"
- EnableIntrinsicFunctions="true"
- AdditionalIncludeDirectories="&quot;..\..\js\src&quot;;&quot;..\pcre-7.4&quot;;&quot;C:\Program Files\boost\latest&quot;;c:\boost;\boost"
- PreprocessorDefinitions="MONGO_EXPOSE_MACROS;OLDJS;STATIC_JS_API;XP_WIN;WIN32;NDEBUG;_CONSOLE;_CRT_SECURE_NO_WARNINGS;HAVE_CONFIG_H;PCRE_STATIC"
- RuntimeLibrary="2"
- EnableFunctionLevelLinking="true"
- UsePrecompiledHeader="2"
- PrecompiledHeaderThrough="pch.h"
- WarningLevel="3"
- DebugInformationFormat="3"
- DisableSpecificWarnings="4355;4800"
- />
- <Tool
- Name="VCManagedResourceCompilerTool"
- />
- <Tool
- Name="VCResourceCompilerTool"
- />
- <Tool
- Name="VCPreLinkEventTool"
- />
- <Tool
- Name="VCLinkerTool"
- AdditionalDependencies="ws2_32.lib Psapi.lib"
- LinkIncremental="1"
- AdditionalLibraryDirectories="&quot;c:\program files\boost\latest\lib&quot;;c:\boost\lib;\boost\lib"
- GenerateDebugInformation="true"
- SubSystem="1"
- OptimizeReferences="2"
- EnableCOMDATFolding="2"
- TargetMachine="1"
- />
- <Tool
- Name="VCALinkTool"
- />
- <Tool
- Name="VCManifestTool"
- />
- <Tool
- Name="VCXDCMakeTool"
- />
- <Tool
- Name="VCBscMakeTool"
- />
- <Tool
- Name="VCFxCopTool"
- />
- <Tool
- Name="VCAppVerifierTool"
- />
- <Tool
- Name="VCPostBuildEventTool"
- />
- </Configuration>
- </Configurations>
- <References>
- </References>
- <Files>
- <Filter
- Name="Source Files"
- Filter="cpp;c;cc;cxx;def;odl;idl;hpj;bat;asm;asmx"
- UniqueIdentifier="{4FC737F1-C7A5-4376-A066-2A32D752A2FF}"
- >
- <File
- RelativePath=".\balance.cpp"
- >
- </File>
- <File
- RelativePath=".\balancer_policy.cpp"
- >
- </File>
- <File
- RelativePath=".\chunk.cpp"
- >
- </File>
- <File
- RelativePath=".\commands_admin.cpp"
- >
- </File>
- <File
- RelativePath=".\commands_public.cpp"
- >
- </File>
- <File
- RelativePath=".\config.cpp"
- >
- </File>
- <File
- RelativePath=".\config_migrate.cpp"
- >
- </File>
- <File
- RelativePath=".\cursors.cpp"
- >
- </File>
- <File
- RelativePath="..\db\queryutil.cpp"
- >
- </File>
- <File
- RelativePath=".\request.cpp"
- >
- </File>
- <File
- RelativePath=".\s_only.cpp"
- >
- </File>
- <File
- RelativePath=".\server.cpp"
- >
- </File>
- <File
- RelativePath=".\shard.cpp"
- >
- </File>
- <File
- RelativePath=".\shardconnection.cpp"
- >
- </File>
- <File
- RelativePath=".\shardkey.cpp"
- >
- </File>
- <File
- RelativePath=".\stats.cpp"
- >
- </File>
- <File
- RelativePath=".\strategy.cpp"
- >
- </File>
- <File
- RelativePath=".\strategy_shard.cpp"
- >
- </File>
- <File
- RelativePath=".\strategy_single.cpp"
- >
- </File>
- <File
- RelativePath="..\scripting\utils.cpp"
- >
- </File>
- </Filter>
- <Filter
- Name="Header Files"
- Filter="h;hpp;hxx;hm;inl;inc;xsd"
- UniqueIdentifier="{93995380-89BD-4b04-88EB-625FBE52EBFB}"
- >
- <File
- RelativePath=".\balancer_policy.h"
- >
- </File>
- <File
- RelativePath=".\shard.h"
- >
- </File>
- <File
- RelativePath=".\strategy.h"
- >
- </File>
- <Filter
- Name="Header Shared"
- >
- <File
- RelativePath="..\util\background.h"
- >
- </File>
- <File
- RelativePath="..\db\commands.h"
- >
- </File>
- <File
- RelativePath="..\client\connpool.cpp"
- >
- </File>
- <File
- RelativePath="..\db\dbmessage.h"
- >
- </File>
- <File
- RelativePath="..\util\goodies.h"
- >
- </File>
- <File
- RelativePath="..\db\jsobj.h"
- >
- </File>
- <File
- RelativePath="..\db\json.h"
- >
- </File>
- <File
- RelativePath="..\pch.h"
- >
- </File>
- <File
- RelativePath="..\util\text.h"
- >
- </File>
- </Filter>
- </Filter>
- <Filter
- Name="Resource Files"
- Filter="rc;ico;cur;bmp;dlg;rc2;rct;bin;rgs;gif;jpg;jpeg;jpe;resx;tiff;tif;png;wav"
- UniqueIdentifier="{67DA6AB6-F800-4c08-8B7A-83BB121AAD01}"
- >
- </Filter>
- <Filter
- Name="libs_etc"
- >
- <File
- RelativePath="..\..\boostw\boost_1_34_1\boost\config\auto_link.hpp"
- >
- </File>
- <File
- RelativePath="..\..\js\src\js.lib"
- >
- </File>
- <File
- RelativePath="..\pcre-7.4\pcrecpp.cc"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\..\boostw\boost_1_34_1\boost\version.hpp"
- >
- </File>
- <Filter
- Name="pcre"
- >
- <File
- RelativePath="..\pcre-7.4\config.h"
- >
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre.h"
- >
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_chartables.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_compile.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_config.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_dfa_exec.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_exec.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_fullinfo.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_get.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_globals.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_info.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_maketables.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_newline.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_ord2utf8.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_refcount.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_scanner.cc"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_stringpiece.cc"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_study.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_tables.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_try_flipped.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_ucp_searchfuncs.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_valid_utf8.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_version.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcre_xclass.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\pcre-7.4\pcreposix.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- </Filter>
- </Filter>
- <Filter
- Name="client"
- >
- <File
- RelativePath="..\client\connpool.h"
- >
- </File>
- <File
- RelativePath="..\client\dbclient.cpp"
- >
- </File>
- <File
- RelativePath="..\client\dbclient.h"
- >
- </File>
- <File
- RelativePath="..\client\dbclientcursor.cpp"
- >
- </File>
- <File
- RelativePath="..\client\model.cpp"
- >
- </File>
- <File
- RelativePath="..\client\model.h"
- >
- </File>
- </Filter>
- <Filter
- Name="Shared Source Files"
- >
- <File
- RelativePath="..\util\assert_util.cpp"
- >
- </File>
- <File
- RelativePath="..\util\background.cpp"
- >
- </File>
- <File
- RelativePath="..\util\base64.cpp"
- >
- </File>
- <File
- RelativePath="..\db\cmdline.cpp"
- >
- </File>
- <File
- RelativePath="..\db\commands.cpp"
- >
- </File>
- <File
- RelativePath="..\db\stats\counters.cpp"
- >
- </File>
- <File
- RelativePath="..\util\debug_util.cpp"
- >
- </File>
- <File
- RelativePath="..\scripting\engine.cpp"
- >
- </File>
- <File
- RelativePath="..\scripting\engine_spidermonkey.cpp"
- >
- </File>
- <File
- RelativePath="..\db\indexkey.cpp"
- >
- </File>
- <File
- RelativePath="..\db\jsobj.cpp"
- >
- </File>
- <File
- RelativePath="..\db\json.cpp"
- >
- </File>
- <File
- RelativePath="..\db\lasterror.cpp"
- >
- </File>
- <File
- RelativePath="..\db\matcher.cpp"
- >
- </File>
- <File
- RelativePath="..\util\md5.c"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\util\md5main.cpp"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="2"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\util\message.cpp"
- >
- </File>
- <File
- RelativePath="..\util\message_server_port.cpp"
- >
- </File>
- <File
- RelativePath="..\util\mmap.cpp"
- >
- </File>
- <File
- RelativePath="..\util\mmap_win.cpp"
- >
- </File>
- <File
- RelativePath="..\shell\mongo_vstudio.cpp"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="0"
- />
- </FileConfiguration>
- </File>
- <File
- RelativePath="..\db\nonce.cpp"
- >
- </File>
- <File
- RelativePath="..\client\parallel.cpp"
- >
- </File>
- <File
- RelativePath="..\util\processinfo_win32.cpp"
- >
- </File>
- <File
- RelativePath="..\util\sock.cpp"
- >
- </File>
- <File
- RelativePath="..\client\syncclusterconnection.cpp"
- >
- </File>
- <File
- RelativePath="..\util\thread_pool.cpp"
- >
- </File>
- <File
- RelativePath="..\util\text.cpp"
- >
- </File>
- <File
- RelativePath="..\util\util.cpp"
- >
- </File>
- </Filter>
- <File
- RelativePath="..\pch.cpp"
- >
- <FileConfiguration
- Name="Debug|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="1"
- />
- </FileConfiguration>
- <FileConfiguration
- Name="Release|Win32"
- >
- <Tool
- Name="VCCLCompilerTool"
- UsePrecompiledHeader="1"
- PrecompiledHeaderThrough="pch.h"
- />
- </FileConfiguration>
- </File>
- </Files>
- <Globals>
- </Globals>
-</VisualStudioProject>
diff --git a/s/dbgrid.vcxproj b/s/dbgrid.vcxproj
index 83fbf68..61a8458 100644
--- a/s/dbgrid.vcxproj
+++ b/s/dbgrid.vcxproj
@@ -85,6 +85,10 @@
<CodeAnalysisRules Condition="'$(Configuration)|$(Platform)'=='Release|x64'" />
<CodeAnalysisRuleAssemblies Condition="'$(Configuration)|$(Platform)'=='Release|Win32'" />
<CodeAnalysisRuleAssemblies Condition="'$(Configuration)|$(Platform)'=='Release|x64'" />
+ <IncludePath Condition="'$(Configuration)|$(Platform)'=='Debug|x64'">..;$(IncludePath)</IncludePath>
+ <IncludePath Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">..;$(IncludePath)</IncludePath>
+ <IncludePath Condition="'$(Configuration)|$(Platform)'=='Release|Win32'">..;$(IncludePath)</IncludePath>
+ <IncludePath Condition="'$(Configuration)|$(Platform)'=='Release|x64'">..;$(IncludePath)</IncludePath>
</PropertyGroup>
<ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">
<ClCompile>
@@ -183,20 +187,31 @@
</Link>
</ItemDefinitionGroup>
<ItemGroup>
+ <ClCompile Include="..\bson\oid.cpp" />
<ClCompile Include="..\client\dbclientcursor.cpp" />
+ <ClCompile Include="..\client\dbclient_rs.cpp" />
<ClCompile Include="..\client\distlock.cpp" />
<ClCompile Include="..\db\dbwebserver.cpp" />
+ <ClCompile Include="..\db\security_key.cpp" />
+ <ClCompile Include="..\scripting\bench.cpp" />
+ <ClCompile Include="..\util\alignedbuilder.cpp">
+ <PrecompiledHeader Condition="'$(Configuration)|$(Platform)'=='Debug|Win32'">NotUsing</PrecompiledHeader>
+ </ClCompile>
+ <ClCompile Include="..\util\concurrency\spin_lock.cpp" />
+ <ClCompile Include="..\util\concurrency\task.cpp" />
<ClCompile Include="..\util\concurrency\thread_pool.cpp" />
<ClCompile Include="..\util\concurrency\vars.cpp" />
<ClCompile Include="..\util\log.cpp" />
<ClCompile Include="..\util\miniwebserver.cpp" />
<ClCompile Include="..\util\processinfo.cpp" />
+ <ClCompile Include="..\util\signal_handlers.cpp" />
<ClCompile Include="..\util\stringutils.cpp" />
<ClCompile Include="..\util\text.cpp" />
<ClCompile Include="..\util\version.cpp" />
<ClCompile Include="balance.cpp" />
<ClCompile Include="balancer_policy.cpp" />
<ClCompile Include="chunk.cpp" />
+ <ClCompile Include="client.cpp" />
<ClCompile Include="commands_admin.cpp" />
<ClCompile Include="commands_public.cpp" />
<ClCompile Include="config.cpp" />
@@ -212,6 +227,7 @@
<ClCompile Include="grid.cpp" />
<ClCompile Include="request.cpp" />
<ClCompile Include="shardconnection.cpp" />
+ <ClCompile Include="shard_version.cpp" />
<ClCompile Include="s_only.cpp" />
<ClCompile Include="server.cpp" />
<ClCompile Include="shard.cpp" />
@@ -512,9 +528,11 @@
<ClCompile Include="..\util\sock.cpp" />
<ClCompile Include="..\client\syncclusterconnection.cpp" />
<ClCompile Include="..\util\util.cpp" />
+ <ClCompile Include="writeback_listener.cpp" />
</ItemGroup>
<ItemGroup>
<ClInclude Include="..\util\processinfo.h" />
+ <ClInclude Include="..\util\signal_handlers.h" />
<ClInclude Include="..\util\version.h" />
<ClInclude Include="balancer_policy.h" />
<ClInclude Include="grid.h" />
diff --git a/s/dbgrid.vcxproj.filters b/s/dbgrid.vcxproj.filters
index bce75b4..b87a1f2 100755
--- a/s/dbgrid.vcxproj.filters
+++ b/s/dbgrid.vcxproj.filters
@@ -287,6 +287,39 @@
<ClCompile Include="..\util\miniwebserver.cpp">
<Filter>Shared Source Files</Filter>
</ClCompile>
+ <ClCompile Include="..\util\concurrency\task.cpp">
+ <Filter>Source Files</Filter>
+ </ClCompile>
+ <ClCompile Include="..\util\signal_handlers.cpp">
+ <Filter>Shared Source Files</Filter>
+ </ClCompile>
+ <ClCompile Include="..\scripting\bench.cpp">
+ <Filter>Shared Source Files</Filter>
+ </ClCompile>
+ <ClCompile Include="writeback_listener.cpp">
+ <Filter>Source Files</Filter>
+ </ClCompile>
+ <ClCompile Include="shard_version.cpp">
+ <Filter>Source Files</Filter>
+ </ClCompile>
+ <ClCompile Include="..\util\concurrency\spin_lock.cpp">
+ <Filter>Source Files</Filter>
+ </ClCompile>
+ <ClCompile Include="..\util\alignedbuilder.cpp">
+ <Filter>Shared Source Files</Filter>
+ </ClCompile>
+ <ClCompile Include="..\bson\oid.cpp">
+ <Filter>Shared Source Files</Filter>
+ </ClCompile>
+ <ClCompile Include="..\client\dbclient_rs.cpp">
+ <Filter>client</Filter>
+ </ClCompile>
+ <ClCompile Include="client.cpp">
+ <Filter>Source Files</Filter>
+ </ClCompile>
+ <ClCompile Include="..\db\security_key.cpp">
+ <Filter>Shared Source Files</Filter>
+ </ClCompile>
</ItemGroup>
<ItemGroup>
<ClInclude Include="gridconfig.h">
@@ -355,6 +388,9 @@
<ClInclude Include="..\util\processinfo.h">
<Filter>Header Files</Filter>
</ClInclude>
+ <ClInclude Include="..\util\signal_handlers.h">
+ <Filter>Shared Source Files</Filter>
+ </ClInclude>
</ItemGroup>
<ItemGroup>
<Library Include="..\..\js\js32d.lib" />
diff --git a/s/grid.cpp b/s/grid.cpp
index 443cd9a..0045754 100644
--- a/s/grid.cpp
+++ b/s/grid.cpp
@@ -19,46 +19,47 @@
#include "pch.h"
#include <iomanip>
-
#include "../client/connpool.h"
#include "../util/stringutils.h"
+#include "../util/unittest.h"
#include "grid.h"
#include "shard.h"
namespace mongo {
-
- DBConfigPtr Grid::getDBConfig( string database , bool create , const string& shardNameHint ){
+
+ DBConfigPtr Grid::getDBConfig( string database , bool create , const string& shardNameHint ) {
{
string::size_type i = database.find( "." );
if ( i != string::npos )
database = database.substr( 0 , i );
}
-
+
if ( database == "config" )
return configServerPtr;
scoped_lock l( _lock );
DBConfigPtr& cc = _databases[database];
- if ( !cc ){
+ if ( !cc ) {
cc.reset(new DBConfig( database ));
- if ( ! cc->load() ){
- if ( create ){
+ if ( ! cc->load() ) {
+ if ( create ) {
// note here that cc->primary == 0.
log() << "couldn't find database [" << database << "] in config db" << endl;
-
- { // lets check case
+
+ {
+ // lets check case
ScopedDbConnection conn( configServer.modelServer() );
BSONObjBuilder b;
b.appendRegex( "_id" , (string)"^" + database + "$" , "i" );
BSONObj d = conn->findOne( ShardNS::database , b.obj() );
conn.done();
- if ( ! d.isEmpty() ){
+ if ( ! d.isEmpty() ) {
cc.reset();
stringstream ss;
- ss << "can't have 2 databases that just differ on case "
+ ss << "can't have 2 databases that just differ on case "
<< " have: " << d["_id"].String()
<< " want to add: " << database;
@@ -67,20 +68,22 @@ namespace mongo {
}
Shard primary;
- if ( database == "admin" ){
+ if ( database == "admin" ) {
primary = configServer.getPrimary();
- } else if ( shardNameHint.empty() ){
+ }
+ else if ( shardNameHint.empty() ) {
primary = Shard::pick();
- } else {
+ }
+ else {
// use the shard name if provided
Shard shard;
shard.reset( shardNameHint );
primary = shard;
}
- if ( primary.ok() ){
+ if ( primary.ok() ) {
cc->setPrimary( primary.getName() ); // saves 'cc' to configDB
log() << "\t put [" << database << "] on: " << primary << endl;
}
@@ -94,53 +97,63 @@ namespace mongo {
cc.reset();
}
}
-
+
}
-
+
return cc;
}
- void Grid::removeDB( string database ){
+ void Grid::removeDB( string database ) {
uassert( 10186 , "removeDB expects db name" , database.find( '.' ) == string::npos );
scoped_lock l( _lock );
_databases.erase( database );
-
+
}
bool Grid::allowLocalHost() const {
return _allowLocalShard;
}
- void Grid::setAllowLocalHost( bool allow ){
+ void Grid::setAllowLocalHost( bool allow ) {
_allowLocalShard = allow;
}
- bool Grid::addShard( string* name , const ConnectionString& servers , long long maxSize , string& errMsg ){
+ bool Grid::addShard( string* name , const ConnectionString& servers , long long maxSize , string& errMsg ) {
// name can be NULL, so privide a dummy one here to avoid testing it elsewhere
string nameInternal;
if ( ! name ) {
name = &nameInternal;
}
- // Check whether the host (or set) exists and run several sanity checks on this request.
+ // Check whether the host (or set) exists and run several sanity checks on this request.
// There are two set of sanity checks: making sure adding this particular shard is consistent
- // with the replica set state (if it exists) and making sure this shards databases can be
+ // with the replica set state (if it exists) and making sure this shards databases can be
// brought into the grid without conflict.
vector<string> dbNames;
try {
ScopedDbConnection newShardConn( servers );
newShardConn->getLastError();
-
- if ( newShardConn->type() == ConnectionString::SYNC ){
+
+ if ( newShardConn->type() == ConnectionString::SYNC ) {
newShardConn.done();
errMsg = "can't use sync cluster as a shard. for replica set, have to use <setname>/<server1>,<server2>,...";
return false;
}
+ BSONObj resIsMongos;
+ bool ok = newShardConn->runCommand( "admin" , BSON( "isdbgrid" << 1 ) , resIsMongos );
+
+ // should return ok=0, cmd not found if it's a normal mongod
+ if ( ok ) {
+ errMsg = "can't add a mongos process as a shard";
+ newShardConn.done();
+ return false;
+ }
+
BSONObj resIsMaster;
- bool ok = newShardConn->runCommand( "admin" , BSON( "isMaster" << 1 ) , resIsMaster );
- if ( !ok ){
+ ok = newShardConn->runCommand( "admin" , BSON( "isMaster" << 1 ) , resIsMaster );
+ if ( !ok ) {
ostringstream ss;
ss << "failed running isMaster: " << resIsMaster;
errMsg = ss.str();
@@ -151,7 +164,7 @@ namespace mongo {
// if the shard has only one host, make sure it is not part of a replica set
string setName = resIsMaster["setName"].str();
string commandSetName = servers.getSetName();
- if ( commandSetName.empty() && ! setName.empty() ){
+ if ( commandSetName.empty() && ! setName.empty() ) {
ostringstream ss;
ss << "host is part of set: " << setName << " use replica set url format <setname>/<server1>,<server2>,....";
errMsg = ss.str();
@@ -160,7 +173,7 @@ namespace mongo {
}
// if the shard is part of replica set, make sure it is the right one
- if ( ! commandSetName.empty() && ( commandSetName != setName ) ){
+ if ( ! commandSetName.empty() && ( commandSetName != setName ) ) {
ostringstream ss;
ss << "host is part of a different set: " << setName;
errMsg = ss.str();
@@ -168,30 +181,39 @@ namespace mongo {
return false;
}
- // if the shard is part of a replica set, make sure all the hosts mentioned in 'servers' are part of
+ // if the shard is part of a replica set, make sure all the hosts mentioned in 'servers' are part of
// the set. It is fine if not all members of the set are present in 'servers'.
bool foundAll = true;
string offendingHost;
- if ( ! commandSetName.empty() ){
+ if ( ! commandSetName.empty() ) {
set<string> hostSet;
BSONObjIterator iter( resIsMaster["hosts"].Obj() );
- while ( iter.more() ){
+ while ( iter.more() ) {
hostSet.insert( iter.next().String() ); // host:port
}
+ if ( resIsMaster["passives"].isABSONObj() ) {
+ BSONObjIterator piter( resIsMaster["passives"].Obj() );
+ while ( piter.more() ) {
+ hostSet.insert( piter.next().String() ); // host:port
+ }
+ }
vector<HostAndPort> hosts = servers.getServers();
- for ( size_t i = 0 ; i < hosts.size() ; i++ ){
+ for ( size_t i = 0 ; i < hosts.size() ; i++ ) {
+ if (!hosts[i].hasPort()) {
+ hosts[i].setPort(CmdLine::DefaultDBPort);
+ }
string host = hosts[i].toString(); // host:port
- if ( hostSet.find( host ) == hostSet.end() ){
+ if ( hostSet.find( host ) == hostSet.end() ) {
offendingHost = host;
foundAll = false;
break;
}
}
}
- if ( ! foundAll ){
+ if ( ! foundAll ) {
ostringstream ss;
- ss << "host " << offendingHost << " does not belong to replica set " << setName;;
+ ss << "host " << offendingHost << " does not belong to replica set as a non-passive member" << setName;;
errMsg = ss.str();
newShardConn.done();
return false;
@@ -199,15 +221,15 @@ namespace mongo {
// shard name defaults to the name of the replica set
if ( name->empty() && ! setName.empty() )
- *name = setName;
+ *name = setName;
- // In order to be accepted as a new shard, that mongod must not have any database name that exists already
- // in any other shards. If that test passes, the new shard's databases are going to be entered as
+ // In order to be accepted as a new shard, that mongod must not have any database name that exists already
+ // in any other shards. If that test passes, the new shard's databases are going to be entered as
// non-sharded db's whose primary is the newly added shard.
BSONObj resListDB;
ok = newShardConn->runCommand( "admin" , BSON( "listDatabases" << 1 ) , resListDB );
- if ( !ok ){
+ if ( !ok ) {
ostringstream ss;
ss << "failed listing " << servers.toString() << "'s databases:" << resListDB;
errMsg = ss.str();
@@ -216,20 +238,21 @@ namespace mongo {
}
BSONObjIterator i( resListDB["databases"].Obj() );
- while ( i.more() ){
+ while ( i.more() ) {
BSONObj dbEntry = i.next().Obj();
const string& dbName = dbEntry["name"].String();
- if ( _isSpecialLocalDB( dbName ) ){
+ if ( _isSpecialLocalDB( dbName ) ) {
// 'local', 'admin', and 'config' are system DBs and should be excluded here
continue;
- } else {
+ }
+ else {
dbNames.push_back( dbName );
}
}
newShardConn.done();
}
- catch ( DBException& e ){
+ catch ( DBException& e ) {
ostringstream ss;
ss << "couldn't connect to new shard ";
ss << e.what();
@@ -238,9 +261,9 @@ namespace mongo {
}
// check that none of the existing shard candidate's db's exist elsewhere
- for ( vector<string>::const_iterator it = dbNames.begin(); it != dbNames.end(); ++it ){
+ for ( vector<string>::const_iterator it = dbNames.begin(); it != dbNames.end(); ++it ) {
DBConfigPtr config = getDBConfig( *it , false );
- if ( config.get() != NULL ){
+ if ( config.get() != NULL ) {
ostringstream ss;
ss << "can't add shard " << servers.toString() << " because a local database '" << *it;
ss << "' exists in another " << config->getPrimary().toString();
@@ -250,26 +273,26 @@ namespace mongo {
}
// if a name for a shard wasn't provided, pick one.
- if ( name->empty() && ! _getNewShardName( name ) ){
+ if ( name->empty() && ! _getNewShardName( name ) ) {
errMsg = "error generating new shard name";
return false;
}
-
+
// build the ConfigDB shard document
BSONObjBuilder b;
b.append( "_id" , *name );
b.append( "host" , servers.toString() );
- if ( maxSize > 0 ){
+ if ( maxSize > 0 ) {
b.append( ShardFields::maxSize.name() , maxSize );
}
BSONObj shardDoc = b.obj();
{
ScopedDbConnection conn( configServer.getPrimary() );
-
+
// check whether the set of hosts (or single host) is not an already a known shard
BSONObj old = conn->findOne( ShardNS::shard , BSON( "host" << servers.toString() ) );
- if ( ! old.isEmpty() ){
+ if ( ! old.isEmpty() ) {
errMsg = "host already used";
conn.done();
return false;
@@ -279,7 +302,7 @@ namespace mongo {
conn->insert( ShardNS::shard , shardDoc );
errMsg = conn->getLastError();
- if ( ! errMsg.empty() ){
+ if ( ! errMsg.empty() ) {
log() << "error adding shard: " << shardDoc << " err: " << errMsg << endl;
conn.done();
return false;
@@ -291,37 +314,37 @@ namespace mongo {
Shard::reloadShardInfo();
// add all databases of the new shard
- for ( vector<string>::const_iterator it = dbNames.begin(); it != dbNames.end(); ++it ){
+ for ( vector<string>::const_iterator it = dbNames.begin(); it != dbNames.end(); ++it ) {
DBConfigPtr config = getDBConfig( *it , true , *name );
- if ( ! config ){
- log() << "adding shard " << servers << " even though could not add database " << *it << endl;
+ if ( ! config ) {
+ log() << "adding shard " << servers << " even though could not add database " << *it << endl;
}
}
return true;
}
-
- bool Grid::knowAboutShard( const string& name ) const{
+
+ bool Grid::knowAboutShard( const string& name ) const {
ShardConnection conn( configServer.getPrimary() , "" );
BSONObj shard = conn->findOne( ShardNS::shard , BSON( "host" << name ) );
conn.done();
return ! shard.isEmpty();
}
- bool Grid::_getNewShardName( string* name ) const{
+ bool Grid::_getNewShardName( string* name ) const {
DEV assert( name );
bool ok = false;
- int count = 0;
+ int count = 0;
ShardConnection conn( configServer.getPrimary() , "" );
- BSONObj o = conn->findOne( ShardNS::shard , Query( fromjson ( "{_id: /^shard/}" ) ).sort( BSON( "_id" << -1 ) ) );
+ BSONObj o = conn->findOne( ShardNS::shard , Query( fromjson ( "{_id: /^shard/}" ) ).sort( BSON( "_id" << -1 ) ) );
if ( ! o.isEmpty() ) {
string last = o["_id"].String();
istringstream is( last.substr( 5 ) );
is >> count;
count++;
- }
+ }
if (count < 9999) {
stringstream ss;
ss << "shard" << setfill('0') << setw(4) << count;
@@ -337,14 +360,75 @@ namespace mongo {
ShardConnection conn( configServer.getPrimary() , "" );
// look for the stop balancer marker
- BSONObj stopMarker = conn->findOne( ShardNS::settings, BSON( "_id" << "balancer" << "stopped" << true ) );
+ BSONObj balancerDoc = conn->findOne( ShardNS::settings, BSON( "_id" << "balancer" ) );
conn.done();
- return stopMarker.isEmpty();
+
+ boost::posix_time::ptime now = boost::posix_time::second_clock::local_time();
+ if ( _balancerStopped( balancerDoc ) || ! _inBalancingWindow( balancerDoc , now ) ) {
+ return false;
+ }
+
+ return true;
+ }
+
+ bool Grid::_balancerStopped( const BSONObj& balancerDoc ) {
+ // check the 'stopped' marker maker
+ // if present, it is a simple bool
+ BSONElement stoppedElem = balancerDoc["stopped"];
+ if ( ! stoppedElem.eoo() && stoppedElem.isBoolean() ) {
+ return stoppedElem.boolean();
+ }
+ return false;
+ }
+
+ bool Grid::_inBalancingWindow( const BSONObj& balancerDoc , const boost::posix_time::ptime& now ) {
+ // check the 'activeWindow' marker
+ // if present, it is an interval during the day when the balancer should be active
+ // { start: "08:00" , stop: "19:30" }, strftime format is %H:%M
+ BSONElement windowElem = balancerDoc["activeWindow"];
+ if ( windowElem.eoo() ) {
+ return true;
+ }
+
+ // check if both 'start' and 'stop' are present
+ if ( ! windowElem.isABSONObj() ) {
+ log(1) << "'activeWindow' format is { start: \"hh:mm\" , stop: ... }" << balancerDoc << endl;
+ return true;
+ }
+ BSONObj intervalDoc = windowElem.Obj();
+ const string start = intervalDoc["start"].str();
+ const string stop = intervalDoc["stop"].str();
+ if ( start.empty() || stop.empty() ) {
+ log(1) << "must specify both start and end of balancing window: " << intervalDoc << endl;
+ return true;
+ }
+
+ // check that both 'start' and 'stop' are valid time-of-day
+ boost::posix_time::ptime startTime, stopTime;
+ if ( ! toPointInTime( start , &startTime ) || ! toPointInTime( stop , &stopTime ) ) {
+ log(1) << "cannot parse active window (use hh:mm 24hs format): " << intervalDoc << endl;
+ return true;
+ }
+
+ // allow balancing if during the activeWindow
+ // note that a window may be open during the night
+ if ( stopTime > startTime ) {
+ if ( ( now >= startTime ) && ( now <= stopTime ) ) {
+ return true;
+ }
+ }
+ else if ( startTime > stopTime ) {
+ if ( ( now >=startTime ) || ( now <= stopTime ) ) {
+ return true;
+ }
+ }
+
+ return false;
}
unsigned long long Grid::getNextOpTime() const {
ScopedDbConnection conn( configServer.getPrimary() );
-
+
BSONObj result;
massert( 10421 , "getoptime failed" , conn->simpleCommand( "admin" , &result , "getoptime" ) );
conn.done();
@@ -352,10 +436,51 @@ namespace mongo {
return result["optime"]._numberLong();
}
- bool Grid::_isSpecialLocalDB( const string& dbName ){
+ bool Grid::_isSpecialLocalDB( const string& dbName ) {
return ( dbName == "local" ) || ( dbName == "admin" ) || ( dbName == "config" );
}
Grid grid;
-}
+ // unit tests
+
+ class BalancingWindowUnitTest : public UnitTest {
+ public:
+ void run() {
+ // T0 < T1 < now < T2 < T3 and Error
+ const string T0 = "9:00";
+ const string T1 = "11:00";
+ boost::posix_time::ptime now( currentDate(), boost::posix_time::hours( 13 ) + boost::posix_time::minutes( 48 ) );
+ const string T2 = "17:00";
+ const string T3 = "21:30";
+ const string E = "28:35";
+
+ BSONObj w1 = BSON( "activeWindow" << BSON( "start" << T0 << "stop" << T1 ) ); // closed in the past
+ BSONObj w2 = BSON( "activeWindow" << BSON( "start" << T2 << "stop" << T3 ) ); // not opened until the future
+ BSONObj w3 = BSON( "activeWindow" << BSON( "start" << T1 << "stop" << T2 ) ); // open now
+ BSONObj w4 = BSON( "activeWindow" << BSON( "start" << T3 << "stop" << T2 ) ); // open since last day
+
+ assert( ! Grid::_inBalancingWindow( w1 , now ) );
+ assert( ! Grid::_inBalancingWindow( w2 , now ) );
+ assert( Grid::_inBalancingWindow( w3 , now ) );
+ assert( Grid::_inBalancingWindow( w4 , now ) );
+
+ // bad input should not stop the balancer
+
+ BSONObj w5; // empty window
+ BSONObj w6 = BSON( "activeWindow" << BSON( "start" << 1 ) ); // missing stop
+ BSONObj w7 = BSON( "activeWindow" << BSON( "stop" << 1 ) ); // missing start
+ BSONObj w8 = BSON( "wrongMarker" << 1 << "start" << 1 << "stop" << 1 ); // active window marker missing
+ BSONObj w9 = BSON( "activeWindow" << BSON( "start" << T3 << "stop" << E ) ); // garbage in window
+
+ assert( Grid::_inBalancingWindow( w5 , now ) );
+ assert( Grid::_inBalancingWindow( w6 , now ) );
+ assert( Grid::_inBalancingWindow( w7 , now ) );
+ assert( Grid::_inBalancingWindow( w8 , now ) );
+ assert( Grid::_inBalancingWindow( w9 , now ) );
+
+ log(1) << "BalancingWidowObjTest passed" << endl;
+ }
+ } BalancingWindowObjTest;
+
+}
diff --git a/s/grid.h b/s/grid.h
index 4f3c2ac..5692a82 100644
--- a/s/grid.h
+++ b/s/grid.h
@@ -18,6 +18,9 @@
#pragma once
+#include <boost/date_time/posix_time/posix_time.hpp>
+
+#include "../util/time_support.h"
#include "../util/concurrency/mutex.h"
#include "config.h" // DBConfigPtr
@@ -37,7 +40,7 @@ namespace mongo {
* will return an empty DBConfig if not in db already
*/
DBConfigPtr getDBConfig( string ns , bool create=true , const string& shardNameHint="" );
-
+
/**
* removes db entry.
* on next getDBConfig call will fetch from db
@@ -57,14 +60,14 @@ namespace mongo {
/**
*
* addShard will create a new shard in the grid. It expects a mongod process to be runing
- * on the provided address. Adding a shard that is a replica set is supported.
+ * on the provided address. Adding a shard that is a replica set is supported.
*
* @param name is an optional string with the name of the shard. if ommited, grid will
- * generate one and update the parameter.
+ * generate one and update the parameter.
* @param servers is the connection string of the shard being added
* @param maxSize is the optional space quota in bytes. Zeros means there's no limitation to
- * space usage
- * @param errMsg is the error description in case the operation failed.
+ * space usage
+ * @param errMsg is the error description in case the operation failed.
* @return true if shard was successfully added.
*/
bool addShard( string* name , const ConnectionString& servers , long long maxSize , string& errMsg );
@@ -73,7 +76,7 @@ namespace mongo {
* @return true if the config database knows about a host 'name'
*/
bool knowAboutShard( const string& name ) const;
-
+
/**
* @return true if the chunk balancing functionality is enabled
*/
@@ -81,6 +84,15 @@ namespace mongo {
unsigned long long getNextOpTime() const;
+ // exposed methods below are for testing only
+
+ /**
+ * @param balancerDoc bson that may contain a window of time for the balancer to work
+ * format { ... , activeWindow: { start: "8:30" , stop: "19:00" } , ... }
+ * @return true if there is no window of time specified for the balancer or it we're currently in it
+ */
+ static bool _inBalancingWindow( const BSONObj& balancerDoc , const boost::posix_time::ptime& now );
+
private:
mongo::mutex _lock; // protects _databases; TODO: change to r/w lock ??
map<string, DBConfigPtr > _databases; // maps ns to DBConfig's
@@ -89,7 +101,7 @@ namespace mongo {
/**
* @param name is the chose name for the shard. Parameter is mandatory.
* @return true if it managed to generate a shard name. May return false if (currently)
- * 10000 shard
+ * 10000 shard
*/
bool _getNewShardName( string* name ) const;
@@ -98,6 +110,13 @@ namespace mongo {
*/
static bool _isSpecialLocalDB( const string& dbName );
+ /**
+ * @param balancerDoc bson that may contain a marker to stop the balancer
+ * format { ... , stopped: [ "true" | "false" ] , ... }
+ * @return true if the marker is present and is set to true
+ */
+ static bool _balancerStopped( const BSONObj& balancerDoc );
+
};
extern Grid grid;
diff --git a/s/request.cpp b/s/request.cpp
index ec245d7..52f2e54 100644
--- a/s/request.cpp
+++ b/s/request.cpp
@@ -1,7 +1,4 @@
-/* dbgrid/request.cpp
-
- Top level handling of requests (operations such as query, insert, ...)
-*/
+// s/request.cpp
/**
* Copyright (C) 2008 10gen Inc.
@@ -34,53 +31,56 @@
#include "stats.h"
#include "cursors.h"
#include "grid.h"
+#include "client.h"
namespace mongo {
- Request::Request( Message& m, AbstractMessagingPort* p ) :
- _m(m) , _d( m ) , _p(p) , _didInit(false){
-
+ Request::Request( Message& m, AbstractMessagingPort* p ) :
+ _m(m) , _d( m ) , _p(p) , _didInit(false) {
+
assert( _d.getns() );
_id = _m.header()->id;
-
+
_clientId = p ? p->getClientId() : 0;
_clientInfo = ClientInfo::get( _clientId );
_clientInfo->newRequest( p );
-
+
}
-
- void Request::init(){
+
+ void Request::init() {
if ( _didInit )
return;
_didInit = true;
reset();
}
-
- void Request::reset( bool reload ){
- if ( _m.operation() == dbKillCursors ){
+
+ void Request::reset( bool reload ) {
+ if ( _m.operation() == dbKillCursors ) {
return;
}
-
+
+ uassert( 13644 , "can't use 'local' database through mongos" , ! str::startsWith( getns() , "local." ) );
+
_config = grid.getDBConfig( getns() );
if ( reload )
uassert( 10192 , "db config reload failed!" , _config->reload() );
- if ( _config->isSharded( getns() ) ){
+ if ( _config->isSharded( getns() ) ) {
_chunkManager = _config->getChunkManager( getns() , reload );
uassert( 10193 , (string)"no shard info for: " + getns() , _chunkManager );
}
else {
_chunkManager.reset();
- }
+ }
_m.header()->id = _id;
-
+
}
-
+
Shard Request::primaryShard() const {
assert( _didInit );
-
- if ( _chunkManager ){
+
+ if ( _chunkManager ) {
if ( _chunkManager->numChunks() > 1 )
throw UserException( 8060 , "can't call primaryShard on a sharded collection" );
return _chunkManager->findChunk( _chunkManager->getShardKey().globalMin() )->getShard();
@@ -89,26 +89,26 @@ namespace mongo {
uassert( 10194 , "can't call primaryShard on a sharded collection!" , s.ok() );
return s;
}
-
- void Request::process( int attempt ){
+
+ void Request::process( int attempt ) {
init();
int op = _m.operation();
assert( op > dbMsg );
-
- if ( op == dbKillCursors ){
+
+ if ( op == dbKillCursors ) {
cursorCache.gotKillCursors( _m );
return;
}
-
+
log(3) << "Request::process ns: " << getns() << " msg id:" << (int)(_m.header()->id) << " attempt: " << attempt << endl;
-
+
Strategy * s = SINGLE;
_counter = &opsNonSharded;
-
+
_d.markSet();
-
- if ( _chunkManager ){
+
+ if ( _chunkManager ) {
s = SHARDED;
_counter = &opsSharded;
}
@@ -119,7 +119,7 @@ namespace mongo {
try {
s->queryOp( *this );
}
- catch ( StaleConfigException& staleConfig ){
+ catch ( StaleConfigException& staleConfig ) {
log() << staleConfig.what() << " attempt: " << attempt << endl;
uassert( 10195 , "too many attempts to update config, failing" , attempt < 5 );
ShardConnection::checkMyConnectionVersions( getns() );
@@ -141,115 +141,31 @@ namespace mongo {
globalOpCounters.gotOp( op , iscmd );
_counter->gotOp( op , iscmd );
}
-
+
bool Request::isCommand() const {
int x = _d.getQueryNToReturn();
return ( x == 1 || x == -1 ) && strstr( getns() , ".$cmd" );
}
- void Request::gotInsert(){
+ void Request::gotInsert() {
globalOpCounters.gotInsert();
_counter->gotInsert();
}
- void Request::reply( Message & response , const string& fromServer ){
+ void Request::reply( Message & response , const string& fromServer ) {
assert( _didInit );
long long cursor =response.header()->getCursor();
- if ( cursor ){
- cursorCache.storeRef( fromServer , cursor );
- }
- _p->reply( _m , response , _id );
- }
-
- ClientInfo::ClientInfo( int clientId ) : _id( clientId ){
- _cur = &_a;
- _prev = &_b;
- newRequest();
- }
-
- ClientInfo::~ClientInfo(){
- if ( _lastAccess ){
- scoped_lock lk( _clientsLock );
- ClientCache::iterator i = _clients.find( _id );
- if ( i != _clients.end() ){
- _clients.erase( i );
+ if ( cursor ) {
+ if ( fromServer.size() ) {
+ cursorCache.storeRef( fromServer , cursor );
}
- }
- }
-
- void ClientInfo::addShard( const string& shard ){
- _cur->insert( shard );
- _sinceLastGetError.insert( shard );
- }
-
- void ClientInfo::newRequest( AbstractMessagingPort* p ){
-
- if ( p ){
- string r = p->remote().toString();
- if ( _remote == "" )
- _remote = r;
- else if ( _remote != r ){
- stringstream ss;
- ss << "remotes don't match old [" << _remote << "] new [" << r << "]";
- throw UserException( 13134 , ss.str() );
+ else {
+ // probably a getMore
+ // make sure we have a ref for this
+ assert( cursorCache.getRef( cursor ).size() );
}
}
-
- _lastAccess = (int) time(0);
-
- set<string> * temp = _cur;
- _cur = _prev;
- _prev = temp;
- _cur->clear();
- }
-
- void ClientInfo::disconnect(){
- _lastAccess = 0;
- }
-
- ClientInfo * ClientInfo::get( int clientId , bool create ){
-
- if ( ! clientId )
- clientId = getClientId();
-
- if ( ! clientId ){
- ClientInfo * info = _tlInfo.get();
- if ( ! info ){
- info = new ClientInfo( 0 );
- _tlInfo.reset( info );
- }
- info->newRequest();
- return info;
- }
-
- scoped_lock lk( _clientsLock );
- ClientCache::iterator i = _clients.find( clientId );
- if ( i != _clients.end() )
- return i->second;
- if ( ! create )
- return 0;
- ClientInfo * info = new ClientInfo( clientId );
- _clients[clientId] = info;
- return info;
- }
-
- void ClientInfo::disconnect( int clientId ){
- if ( ! clientId )
- return;
-
- scoped_lock lk( _clientsLock );
- ClientCache::iterator i = _clients.find( clientId );
- if ( i == _clients.end() )
- return;
-
- ClientInfo* ci = i->second;
- ci->disconnect();
- delete ci;
- _clients.erase( i );
+ _p->reply( _m , response , _id );
}
- ClientCache& ClientInfo::_clients = *(new ClientCache());
- mongo::mutex ClientInfo::_clientsLock("_clientsLock");
- boost::thread_specific_ptr<ClientInfo> ClientInfo::_tlInfo;
-
} // namespace mongo
diff --git a/s/request.h b/s/request.h
index f063d0c..5b4c228 100644
--- a/s/request.h
+++ b/s/request.h
@@ -26,16 +26,16 @@
namespace mongo {
-
+
class OpCounters;
class ClientInfo;
-
+
class Request : boost::noncopyable {
public:
Request( Message& m, AbstractMessagingPort* p );
// ---- message info -----
-
+
const char * getns() const {
return _d.getns();
@@ -60,12 +60,12 @@ namespace mongo {
assert( _didInit );
return _config->isShardingEnabled();
}
-
+
ChunkManagerPtr getChunkManager() const {
assert( _didInit );
return _chunkManager;
}
-
+
int getClientId() const {
return _clientId;
}
@@ -74,14 +74,14 @@ namespace mongo {
}
// ---- remote location info -----
-
-
+
+
Shard primaryShard() const ;
-
+
// ---- low level access ----
void reply( Message & response , const string& fromServer );
-
+
Message& m() { return _m; }
DbMessage& d() { return _d; }
AbstractMessagingPort* p() const { return _p; }
@@ -93,16 +93,16 @@ namespace mongo {
void init();
void reset( bool reload=false );
-
+
private:
Message& _m;
DbMessage _d;
AbstractMessagingPort* _p;
-
+
MSGID _id;
DBConfigPtr _config;
ChunkManagerPtr _chunkManager;
-
+
int _clientId;
ClientInfo * _clientInfo;
@@ -111,45 +111,6 @@ namespace mongo {
bool _didInit;
};
- typedef map<int,ClientInfo*> ClientCache;
-
- class ClientInfo {
- public:
- ClientInfo( int clientId );
- ~ClientInfo();
-
- string getRemote() const { return _remote; }
-
- void addShard( const string& shard );
- set<string> * getPrev() const { return _prev; };
-
- void newRequest( AbstractMessagingPort* p = 0 );
- void disconnect();
-
- static ClientInfo * get( int clientId = 0 , bool create = true );
- static void disconnect( int clientId );
-
- const set<string>& sinceLastGetError() const { return _sinceLastGetError; }
- void clearSinceLastGetError(){
- _sinceLastGetError.clear();
- }
-
- private:
- int _id;
- string _remote;
-
- set<string> _a;
- set<string> _b;
- set<string> * _cur;
- set<string> * _prev;
- int _lastAccess;
-
- set<string> _sinceLastGetError;
-
- static mongo::mutex _clientsLock;
- static ClientCache& _clients;
- static boost::thread_specific_ptr<ClientInfo> _tlInfo;
- };
}
#include "strategy.h"
diff --git a/s/s_only.cpp b/s/s_only.cpp
index 1f66e70..83bceac 100644
--- a/s/s_only.cpp
+++ b/s/s_only.cpp
@@ -16,6 +16,8 @@
*/
#include "pch.h"
+#include "request.h"
+#include "client.h"
#include "../client/dbclient.h"
#include "../db/dbhelpers.h"
#include "../db/matcher.h"
@@ -27,53 +29,54 @@
*/
namespace mongo {
- auto_ptr<CursorIterator> Helpers::find( const char *ns , BSONObj query , bool requireIndex ){
- uassert( 10196 , "Helpers::find can't be used in mongos" , 0 );
- auto_ptr<CursorIterator> i;
- return i;
- }
-
boost::thread_specific_ptr<Client> currentClient;
- Client::Client(const char *desc , MessagingPort *p) :
- _context(0),
- _shutdown(false),
- _desc(desc),
- _god(0),
- _lastOp(0),
- _mp(p)
- {
+ Client::Client(const char *desc , MessagingPort *p) :
+ _context(0),
+ _shutdown(false),
+ _desc(desc),
+ _god(0),
+ _lastOp(0),
+ _mp(p) {
}
- Client::~Client(){}
- bool Client::shutdown(){ return true; }
+ Client::~Client() {}
+ bool Client::shutdown() { return true; }
- bool webHaveAdminUsers(){
- return false;
+ Client& Client::initThread(const char *desc, MessagingPort *mp) {
+ setThreadName(desc);
+ assert( currentClient.get() == 0 );
+ Client *c = new Client(desc, mp);
+ currentClient.reset(c);
+ mongo::lastError.initThread();
+ return *c;
}
- BSONObj webGetAdminUser( const string& username ){
- return BSONObj();
+ string Client::clientAddress(bool includePort) const {
+ ClientInfo * ci = ClientInfo::get();
+ if ( ci )
+ return ci->getRemote();
+ return "";
}
-
+
bool execCommand( Command * c ,
- Client& client , int queryOptions ,
- const char *ns, BSONObj& cmdObj ,
- BSONObjBuilder& result,
- bool fromRepl ){
+ Client& client , int queryOptions ,
+ const char *ns, BSONObj& cmdObj ,
+ BSONObjBuilder& result,
+ bool fromRepl ) {
assert(c);
-
+
string dbname = nsToDatabase( ns );
-
- if ( cmdObj["help"].trueValue() ){
+
+ if ( cmdObj["help"].trueValue() ) {
stringstream ss;
ss << "help for: " << c->name << " ";
c->help( ss );
result.append( "help" , ss.str() );
result.append( "lockType" , c->locktype() );
return true;
- }
+ }
- if ( c->adminOnly() ){
+ if ( c->adminOnly() ) {
if ( dbname != "admin" ) {
result.append( "errmsg" , "access denied- use admin db" );
log() << "command denied: " << cmdObj.toString() << endl;
diff --git a/s/server.cpp b/s/server.cpp
index c3dc24c..9bdeede 100644
--- a/s/server.cpp
+++ b/s/server.cpp
@@ -23,37 +23,41 @@
#include "../util/message_server.h"
#include "../util/stringutils.h"
#include "../util/version.h"
+#include "../util/signal_handlers.h"
+#include "../util/admin_access.h"
#include "../db/dbwebserver.h"
#include "server.h"
#include "request.h"
+#include "client.h"
#include "config.h"
#include "chunk.h"
#include "balance.h"
#include "grid.h"
#include "cursors.h"
+#include "shard_version.h"
namespace mongo {
-
- CmdLine cmdLine;
+
+ CmdLine cmdLine;
Database *database = 0;
string mongosCommand;
bool dbexitCalled = false;
- bool inShutdown(){
+ bool inShutdown() {
return dbexitCalled;
}
-
+
string getDbContext() {
return "?";
}
- bool haveLocalShardingInfo( const string& ns ){
+ bool haveLocalShardingInfo( const string& ns ) {
assert( 0 );
return false;
}
-
- void usage( char * argv[] ){
+
+ void usage( char * argv[] ) {
out() << argv[0] << " usage:\n\n";
out() << " -v+ verbose 1: general 2: more 3: per request 4: more\n";
out() << " --port <portno>\n";
@@ -64,23 +68,23 @@ namespace mongo {
class ShardingConnectionHook : public DBConnectionHook {
public:
- virtual void onHandedOut( DBClientBase * conn ){
+ virtual void onHandedOut( DBClientBase * conn ) {
ClientInfo::get()->addShard( conn->getServerAddress() );
}
} shardingConnectionHook;
-
+
class ShardedMessageHandler : public MessageHandler {
public:
- virtual ~ShardedMessageHandler(){}
+ virtual ~ShardedMessageHandler() {}
- virtual void process( Message& m , AbstractMessagingPort* p ){
+ virtual void process( Message& m , AbstractMessagingPort* p ) {
assert( p );
Request r( m , p );
LastError * le = lastError.startRequest( m , r.getClientId() );
assert( le );
-
- if ( logLevel > 5 ){
+
+ if ( logLevel > 5 ) {
log(5) << "client id: " << hex << r.getClientId() << "\t" << r.getns() << "\t" << dec << r.op() << endl;
}
try {
@@ -88,43 +92,67 @@ namespace mongo {
setClientId( r.getClientId() );
r.process();
}
- catch ( DBException& e ){
+ catch ( AssertionException & e ) {
+ log( e.isUserAssertion() ? 1 : 0 ) << "AssertionException in process: " << e.what() << endl;
+
+ le->raiseError( e.getCode() , e.what() );
+
+ m.header()->id = r.id();
+
+ if ( r.expectResponse() ) {
+ BSONObj err = BSON( "$err" << e.what() << "code" << e.getCode() );
+ replyToQuery( ResultFlag_ErrSet, p , m , err );
+ }
+ }
+ catch ( DBException& e ) {
log() << "DBException in process: " << e.what() << endl;
-
+
le->raiseError( e.getCode() , e.what() );
-
+
m.header()->id = r.id();
-
- if ( r.expectResponse() ){
+
+ if ( r.expectResponse() ) {
BSONObj err = BSON( "$err" << e.what() << "code" << e.getCode() );
replyToQuery( ResultFlag_ErrSet, p , m , err );
}
}
}
- virtual void disconnected( AbstractMessagingPort* p ){
+ virtual void disconnected( AbstractMessagingPort* p ) {
ClientInfo::disconnect( p->getClientId() );
lastError.disconnect( p->getClientId() );
}
};
- void sighandler(int sig){
+ void sighandler(int sig) {
dbexit(EXIT_CLEAN, (string("received signal ") + BSONObjBuilder::numStr(sig)).c_str());
}
-
- void setupSignals(){
+
+ void setupSignals( bool inFork ) {
signal(SIGTERM, sighandler);
signal(SIGINT, sighandler);
+
+#if defined(SIGQUIT)
+ signal( SIGQUIT , printStackAndExit );
+#endif
+ signal( SIGSEGV , printStackAndExit );
+ signal( SIGABRT , printStackAndExit );
+ signal( SIGFPE , printStackAndExit );
+#if defined(SIGBUS)
+ signal( SIGBUS , printStackAndExit );
+#endif
}
- void init(){
+ void init() {
serverID.init();
setupSIGTRAPforGDB();
setupCoreSignals();
- setupSignals();
+ setupSignals( false );
}
- void start( const MessageServer::Options& opts ){
+ void start( const MessageServer::Options& opts ) {
+ setThreadName( "mongosMain" );
+ installChunkShardVersioning();
balancer.go();
cursorCache.startTimeoutThread();
@@ -137,12 +165,12 @@ namespace mongo {
server->run();
}
- DBClientBase *createDirectClient(){
+ DBClientBase *createDirectClient() {
uassert( 10197 , "createDirectClient not implemented for sharding yet" , 0 );
return 0;
}
- void printShardingVersionInfo(){
+ void printShardingVersionInfo() {
log() << mongosCommand << " " << mongodVersion() << " starting (--help for usage)" << endl;
printGitVersion();
printSysInfo();
@@ -156,91 +184,108 @@ using namespace mongo;
namespace po = boost::program_options;
-int main(int argc, char* argv[], char *envp[] ) {
+int _main(int argc, char* argv[]) {
static StaticObserver staticObserver;
mongosCommand = argv[0];
- po::options_description options("Sharding options");
+ po::options_description options("General options");
+ po::options_description sharding_options("Sharding options");
po::options_description hidden("Hidden options");
po::positional_options_description positional;
-
+
CmdLine::addGlobalOptions( options , hidden );
-
- options.add_options()
- ( "configdb" , po::value<string>() , "1 or 3 comma separated config servers" )
- ( "test" , "just run unit tests" )
- ( "upgrade" , "upgrade meta data version" )
- ( "chunkSize" , po::value<int>(), "maximum amount of data per chunk" )
- ( "ipv6", "enable IPv6 support (disabled by default)" )
- ;
-
+ sharding_options.add_options()
+ ( "configdb" , po::value<string>() , "1 or 3 comma separated config servers" )
+ ( "test" , "just run unit tests" )
+ ( "upgrade" , "upgrade meta data version" )
+ ( "chunkSize" , po::value<int>(), "maximum amount of data per chunk" )
+ ( "ipv6", "enable IPv6 support (disabled by default)" )
+ ( "jsonp","allow JSONP access via http (has security implications)" )
+ ;
+
+ options.add(sharding_options);
// parse options
po::variables_map params;
if ( ! CmdLine::store( argc , argv , options , hidden , positional , params ) )
return 0;
-
- if ( params.count( "help" ) ){
+
+ // The default value may vary depending on compile options, but for mongos
+ // we want durability to be disabled.
+ cmdLine.dur = false;
+
+ if ( params.count( "help" ) ) {
cout << options << endl;
return 0;
}
- if ( params.count( "version" ) ){
+ if ( params.count( "version" ) ) {
printShardingVersionInfo();
return 0;
}
- if ( params.count( "chunkSize" ) ){
+ if ( params.count( "chunkSize" ) ) {
Chunk::MaxChunkSize = params["chunkSize"].as<int>() * 1024 * 1024;
}
- if ( params.count( "ipv6" ) ){
+ if ( params.count( "ipv6" ) ) {
enableIPv6();
}
- if ( params.count( "test" ) ){
+ if ( params.count( "jsonp" ) ) {
+ cmdLine.jsonp = true;
+ }
+
+ if ( params.count( "test" ) ) {
logLevel = 5;
UnitTest::runTests();
cout << "tests passed" << endl;
return 0;
}
-
- if ( ! params.count( "configdb" ) ){
+
+ if ( ! params.count( "configdb" ) ) {
out() << "error: no args for --configdb" << endl;
return 4;
}
vector<string> configdbs;
splitStringDelim( params["configdb"].as<string>() , &configdbs , ',' );
- if ( configdbs.size() != 1 && configdbs.size() != 3 ){
+ if ( configdbs.size() != 1 && configdbs.size() != 3 ) {
out() << "need either 1 or 3 configdbs" << endl;
return 5;
}
// we either have a seeting were all process are in localhost or none is
- for ( vector<string>::const_iterator it = configdbs.begin() ; it != configdbs.end() ; ++it ){
+ for ( vector<string>::const_iterator it = configdbs.begin() ; it != configdbs.end() ; ++it ) {
try {
HostAndPort configAddr( *it ); // will throw if address format is invalid
- if ( it == configdbs.begin() ){
+ if ( it == configdbs.begin() ) {
grid.setAllowLocalHost( configAddr.isLocalHost() );
}
- if ( configAddr.isLocalHost() != grid.allowLocalHost() ){
+ if ( configAddr.isLocalHost() != grid.allowLocalHost() ) {
out() << "cannot mix localhost and ip addresses in configdbs" << endl;
return 10;
}
- }
+ }
catch ( DBException& e) {
out() << "configdb: " << e.what() << endl;
return 9;
}
}
+ // set some global state
+
pool.addHook( &shardingConnectionHook );
+ pool.setName( "mongos connectionpool" );
+
+ DBClientConnection::setLazyKillCursor( false );
+ ReplicaSetMonitor::setConfigChangeHook( boost::bind( &ConfigServer::replicaSetChange , &configServer , _1 ) );
+
if ( argc <= 1 ) {
usage( argv );
return 3;
@@ -252,22 +297,22 @@ int main(int argc, char* argv[], char *envp[] ) {
usage( argv );
return 1;
}
-
+
printShardingVersionInfo();
-
- if ( ! configServer.init( configdbs ) ){
+
+ if ( ! configServer.init( configdbs ) ) {
cout << "couldn't resolve config db address" << endl;
return 7;
}
-
- if ( ! configServer.ok( true ) ){
+
+ if ( ! configServer.ok( true ) ) {
cout << "configServer startup check failed" << endl;
return 8;
}
-
+
int configError = configServer.checkConfigVersion( params.count( "upgrade" ) );
- if ( configError ){
- if ( configError > 0 ){
+ if ( configError ) {
+ if ( configError > 0 ) {
cout << "upgrade success!" << endl;
}
else {
@@ -279,8 +324,8 @@ int main(int argc, char* argv[], char *envp[] ) {
init();
- boost::thread web( webServerThread );
-
+ boost::thread web( boost::bind(&webServerThread, new NoAdminAccess() /* takes ownership */) );
+
MessageServer::Options opts;
opts.port = cmdLine.port;
opts.ipList = cmdLine.bind_ip;
@@ -289,10 +334,30 @@ int main(int argc, char* argv[], char *envp[] ) {
dbexit( EXIT_CLEAN );
return 0;
}
+int main(int argc, char* argv[]) {
+ try {
+ return _main(argc, argv);
+ }
+ catch(DBException& e) {
+ cout << "uncaught exception in mongos main:" << endl;
+ cout << e.toString() << endl;
+ }
+ catch(std::exception& e) {
+ cout << "uncaught exception in mongos main:" << endl;
+ cout << e.what() << endl;
+ }
+ catch(...) {
+ cout << "uncaught exception in mongos main" << endl;
+ }
+ return 20;
+}
#undef exit
-void mongo::dbexit( ExitCode rc, const char *why) {
+void mongo::dbexit( ExitCode rc, const char *why, bool tryToGetLock ) {
dbexitCalled = true;
- log() << "dbexit: " << why << " rc:" << rc << endl;
+ log() << "dbexit: " << why
+ << " rc:" << rc
+ << " " << ( why ? why : "" )
+ << endl;
::exit(rc);
}
diff --git a/s/server.h b/s/server.h
index c45d77d..1a5c9ea 100644
--- a/s/server.h
+++ b/s/server.h
@@ -21,9 +21,9 @@
#include "../db/jsobj.h"
namespace mongo {
-
+
extern OID serverID;
-
+
// from request.cpp
void processRequest(Message& m, MessagingPort& p);
}
diff --git a/s/shard.cpp b/s/shard.cpp
index 4d73a66..dbfd8f9 100644
--- a/s/shard.cpp
+++ b/s/shard.cpp
@@ -1,147 +1,155 @@
// shard.cpp
/**
-* Copyright (C) 2008 10gen Inc.
-*
-* This program is free software: you can redistribute it and/or modify
-* it under the terms of the GNU Affero General Public License, version 3,
-* as published by the Free Software Foundation.
-*
-* This program is distributed in the hope that it will be useful,
-* but WITHOUT ANY WARRANTY; without even the implied warranty of
-* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
-* GNU Affero General Public License for more details.
-*
-* You should have received a copy of the GNU Affero General Public License
-* along with this program. If not, see <http://www.gnu.org/licenses/>.
-*/
+ * Copyright (C) 2008 10gen Inc.
+ *
+ * This program is free software: you can redistribute it and/or modify
+ * it under the terms of the GNU Affero General Public License, version 3,
+ * as published by the Free Software Foundation.
+ *
+ * This program is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+ * GNU Affero General Public License for more details.
+ *
+ * You should have received a copy of the GNU Affero General Public License
+ * along with this program. If not, see <http://www.gnu.org/licenses/>.
+ */
#include "pch.h"
#include "shard.h"
#include "config.h"
#include "request.h"
+#include "../db/commands.h"
#include <set>
namespace mongo {
-
+
class StaticShardInfo {
public:
StaticShardInfo() : _mutex("StaticShardInfo") { }
- void reload(){
+ void reload() {
list<BSONObj> all;
{
ScopedDbConnection conn( configServer.getPrimary() );
auto_ptr<DBClientCursor> c = conn->query( ShardNS::shard , Query() );
- assert( c.get() );
- while ( c->more() ){
+ massert( 13632 , "couldn't get updated shard list from config server" , c.get() );
+ while ( c->more() ) {
all.push_back( c->next().getOwned() );
}
conn.done();
}
-
+
scoped_lock lk( _mutex );
-
+
// We use the _lookup table for all shards and for the primary config DB. The config DB info,
// however, does not come from the ShardNS::shard. So when cleaning the _lookup table we leave
// the config state intact. The rationale is that this way we could drop shards that
// were removed without reinitializing the config DB information.
map<string,Shard>::iterator i = _lookup.find( "config" );
- if ( i != _lookup.end() ){
+ if ( i != _lookup.end() ) {
Shard config = i->second;
_lookup.clear();
_lookup[ "config" ] = config;
- } else {
+ }
+ else {
_lookup.clear();
}
- for ( list<BSONObj>::iterator i=all.begin(); i!=all.end(); ++i ){
+ for ( list<BSONObj>::iterator i=all.begin(); i!=all.end(); ++i ) {
BSONObj o = *i;
string name = o["_id"].String();
string host = o["host"].String();
long long maxSize = 0;
BSONElement maxSizeElem = o[ ShardFields::maxSize.name() ];
- if ( ! maxSizeElem.eoo() ){
+ if ( ! maxSizeElem.eoo() ) {
maxSize = maxSizeElem.numberLong();
}
bool isDraining = false;
BSONElement isDrainingElem = o[ ShardFields::draining.name() ];
- if ( ! isDrainingElem.eoo() ){
+ if ( ! isDrainingElem.eoo() ) {
isDraining = isDrainingElem.Bool();
}
Shard s( name , host , maxSize , isDraining );
_lookup[name] = s;
- _lookup[host] = s;
-
- // add rs name to lookup (if it exists)
- size_t pos;
- if ((pos = host.find('/', 0)) != string::npos) {
- _lookup[host.substr(0, pos)] = s;
- }
+ _installHost( host , s );
}
}
- bool isMember( const string& addr ){
- scoped_lock lk( _mutex );
- map<string,Shard>::iterator i = _lookup.find( addr );
- return i != _lookup.end();
- }
+ const Shard& find( const string& ident ) {
+ string mykey = ident;
- const Shard& find( const string& ident ){
{
- scoped_lock lk( _mutex );
- map<string,Shard>::iterator i = _lookup.find( ident );
+ // if its a replica set, just use set name
+ size_t pos = mykey.find( '/' );
+ if ( pos != string::npos )
+ mykey = mykey.substr(0,pos);
+ }
- // if normal find didn't find anything, try to find by rs name
- size_t pos;
- if ( i == _lookup.end() && (pos = ident.find('/', 0)) != string::npos) {
- i = _lookup.find( ident.substr(0, pos) );
- }
+ {
+ scoped_lock lk( _mutex );
+ map<string,Shard>::iterator i = _lookup.find( mykey );
if ( i != _lookup.end() )
return i->second;
}
-
+
// not in our maps, re-load all
reload();
scoped_lock lk( _mutex );
- map<string,Shard>::iterator i = _lookup.find( ident );
- uassert( 13129 , (string)"can't find shard for: " + ident , i != _lookup.end() );
- return i->second;
+ map<string,Shard>::iterator i = _lookup.find( mykey );
+ massert( 13129 , (string)"can't find shard for: " + mykey , i != _lookup.end() );
+ return i->second;
}
-
- void set( const string& name , const string& addr , bool setName = true , bool setAddr = true ){
- Shard s(name,addr);
+
+ void set( const string& name , const Shard& s , bool setName = true , bool setAddr = true ) {
scoped_lock lk( _mutex );
if ( setName )
_lookup[name] = s;
if ( setAddr )
- _lookup[addr] = s;
+ _installHost( s.getConnString() , s );
+ }
+
+ void _installHost( const string& host , const Shard& s ) {
+ _lookup[host] = s;
+
+ const ConnectionString& cs = s.getAddress();
+ if ( cs.type() == ConnectionString::SET ) {
+ if ( cs.getSetName().size() )
+ _lookup[ cs.getSetName() ] = s;
+
+ vector<HostAndPort> servers = cs.getServers();
+ for ( unsigned i=0; i<servers.size(); i++ ) {
+ _lookup[ servers[i].toString() ] = s;
+ }
+ }
}
- void remove( const string& name ){
+ void remove( const string& name ) {
scoped_lock lk( _mutex );
- for ( map<string,Shard>::iterator i = _lookup.begin(); i!=_lookup.end(); ){
+ for ( map<string,Shard>::iterator i = _lookup.begin(); i!=_lookup.end(); ) {
Shard s = i->second;
- if ( s.getName() == name ){
+ if ( s.getName() == name ) {
_lookup.erase(i++);
- } else {
+ }
+ else {
++i;
}
}
}
-
- void getAllShards( vector<Shard>& all ){
+
+ void getAllShards( vector<Shard>& all ) const {
scoped_lock lk( _mutex );
std::set<string> seen;
- for ( map<string,Shard>::iterator i = _lookup.begin(); i!=_lookup.end(); ++i ){
- Shard s = i->second;
+ for ( map<string,Shard>::const_iterator i = _lookup.begin(); i!=_lookup.end(); ++i ) {
+ const Shard& s = i->second;
if ( s.getName() == "config" )
continue;
if ( seen.count( s.getName() ) )
@@ -150,49 +158,131 @@ namespace mongo {
all.push_back( s );
}
}
+
+ bool isAShardNode( const string& addr ) const {
+ scoped_lock lk( _mutex );
+
+ // check direct nods or set names
+ map<string,Shard>::const_iterator i = _lookup.find( addr );
+ if ( i != _lookup.end() )
+ return true;
+
+ // check for set nodes
+ for ( map<string,Shard>::const_iterator i = _lookup.begin(); i!=_lookup.end(); ++i ) {
+ if ( i->first == "config" )
+ continue;
+
+ const Shard& s = i->second;
+ if ( s.containsNode( addr ) )
+ return true;
+ }
+
+ return false;
+ }
+
+ bool getShardMap( BSONObjBuilder& result , string& errmsg ) const {
+ scoped_lock lk( _mutex );
+
+ BSONObjBuilder b( _lookup.size() + 50 );
+
+ for ( map<string,Shard>::const_iterator i = _lookup.begin(); i!=_lookup.end(); ++i ) {
+ b.append( i->first , i->second.getConnString() );
+ }
+
+ result.append( "map" , b.obj() );
+
+ return true;
+ }
private:
map<string,Shard> _lookup;
- mongo::mutex _mutex;
+ mutable mongo::mutex _mutex;
} staticShardInfo;
+
- void Shard::setAddress( const string& addr , bool authoritative ){
- assert( _name.size() );
+ class CmdGetShardMap : public Command {
+ public:
+ CmdGetShardMap() : Command( "getShardMap" ){}
+ virtual void help( stringstream &help ) const { help<<"internal"; }
+ virtual LockType locktype() const { return NONE; }
+ virtual bool slaveOk() const { return true; }
+ virtual bool adminOnly() const { return true; }
+
+ virtual bool run(const string&, mongo::BSONObj&, std::string& errmsg , mongo::BSONObjBuilder& result, bool) {
+ return staticShardInfo.getShardMap( result , errmsg );
+ }
+ } cmdGetShardMap;
+
+
+ void Shard::_setAddr( const string& addr ) {
_addr = addr;
- if ( authoritative )
- staticShardInfo.set( _name , _addr , true , false );
+ if ( _addr.size() ) {
+ _cs = ConnectionString( addr , ConnectionString::SET );
+ _rsInit();
+ }
}
-
- void Shard::reset( const string& ident ){
+
+ void Shard::_rsInit() {
+ if ( _cs.type() == ConnectionString::SET ) {
+ string x = _cs.getSetName();
+ if ( x.size() == 0 ) {
+ warning() << "no set name for shard: " << _name << " " << _cs.toString() << endl;
+ }
+ assert( x.size() );
+ _rs = ReplicaSetMonitor::get( x , _cs.getServers() );
+ }
+ }
+
+ void Shard::setAddress( const ConnectionString& cs) {
+ assert( _name.size() );
+ _addr = cs.toString();
+ _cs = cs;
+ _rsInit();
+ staticShardInfo.set( _name , *this , true , false );
+ }
+
+ void Shard::reset( const string& ident ) {
const Shard& s = staticShardInfo.find( ident );
- uassert( 13128 , (string)"can't find shard for: " + ident , s.ok() );
+ massert( 13128 , (string)"can't find shard for: " + ident , s.ok() );
_name = s._name;
_addr = s._addr;
+ _cs = s._cs;
+ _rsInit();
_maxSize = s._maxSize;
_isDraining = s._isDraining;
}
-
- void Shard::getAllShards( vector<Shard>& all ){
+
+ bool Shard::containsNode( const string& node ) const {
+ if ( _addr == node )
+ return true;
+
+ if ( _rs && _rs->contains( node ) )
+ return true;
+
+ return false;
+ }
+
+ void Shard::getAllShards( vector<Shard>& all ) {
staticShardInfo.getAllShards( all );
}
- bool Shard::isAShard( const string& ident ){
- return staticShardInfo.isMember( ident );
+ bool Shard::isAShardNode( const string& ident ) {
+ return staticShardInfo.isAShardNode( ident );
}
- void Shard::printShardInfo( ostream& out ){
+ void Shard::printShardInfo( ostream& out ) {
vector<Shard> all;
getAllShards( all );
for ( unsigned i=0; i<all.size(); i++ )
out << all[i].toString() << "\n";
out.flush();
}
-
+
BSONObj Shard::runCommand( const string& db , const BSONObj& cmd ) const {
ScopedDbConnection conn( this );
BSONObj res;
bool ok = conn->runCommand( db , cmd , res );
- if ( ! ok ){
+ if ( ! ok ) {
stringstream ss;
ss << "runCommand (" << cmd << ") on shard (" << _name << ") failed : " << res;
throw UserException( 13136 , ss.str() );
@@ -201,49 +291,50 @@ namespace mongo {
conn.done();
return res;
}
-
+
ShardStatus Shard::getStatus() const {
return ShardStatus( *this , runCommand( "admin" , BSON( "serverStatus" << 1 ) ) );
}
-
- void Shard::reloadShardInfo(){
+
+ void Shard::reloadShardInfo() {
staticShardInfo.reload();
}
- bool Shard::isMember( const string& addr ){
- return staticShardInfo.isMember( addr );
- }
-
- void Shard::removeShard( const string& name ){
+ void Shard::removeShard( const string& name ) {
staticShardInfo.remove( name );
}
- Shard Shard::pick(){
+ Shard Shard::pick( const Shard& current ) {
vector<Shard> all;
staticShardInfo.getAllShards( all );
- if ( all.size() == 0 ){
+ if ( all.size() == 0 ) {
staticShardInfo.reload();
staticShardInfo.getAllShards( all );
if ( all.size() == 0 )
return EMPTY;
}
-
+
+ // if current shard was provided, pick a different shard only if it is a better choice
ShardStatus best = all[0].getStatus();
-
- for ( size_t i=1; i<all.size(); i++ ){
+ if ( current != EMPTY ) {
+ best = current.getStatus();
+ }
+
+ for ( size_t i=0; i<all.size(); i++ ) {
ShardStatus t = all[i].getStatus();
if ( t < best )
best = t;
}
- log(1) << "picking shard: " << best << endl;
+ log(1) << "best shard for new allocation is " << best << endl;
return best.shard();
}
ShardStatus::ShardStatus( const Shard& shard , const BSONObj& obj )
: _shard( shard ) {
_mapped = obj.getFieldDotted( "mem.mapped" ).numberLong();
+ _hasOpsQueued = obj["writeBacksQueued"].Bool();
_writeLock = 0; // TODO
}
diff --git a/s/shard.h b/s/shard.h
index ee807d8..836ffe7 100644
--- a/s/shard.h
+++ b/s/shard.h
@@ -1,4 +1,4 @@
-// shard.h
+// @file shard.h
/**
* Copyright (C) 2008 10gen Inc.
@@ -26,48 +26,56 @@ namespace mongo {
class ShardConnection;
class ShardStatus;
+ /*
+ * A "shard" is a database (replica pair typically) which represents
+ * one partition of the overall database.
+ */
+
class Shard {
public:
Shard()
- : _name("") , _addr("") , _maxSize(0) , _isDraining( false ){
+ : _name("") , _addr("") , _maxSize(0) , _isDraining( false ) {
}
Shard( const string& name , const string& addr, long long maxSize = 0 , bool isDraining = false )
- : _name(name) , _addr( addr ) , _maxSize( maxSize ) , _isDraining( isDraining ){
+ : _name(name) , _addr( addr ) , _maxSize( maxSize ) , _isDraining( isDraining ) {
+ _setAddr( addr );
}
- Shard( const string& ident ){
+ Shard( const string& ident ) {
reset( ident );
}
Shard( const Shard& other )
- : _name( other._name ) , _addr( other._addr ) , _maxSize( other._maxSize ) , _isDraining( other._isDraining ){
+ : _name( other._name ) , _addr( other._addr ) , _cs( other._cs ) ,
+ _maxSize( other._maxSize ) , _isDraining( other._isDraining ) , _rs( other._rs ) {
}
Shard( const Shard* other )
- : _name( other->_name ) , _addr( other->_addr ), _maxSize( other->_maxSize ) , _isDraining( other->_isDraining ){
+ : _name( other->_name ) , _addr( other->_addr ), _cs( other->_cs ) ,
+ _maxSize( other->_maxSize ) , _isDraining( other->_isDraining ) , _rs( other->_rs ) {
}
-
- static Shard make( const string& ident ){
+
+ static Shard make( const string& ident ) {
Shard s;
s.reset( ident );
return s;
}
- static bool isAShard( const string& ident );
-
/**
* @param ident either name or address
*/
void reset( const string& ident );
+
+ void setAddress( const ConnectionString& cs );
- void setAddress( const string& addr , bool authoritative = false );
+ ConnectionString getAddress() const { return _cs; }
string getName() const {
assert( _name.size() );
return _name;
}
-
+
string getConnString() const {
assert( _addr.size() );
return _addr;
@@ -92,7 +100,7 @@ namespace mongo {
bool operator==( const Shard& s ) const {
bool n = _name == s._name;
bool a = _addr == s._addr;
-
+
assert( n == a ); // names and address are 1 to 1
return n;
}
@@ -107,7 +115,7 @@ namespace mongo {
bool operator==( const string& s ) const {
return _name == s || _addr == s;
}
-
+
bool operator!=( const string& s ) const {
return _name != s && _addr != s;
}
@@ -115,44 +123,58 @@ namespace mongo {
bool operator<(const Shard& o) const {
return _name < o._name;
}
-
+
bool ok() const {
return _addr.size() > 0 && _addr.size() > 0;
}
-
+
BSONObj runCommand( const string& db , const string& simple ) const {
return runCommand( db , BSON( simple << 1 ) );
}
BSONObj runCommand( const string& db , const BSONObj& cmd ) const ;
-
+
ShardStatus getStatus() const ;
+ /**
+ * mostly for replica set
+ * retursn true if node is the shard
+ * of if the replica set contains node
+ */
+ bool containsNode( const string& node ) const;
+
static void getAllShards( vector<Shard>& all );
static void printShardInfo( ostream& out );
-
+
/**
- * picks a Shard for more load
+ * @parm current - shard where the chunk/database currently lives in
+ * @return the currently emptiest shard, if best then current, or EMPTY
*/
- static Shard pick();
-
+ static Shard pick( const Shard& current = EMPTY );
+
static void reloadShardInfo();
static void removeShard( const string& name );
- static bool isMember( const string& addr );
+ static bool isAShardNode( const string& ident );
static Shard EMPTY;
-
+
private:
+
+ void _rsInit();
+ void _setAddr( const string& addr );
+
string _name;
string _addr;
- long long _maxSize; // in MBytes, 0 is unlimited
+ ConnectionString _cs;
+ long long _maxSize; // in MBytes, 0 is unlimited
bool _isDraining; // shard is currently being removed
+ ReplicaSetMonitorPtr _rs;
};
class ShardStatus {
public:
-
+
ShardStatus( const Shard& shard , const BSONObj& obj );
friend ostream& operator << (ostream& out, const ShardStatus& s) {
@@ -162,14 +184,14 @@ namespace mongo {
string toString() const {
stringstream ss;
- ss << "shard: " << _shard << " mapped: " << _mapped << " writeLock: " << _writeLock;
+ ss << "shard: " << _shard << " mapped: " << _mapped << " writeLock: " << _writeLock;
return ss.str();
}
- bool operator<( const ShardStatus& other ) const{
+ bool operator<( const ShardStatus& other ) const {
return _mapped < other._mapped;
}
-
+
Shard shard() const {
return _shard;
}
@@ -178,9 +200,14 @@ namespace mongo {
return _mapped;
}
+ bool hasOpsQueued() const {
+ return _hasOpsQueued;
+ }
+
private:
Shard _shard;
long long _mapped;
+ bool _hasOpsQueued; // true if 'writebacks' are pending
double _writeLock;
};
@@ -195,19 +222,19 @@ namespace mongo {
void done();
void kill();
- DBClientBase& conn(){
+ DBClientBase& conn() {
_finishInit();
assert( _conn );
return *_conn;
}
-
- DBClientBase* operator->(){
+
+ DBClientBase* operator->() {
_finishInit();
assert( _conn );
return _conn;
}
- DBClientBase* get(){
+ DBClientBase* get() {
_finishInit();
assert( _conn );
return _conn;
@@ -224,7 +251,7 @@ namespace mongo {
static void sync();
- void donotCheckVersion(){
+ void donotCheckVersion() {
_setVersion = false;
_finishedInit = true;
}
@@ -236,11 +263,11 @@ namespace mongo {
/** checks all of my thread local connections for the version of this ns */
static void checkMyConnectionVersions( const string & ns );
-
+
private:
void _init();
void _finishInit();
-
+
bool _finishedInit;
string _addr;
diff --git a/s/shard_version.cpp b/s/shard_version.cpp
new file mode 100644
index 0000000..0f3e80f
--- /dev/null
+++ b/s/shard_version.cpp
@@ -0,0 +1,151 @@
+// @file shard_version.cpp
+
+/**
+* Copyright (C) 2010 10gen Inc.
+*
+* This program is free software: you can redistribute it and/or modify
+* it under the terms of the GNU Affero General Public License, version 3,
+* as published by the Free Software Foundation.
+*
+* This program is distributed in the hope that it will be useful,
+* but WITHOUT ANY WARRANTY; without even the implied warranty of
+* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+* GNU Affero General Public License for more details.
+*
+* You should have received a copy of the GNU Affero General Public License
+* along with this program. If not, see <http://www.gnu.org/licenses/>.
+*/
+
+#include "pch.h"
+
+#include "chunk.h"
+#include "config.h"
+#include "grid.h"
+#include "util.h"
+#include "shard.h"
+#include "writeback_listener.h"
+
+#include "shard_version.h"
+
+namespace mongo {
+
+ // when running in sharded mode, use chunk shard version control
+
+ static bool checkShardVersion( DBClientBase & conn , const string& ns , bool authoritative = false , int tryNumber = 1 );
+ static void resetShardVersion( DBClientBase * conn );
+
+ void installChunkShardVersioning() {
+ //
+ // Overriding no-op behavior in shardconnection.cpp
+ //
+ // TODO: Better encapsulate this mechanism.
+ //
+ checkShardVersionCB = checkShardVersion;
+ resetShardVersionCB = resetShardVersion;
+ }
+
+ struct ConnectionShardStatus {
+
+ typedef unsigned long long S;
+
+ ConnectionShardStatus()
+ : _mutex( "ConnectionShardStatus" ) {
+ }
+
+ S getSequence( DBClientBase * conn , const string& ns ) {
+ scoped_lock lk( _mutex );
+ return _map[conn][ns];
+ }
+
+ void setSequence( DBClientBase * conn , const string& ns , const S& s ) {
+ scoped_lock lk( _mutex );
+ _map[conn][ns] = s;
+ }
+
+ void reset( DBClientBase * conn ) {
+ scoped_lock lk( _mutex );
+ _map.erase( conn );
+ }
+
+ // protects _map
+ mongo::mutex _mutex;
+
+ // a map from a connection into ChunkManager's sequence number for each namespace
+ map<DBClientBase*, map<string,unsigned long long> > _map;
+
+ } connectionShardStatus;
+
+ void resetShardVersion( DBClientBase * conn ) {
+ connectionShardStatus.reset( conn );
+ }
+
+ /**
+ * @return true if had to do something
+ */
+ bool checkShardVersion( DBClientBase& conn , const string& ns , bool authoritative , int tryNumber ) {
+ // TODO: cache, optimize, etc...
+
+ WriteBackListener::init( conn );
+
+ DBConfigPtr conf = grid.getDBConfig( ns );
+ if ( ! conf )
+ return false;
+
+ unsigned long long officialSequenceNumber = 0;
+
+ ChunkManagerPtr manager;
+ const bool isSharded = conf->isSharded( ns );
+ if ( isSharded ) {
+ manager = conf->getChunkManager( ns , authoritative );
+ officialSequenceNumber = manager->getSequenceNumber();
+ }
+
+ // has the ChunkManager been reloaded since the last time we updated the connection-level version?
+ // (ie, last time we issued the setShardVersions below)
+ unsigned long long sequenceNumber = connectionShardStatus.getSequence(&conn,ns);
+ if ( sequenceNumber == officialSequenceNumber ) {
+ return false;
+ }
+
+
+ ShardChunkVersion version = 0;
+ if ( isSharded ) {
+ version = manager->getVersion( Shard::make( conn.getServerAddress() ) );
+ }
+
+ log(2) << " have to set shard version for conn: " << &conn << " ns:" << ns
+ << " my last seq: " << sequenceNumber << " current: " << officialSequenceNumber
+ << " version: " << version << " manager: " << manager.get()
+ << endl;
+
+ BSONObj result;
+ if ( setShardVersion( conn , ns , version , authoritative , result ) ) {
+ // success!
+ log(1) << " setShardVersion success!" << endl;
+ connectionShardStatus.setSequence( &conn , ns , officialSequenceNumber );
+ return true;
+ }
+
+ log(1) << " setShardVersion failed!\n" << result << endl;
+
+ if ( result.getBoolField( "need_authoritative" ) )
+ massert( 10428 , "need_authoritative set but in authoritative mode already" , ! authoritative );
+
+ if ( ! authoritative ) {
+ checkShardVersion( conn , ns , 1 , tryNumber + 1 );
+ return true;
+ }
+
+ if ( tryNumber < 4 ) {
+ log(1) << "going to retry checkShardVersion" << endl;
+ sleepmillis( 10 );
+ checkShardVersion( conn , ns , 1 , tryNumber + 1 );
+ return true;
+ }
+
+ log() << " setShardVersion failed: " << result << endl;
+ massert( 10429 , (string)"setShardVersion failed! " + result.jsonString() , 0 );
+ return true;
+ }
+
+} // namespace mongo
diff --git a/s/d_util.cpp b/s/shard_version.h
index a750fbc..023b7fc 100644
--- a/s/d_util.cpp
+++ b/s/shard_version.h
@@ -1,7 +1,7 @@
-// util.cpp
+// @file shard_version.h
/**
-* Copyright (C) 2008 10gen Inc.
+* Copyright (C) 2010 10gen Inc.
*
* This program is free software: you can redistribute it and/or modify
* it under the terms of the GNU Affero General Public License, version 3,
@@ -16,26 +16,16 @@
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
-
-/**
- these are commands that live in mongod
- mostly around shard management and checking
- */
-
-#include "pch.h"
-#include "util.h"
-
-using namespace std;
+#pragma once
namespace mongo {
- bool checkShardVersion( DBClientBase & conn , const string& ns , bool authoritative , int tryNumber ){
- // no-op in mongod
- return false;
- }
-
- void resetShardVersion( DBClientBase * conn ){
- // no-op in mongod
- }
+ /*
+ * Install chunk shard vesion callbaks in shardconnection code. This activates
+ * the chunk shard version control that mongos needs.
+ *
+ * MUST be called before accepting any connections.
+ */
+ void installChunkShardVersioning();
-}
+} // namespace mongo
diff --git a/s/shardconnection.cpp b/s/shardconnection.cpp
index 694693b..d05f5b1 100644
--- a/s/shardconnection.cpp
+++ b/s/shardconnection.cpp
@@ -23,7 +23,24 @@
#include <set>
namespace mongo {
-
+
+ // The code in shardconnection may run not only in mongos context. When elsewhere, chunk shard versioning
+ // is disabled. To enable chunk shard versioning, provide the check/resetShardVerionCB's below
+ //
+ // TODO: better encapsulate this mechanism.
+
+ bool defaultCheckShardVersion( DBClientBase & conn , const string& ns , bool authoritative , int tryNumber ) {
+ // no-op in mongod
+ return false;
+ }
+
+ void defaultResetShardVersion( DBClientBase * conn ) {
+ // no-op in mongod
+ }
+
+ boost::function4<bool, DBClientBase&, const string&, bool, int> checkShardVersionCB = defaultCheckShardVersion;
+ boost::function1<void, DBClientBase*> resetShardVersionCB = defaultResetShardVersion;
+
/**
* holds all the actual db connections for a client to various servers
* 1 pre thread, so don't have to worry about thread safety
@@ -31,39 +48,22 @@ namespace mongo {
class ClientConnections : boost::noncopyable {
public:
struct Status : boost::noncopyable {
- Status() : created(0), avail(0){}
+ Status() : created(0), avail(0) {}
- long long created;
+ long long created;
DBClientBase* avail;
};
- Nullstream& debug( Status * s = 0 , const string& addr = "" ){
- static int ll = 9;
+ ClientConnections() {}
- if ( logLevel < ll )
- return nullstream;
- Nullstream& l = log(ll);
-
- l << "ClientConnections DEBUG " << this << " ";
- if ( s ){
- l << "s: " << s << " addr: " << addr << " ";
- }
- return l;
- }
-
- ClientConnections() : _mutex("ClientConnections") {
- debug() << " NEW " << endl;
- }
-
- ~ClientConnections(){
- debug() << " KILLING " << endl;
- for ( map<string,Status*>::iterator i=_hosts.begin(); i!=_hosts.end(); ++i ){
+ ~ClientConnections() {
+ for ( HostMap::iterator i=_hosts.begin(); i!=_hosts.end(); ++i ) {
string addr = i->first;
Status* ss = i->second;
assert( ss );
- if ( ss->avail ){
- /* if we're shutting down, don't want to initiate release mechanism as it is slow,
+ if ( ss->avail ) {
+ /* if we're shutting down, don't want to initiate release mechanism as it is slow,
and isn't needed since all connections will be closed anyway */
if ( inShutdown() )
delete ss->avail;
@@ -75,49 +75,41 @@ namespace mongo {
}
_hosts.clear();
}
-
- DBClientBase * get( const string& addr , const string& ns ){
+
+ DBClientBase * get( const string& addr , const string& ns ) {
_check( ns );
- scoped_lock lk( _mutex );
+
Status* &s = _hosts[addr];
if ( ! s )
s = new Status();
-
- debug( s , addr ) << "WANT ONE pool avail: " << s->avail << endl;
-
- if ( s->avail ){
+
+ if ( s->avail ) {
DBClientBase* c = s->avail;
s->avail = 0;
- debug( s , addr ) << "GOT " << c << endl;
pool.onHandedOut( c );
return c;
}
- debug() << "CREATING NEW CONNECTION" << endl;
s->created++;
return pool.get( addr );
}
-
- void done( const string& addr , DBClientBase* conn ){
- scoped_lock lk( _mutex );
+
+ void done( const string& addr , DBClientBase* conn ) {
Status* s = _hosts[addr];
assert( s );
- if ( s->avail ){
- debug( s , addr ) << "DONE WITH TEMP" << endl;
+ if ( s->avail ) {
release( addr , conn );
return;
}
s->avail = conn;
- debug( s , addr ) << "PUSHING: " << conn << endl;
}
-
- void sync(){
- scoped_lock lk( _mutex );
- for ( map<string,Status*>::iterator i=_hosts.begin(); i!=_hosts.end(); ++i ){
+
+ void sync() {
+ for ( HostMap::iterator i=_hosts.begin(); i!=_hosts.end(); ++i ) {
string addr = i->first;
Status* ss = i->second;
- if ( ss->avail ){
+ if ( ss->avail ) {
ss->avail->getLastError();
release( addr , ss->avail );
ss->avail = 0;
@@ -127,63 +119,67 @@ namespace mongo {
_hosts.clear();
}
- void checkVersions( const string& ns ){
+ void checkVersions( const string& ns ) {
vector<Shard> all;
Shard::getAllShards( all );
- scoped_lock lk( _mutex );
- for ( unsigned i=0; i<all.size(); i++ ){
+ for ( unsigned i=0; i<all.size(); i++ ) {
Status* &s = _hosts[all[i].getConnString()];
if ( ! s )
s = new Status();
}
- for ( map<string,Status*>::iterator i=_hosts.begin(); i!=_hosts.end(); ++i ){
- if ( ! Shard::isAShard( i->first ) )
+ for ( HostMap::iterator i=_hosts.begin(); i!=_hosts.end(); ++i ) {
+ if ( ! Shard::isAShardNode( i->first ) )
continue;
Status* ss = i->second;
assert( ss );
if ( ! ss->avail )
ss->avail = pool.get( i->first );
- checkShardVersion( *ss->avail , ns );
+ checkShardVersionCB( *ss->avail , ns , false , 1 );
}
}
- void release( const string& addr , DBClientBase * conn ){
- resetShardVersion( conn );
+ void release( const string& addr , DBClientBase * conn ) {
+ resetShardVersionCB( conn );
BSONObj res;
-
+
try {
- if ( conn->simpleCommand( "admin" , &res , "unsetSharding" ) ){
+ if ( conn->simpleCommand( "admin" , &res , "unsetSharding" ) ) {
pool.release( addr , conn );
}
else {
- log(LL_ERROR) << " couldn't unset sharding :( " << res << endl;
+ error() << "unset sharding failed : " << res << endl;
delete conn;
}
}
- catch ( std::exception& e ){
- log(LL_ERROR) << "couldn't unsert sharding : " << e.what() << endl;
+ catch ( SocketException& e ) {
+ // server down or something
+ LOG(1) << "socket exception trying to unset sharding: " << e.toString() << endl;
+ delete conn;
+ }
+ catch ( std::exception& e ) {
+ error() << "couldn't unset sharding : " << e.what() << endl;
delete conn;
}
}
-
- void _check( const string& ns ){
+
+ void _check( const string& ns ) {
if ( ns.size() == 0 || _seenNS.count( ns ) )
return;
_seenNS.insert( ns );
checkVersions( ns );
}
-
- map<string,Status*> _hosts;
- mongo::mutex _mutex;
+
+ typedef map<string,Status*,DBConnectionPool::serverNameCompare> HostMap;
+ HostMap _hosts;
set<string> _seenNS;
// -----
-
+
static thread_specific_ptr<ClientConnections> _perThread;
- static ClientConnections* get(){
+ static ClientConnections* threadInstance() {
ClientConnections* cc = _perThread.get();
- if ( ! cc ){
+ if ( ! cc ) {
cc = new ClientConnections();
_perThread.reset( cc );
}
@@ -202,57 +198,58 @@ namespace mongo {
: _addr( s.getConnString() ) , _ns( ns ) {
_init();
}
-
+
ShardConnection::ShardConnection( const string& addr , const string& ns )
: _addr( addr ) , _ns( ns ) {
_init();
}
-
- void ShardConnection::_init(){
+
+ void ShardConnection::_init() {
assert( _addr.size() );
- _conn = ClientConnections::get()->get( _addr , _ns );
+ _conn = ClientConnections::threadInstance()->get( _addr , _ns );
_finishedInit = false;
}
- void ShardConnection::_finishInit(){
+ void ShardConnection::_finishInit() {
if ( _finishedInit )
return;
_finishedInit = true;
-
- if ( _ns.size() ){
- _setVersion = checkShardVersion( *_conn , _ns );
+
+ if ( _ns.size() ) {
+ _setVersion = checkShardVersionCB( *_conn , _ns , false , 1 );
}
else {
_setVersion = false;
}
-
+
}
- void ShardConnection::done(){
- if ( _conn ){
- ClientConnections::get()->done( _addr , _conn );
+ void ShardConnection::done() {
+ if ( _conn ) {
+ ClientConnections::threadInstance()->done( _addr , _conn );
_conn = 0;
_finishedInit = true;
}
}
- void ShardConnection::kill(){
- if ( _conn ){
+ void ShardConnection::kill() {
+ if ( _conn ) {
+ resetShardVersionCB( _conn );
delete _conn;
_conn = 0;
_finishedInit = true;
}
}
- void ShardConnection::sync(){
- ClientConnections::get()->sync();
+ void ShardConnection::sync() {
+ ClientConnections::threadInstance()->sync();
}
- bool ShardConnection::runCommand( const string& db , const BSONObj& cmd , BSONObj& res ){
+ bool ShardConnection::runCommand( const string& db , const BSONObj& cmd , BSONObj& res ) {
assert( _conn );
bool ok = _conn->runCommand( db , cmd , res );
- if ( ! ok ){
- if ( res["code"].numberInt() == StaleConfigInContextCode ){
+ if ( ! ok ) {
+ if ( res["code"].numberInt() == StaleConfigInContextCode ) {
string big = res["errmsg"].String();
string ns,raw;
massert( 13409 , (string)"can't parse ns from: " + big , StaleConfigException::parse( big , ns , raw ) );
@@ -263,12 +260,12 @@ namespace mongo {
return ok;
}
- void ShardConnection::checkMyConnectionVersions( const string & ns ){
- ClientConnections::get()->checkVersions( ns );
+ void ShardConnection::checkMyConnectionVersions( const string & ns ) {
+ ClientConnections::threadInstance()->checkVersions( ns );
}
ShardConnection::~ShardConnection() {
- if ( _conn ){
+ if ( _conn ) {
if ( ! _conn->isFailed() ) {
/* see done() comments above for why we log this line */
log() << "~ScopedDBConnection: _conn != null" << endl;
diff --git a/s/shardkey.cpp b/s/shardkey.cpp
index e4deeec..84cdb4b 100644
--- a/s/shardkey.cpp
+++ b/s/shardkey.cpp
@@ -20,6 +20,7 @@
#include "chunk.h"
#include "../db/jsobj.h"
#include "../util/unittest.h"
+#include "../util/timer.h"
namespace mongo {
@@ -30,12 +31,12 @@ namespace mongo {
BSONObjBuilder max;
BSONObjIterator it(p);
- while (it.more()){
+ while (it.more()) {
BSONElement e (it.next());
min.appendMinKey(e.fieldName());
max.appendMaxKey(e.fieldName());
}
-
+
gMin = min.obj();
gMax = max.obj();
}
@@ -49,11 +50,11 @@ namespace mongo {
}
bool ShardKeyPattern::hasShardKey( const BSONObj& obj ) const {
- /* this is written s.t. if obj has lots of fields, if the shard key fields are early,
+ /* this is written s.t. if obj has lots of fields, if the shard key fields are early,
it is fast. so a bit more work to try to be semi-fast.
*/
- for(set<string>::const_iterator it = patternfields.begin(); it != patternfields.end(); ++it){
+ for(set<string>::const_iterator it = patternfields.begin(); it != patternfields.end(); ++it) {
if(obj.getFieldDotted(it->c_str()).eoo())
return false;
}
@@ -63,28 +64,90 @@ namespace mongo {
bool ShardKeyPattern::isPrefixOf( const BSONObj& otherPattern ) const {
BSONObjIterator a( pattern );
BSONObjIterator b( otherPattern );
-
- while ( a.more() && b.more() ){
+
+ while ( a.more() && b.more() ) {
BSONElement x = a.next();
BSONElement y = b.next();
if ( strcmp( x.fieldName() , y.fieldName() ) )
return false;
}
-
+
return ! a.more();
}
-
+
string ShardKeyPattern::toString() const {
return pattern.toString();
}
-
- /* things to test for compound :
+
+ BSONObj ShardKeyPattern::moveToFront(const BSONObj& obj) const {
+ vector<const char*> keysToMove;
+ keysToMove.push_back("_id");
+ BSONForEach(e, pattern) {
+ if (strchr(e.fieldName(), '.') == NULL)
+ keysToMove.push_back(e.fieldName());
+ }
+
+ if (keysToMove.size() == 1) {
+ return obj;
+
+ }
+ else {
+ BufBuilder buf (obj.objsize());
+ buf.appendNum(obj.objsize());
+
+ vector<pair<const char*, size_t> > copies;
+ pair<const char*, size_t> toCopy ((const char*)NULL, 0); // C++ NULL isn't a pointer type yet
+
+ BSONForEach(e, obj) {
+ bool moveToFront = false;
+ for (vector<const char*>::const_iterator it(keysToMove.begin()), end(keysToMove.end()); it!=end; ++it) {
+ if (strcmp(e.fieldName(), *it) == 0) {
+ moveToFront = true;
+ break;
+ }
+ }
+
+ if (moveToFront) {
+ buf.appendBuf(e.fieldName()-1, e.size());
+ if (toCopy.first) {
+ copies.push_back(toCopy);
+ toCopy.first = NULL;
+ }
+ }
+ else {
+ if (!toCopy.first) {
+ toCopy.first = e.fieldName()-1;
+ toCopy.second = e.size();
+ }
+ else {
+ toCopy.second += e.size();
+ }
+ }
+ }
+
+ for (vector<pair<const char*, size_t> >::const_iterator it(copies.begin()), end(copies.end()); it!=end; ++it) {
+ buf.appendBuf(it->first, it->second);
+ }
+
+ if (toCopy.first) {
+ buf.appendBuf(toCopy.first, toCopy.second);
+ }
+
+ buf.appendChar('\0');
+
+ BSONObj out (buf.buf(), true);
+ buf.decouple();
+ return out;
+ }
+ }
+
+ /* things to test for compound :
\ middle (deprecating?)
*/
class ShardKeyUnitTest : public UnitTest {
public:
-
- void testIsPrefixOf(){
+
+ void testIsPrefixOf() {
{
ShardKeyPattern k( BSON( "x" << 1 ) );
assert( ! k.isPrefixOf( BSON( "a" << 1 ) ) );
@@ -92,7 +155,7 @@ namespace mongo {
assert( k.isPrefixOf( BSON( "x" << 1 << "a" << 1 ) ) );
assert( ! k.isPrefixOf( BSON( "a" << 1 << "x" << 1 ) ) );
}
- {
+ {
ShardKeyPattern k( BSON( "x" << 1 << "y" << 1 ) );
assert( ! k.isPrefixOf( BSON( "x" << 1 ) ) );
assert( ! k.isPrefixOf( BSON( "x" << 1 << "z" << 1 ) ) );
@@ -100,8 +163,8 @@ namespace mongo {
assert( k.isPrefixOf( BSON( "x" << 1 << "y" << 1 << "z" << 1 ) ) );
}
}
-
- void hasshardkeytest() {
+
+ void hasshardkeytest() {
BSONObj x = fromjson("{ zid : \"abcdefg\", num: 1.0, name: \"eliot\" }");
ShardKeyPattern k( BSON( "num" << 1 ) );
assert( k.hasShardKey(x) );
@@ -117,31 +180,68 @@ namespace mongo {
}
- void extractkeytest() {
+ void extractkeytest() {
ShardKeyPattern k( fromjson("{a:1,'sub.b':-1,'sub.c':1}") );
BSONObj x = fromjson("{a:1,'sub.b':2,'sub.c':3}");
assert( k.extractKey( fromjson("{a:1,sub:{b:2,c:3}}") ).woEqual(x) );
assert( k.extractKey( fromjson("{sub:{b:2,c:3},a:1}") ).woEqual(x) );
}
- void run(){
+ void moveToFrontTest() {
+ ShardKeyPattern sk (BSON("a" << 1 << "b" << 1));
+
+ BSONObj ret;
+
+ ret = sk.moveToFront(BSON("z" << 1 << "_id" << 1 << "y" << 1 << "a" << 1 << "x" << 1 << "b" << 1 << "w" << 1));
+ assert(ret.woEqual(BSON("_id" << 1 << "a" << 1 << "b" << 1 << "z" << 1 << "y" << 1 << "x" << 1 << "w" << 1)));
+
+ ret = sk.moveToFront(BSON("_id" << 1 << "a" << 1 << "b" << 1 << "z" << 1 << "y" << 1 << "x" << 1 << "w" << 1));
+ assert(ret.woEqual(BSON("_id" << 1 << "a" << 1 << "b" << 1 << "z" << 1 << "y" << 1 << "x" << 1 << "w" << 1)));
+
+ ret = sk.moveToFront(BSON("z" << 1 << "y" << 1 << "a" << 1 << "b" << 1 << "Z" << 1 << "Y" << 1));
+ assert(ret.woEqual(BSON("a" << 1 << "b" << 1 << "z" << 1 << "y" << 1 << "Z" << 1 << "Y" << 1)));
+
+ }
+
+ void moveToFrontBenchmark(int numFields) {
+ BSONObjBuilder bb;
+ bb.append("_id", 1);
+ for (int i=0; i < numFields; i++)
+ bb.append(BSONObjBuilder::numStr(i), 1);
+ bb.append("key", 1);
+ BSONObj o = bb.obj();
+
+ ShardKeyPattern sk (BSON("key" << 1));
+
+ Timer t;
+ const int iterations = 100*1000;
+ for (int i=0; i< iterations; i++) {
+ sk.moveToFront(o);
+ }
+
+ const double secs = t.micros() / 1000000.0;
+ const double ops_per_sec = iterations / secs;
+
+ cout << "moveToFront (" << numFields << " fields) secs: " << secs << " ops_per_sec: " << ops_per_sec << endl;
+ }
+ void run() {
extractkeytest();
ShardKeyPattern k( BSON( "key" << 1 ) );
-
+
BSONObj min = k.globalMin();
// cout << min.jsonString(TenGen) << endl;
BSONObj max = k.globalMax();
-
+
BSONObj k1 = BSON( "key" << 5 );
assert( k.compare( min , max ) < 0 );
assert( k.compare( min , k1 ) < 0 );
assert( k.compare( max , min ) > 0 );
assert( k.compare( min , min ) == 0 );
-
+
hasshardkeytest();
assert( k.hasShardKey( k1 ) );
assert( ! k.hasShardKey( BSON( "key2" << 1 ) ) );
@@ -150,12 +250,20 @@ namespace mongo {
BSONObj b = BSON( "key" << 999 );
assert( k.compare(a,b) < 0 );
-
+
testIsPrefixOf();
// add middle multitype tests
+ moveToFrontTest();
+
+ if (0) { // toggle to run benchmark
+ moveToFrontBenchmark(0);
+ moveToFrontBenchmark(10);
+ moveToFrontBenchmark(100);
+ }
+
log(1) << "shardKeyTest passed" << endl;
}
} shardKeyTest;
-
+
} // namespace mongo
diff --git a/s/shardkey.h b/s/shardkey.h
index 976bbef..96301ff 100644
--- a/s/shardkey.h
+++ b/s/shardkey.h
@@ -21,7 +21,7 @@
#include "../client/dbclient.h"
namespace mongo {
-
+
class Chunk;
/* A ShardKeyPattern is a pattern indicating what data to extract from the object to make the shard key from.
@@ -30,10 +30,10 @@ namespace mongo {
class ShardKeyPattern {
public:
ShardKeyPattern( BSONObj p = BSONObj() );
-
+
/**
global min is the lowest possible value for this key
- e.g. { num : MinKey }
+ e.g. { num : MinKey }
*/
BSONObj globalMin() const { return gMin; }
@@ -42,15 +42,15 @@ namespace mongo {
*/
BSONObj globalMax() const { return gMax; }
- bool isGlobalMin( const BSONObj& k ) const{
+ bool isGlobalMin( const BSONObj& k ) const {
return k.woCompare( globalMin() ) == 0;
}
- bool isGlobalMax( const BSONObj& k ) const{
+ bool isGlobalMax( const BSONObj& k ) const {
return k.woCompare( globalMax() ) == 0;
}
-
- bool isGlobal( const BSONObj& k ) const{
+
+ bool isGlobal( const BSONObj& k ) const {
return isGlobalMin( k ) || isGlobalMax( k );
}
@@ -60,22 +60,25 @@ namespace mongo {
l > r positive
*/
int compare( const BSONObj& l , const BSONObj& r ) const;
-
+
/**
@return whether or not obj has all fields in this shard key pattern
- e.g.
- ShardKey({num:1}).hasShardKey({ name:"joe", num:3 }) is true
+ e.g.
+ ShardKey({num:1}).hasShardKey({ name:"joe", num:3 }) is true
*/
bool hasShardKey( const BSONObj& obj ) const;
-
+
BSONObj key() const { return pattern; }
string toString() const;
BSONObj extractKey(const BSONObj& from) const;
-
+
+ bool partOfShardKey(const char* key ) const {
+ return pattern.hasField(key);
+ }
bool partOfShardKey(const string& key ) const {
- return patternfields.count( key ) > 0;
+ return pattern.hasField(key.c_str());
}
/**
@@ -83,7 +86,12 @@ namespace mongo {
* true if 'this' is a prefix (not necessarily contained) of 'otherPattern'.
*/
bool isPrefixOf( const BSONObj& otherPattern ) const;
-
+
+ /**
+ * @return BSONObj with _id and shardkey at front. May return original object.
+ */
+ BSONObj moveToFront(const BSONObj& obj) const;
+
private:
BSONObj pattern;
BSONObj gMin;
@@ -93,10 +101,10 @@ namespace mongo {
set<string> patternfields;
};
- inline BSONObj ShardKeyPattern::extractKey(const BSONObj& from) const {
+ inline BSONObj ShardKeyPattern::extractKey(const BSONObj& from) const {
BSONObj k = from.extractFields(pattern);
uassert(13334, "Shard Key must be less than 512 bytes", k.objsize() < 512);
return k;
}
-}
+}
diff --git a/s/stats.cpp b/s/stats.cpp
index bb7a975..460ada3 100644
--- a/s/stats.cpp
+++ b/s/stats.cpp
@@ -20,7 +20,7 @@
#include "stats.h"
namespace mongo {
-
+
OpCounters opsNonSharded;
OpCounters opsSharded;
diff --git a/s/stats.h b/s/stats.h
index cbabf25..a7cc784 100644
--- a/s/stats.h
+++ b/s/stats.h
@@ -22,7 +22,7 @@
#include "../db/stats/counters.h"
namespace mongo {
-
+
extern OpCounters opsNonSharded;
extern OpCounters opsSharded;
diff --git a/s/strategy.cpp b/s/strategy.cpp
index b3c8f5b..7c1fb0b 100644
--- a/s/strategy.cpp
+++ b/s/strategy.cpp
@@ -1,3 +1,5 @@
+// @file strategy.cpp
+
/*
* Copyright (C) 2010 10gen Inc.
*
@@ -14,312 +16,64 @@
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
-// stragegy.cpp
-
#include "pch.h"
-#include "request.h"
-#include "../util/background.h"
+
#include "../client/connpool.h"
#include "../db/commands.h"
-#include "server.h"
#include "grid.h"
+#include "request.h"
+#include "server.h"
+#include "writeback_listener.h"
+
+#include "strategy.h"
namespace mongo {
// ----- Strategy ------
- void Strategy::doWrite( int op , Request& r , const Shard& shard , bool checkVersion ){
+ void Strategy::doWrite( int op , Request& r , const Shard& shard , bool checkVersion ) {
ShardConnection conn( shard , r.getns() );
if ( ! checkVersion )
conn.donotCheckVersion();
- else if ( conn.setVersion() ){
+ else if ( conn.setVersion() ) {
conn.done();
throw StaleConfigException( r.getns() , "doWRite" , true );
}
conn->say( r.m() );
conn.done();
}
-
- void Strategy::doQuery( Request& r , const Shard& shard ){
- try{
- ShardConnection dbcon( shard , r.getns() );
- DBClientBase &c = dbcon.conn();
-
- Message response;
- bool ok = c.call( r.m(), response);
- {
- QueryResult *qr = (QueryResult *) response.singleData();
- if ( qr->resultFlags() & ResultFlag_ShardConfigStale ){
- dbcon.done();
- throw StaleConfigException( r.getns() , "Strategy::doQuery" );
- }
- }
+ void Strategy::doQuery( Request& r , const Shard& shard ) {
- uassert( 10200 , "mongos: error calling db", ok);
- r.reply( response , c.getServerAddress() );
- dbcon.done();
- }
- catch ( AssertionException& e ) {
- BSONObjBuilder err;
- e.getInfo().append( err );
- BSONObj errObj = err.done();
- replyToQuery(ResultFlag_ErrSet, r.p() , r.m() , errObj);
- }
- }
-
- void Strategy::insert( const Shard& shard , const char * ns , const BSONObj& obj ){
- ShardConnection dbcon( shard , ns );
- if ( dbcon.setVersion() ){
- dbcon.done();
- throw StaleConfigException( ns , "for insert" );
- }
- dbcon->insert( ns , obj );
- dbcon.done();
- }
-
- class WriteBackListener : public BackgroundJob {
- protected:
- string name() { return "WriteBackListener"; }
- WriteBackListener( const string& addr ) : _addr( addr ){
- log() << "creating WriteBackListener for: " << addr << endl;
- }
-
- void run(){
- OID lastID;
- lastID.clear();
- int secsToSleep = 0;
- while ( Shard::isMember( _addr ) ){
-
- if ( lastID.isSet() ){
- scoped_lock lk( _seenWritebacksLock );
- _seenWritebacks.insert( lastID );
- lastID.clear();
- }
-
- try {
- ScopedDbConnection conn( _addr );
-
- BSONObj result;
-
- {
- BSONObjBuilder cmd;
- cmd.appendOID( "writebacklisten" , &serverID ); // Command will block for data
- if ( ! conn->runCommand( "admin" , cmd.obj() , result ) ){
- log() << "writebacklisten command failed! " << result << endl;
- conn.done();
- continue;
- }
-
- }
-
- log(1) << "writebacklisten result: " << result << endl;
-
- BSONObj data = result.getObjectField( "data" );
- if ( data.getBoolField( "writeBack" ) ){
- string ns = data["ns"].valuestrsafe();
- {
- BSONElement e = data["id"];
- if ( e.type() == jstOID )
- lastID = e.OID();
- }
- int len;
+ ShardConnection dbcon( shard , r.getns() );
+ DBClientBase &c = dbcon.conn();
- Message m( (void*)data["msg"].binData( len ) , false );
- massert( 10427 , "invalid writeback message" , m.header()->valid() );
+ string actualServer;
- DBConfigPtr db = grid.getDBConfig( ns );
- ShardChunkVersion needVersion( data["version"] );
-
- log(1) << "writeback id: " << lastID << " needVersion : " << needVersion.toString()
- << " mine : " << db->getChunkManager( ns )->getVersion().toString() << endl;// TODO change to log(3)
-
- if ( logLevel ) log(1) << debugString( m ) << endl;
+ Message response;
+ bool ok = c.call( r.m(), response, true , &actualServer );
+ uassert( 10200 , "mongos: error calling db", ok );
- if ( needVersion.isSet() && needVersion <= db->getChunkManager( ns )->getVersion() ){
- // this means when the write went originally, the version was old
- // if we're here, it means we've already updated the config, so don't need to do again
- //db->getChunkManager( ns , true ); // SERVER-1349
- }
- else {
- db->getChunkManager( ns , true );
- }
-
- Request r( m , 0 );
- r.init();
- r.process();
- }
- else {
- log() << "unknown writeBack result: " << result << endl;
- }
-
- conn.done();
- secsToSleep = 0;
- continue;
- }
- catch ( std::exception e ){
- log() << "WriteBackListener exception : " << e.what() << endl;
-
- // It's possible this shard was removed
- Shard::reloadShardInfo();
- }
- catch ( ... ){
- log() << "WriteBackListener uncaught exception!" << endl;
- }
- secsToSleep++;
- sleepsecs(secsToSleep);
- if ( secsToSleep > 10 )
- secsToSleep = 0;
+ {
+ QueryResult *qr = (QueryResult *) response.singleData();
+ if ( qr->resultFlags() & ResultFlag_ShardConfigStale ) {
+ dbcon.done();
+ throw StaleConfigException( r.getns() , "Strategy::doQuery" );
}
-
- log() << "WriteBackListener exiting : address no longer in cluster " << _addr;
-
}
-
- private:
- string _addr;
- static map<string,WriteBackListener*> _cache;
- static mongo::mutex _cacheLock;
-
- static set<OID> _seenWritebacks;
- static mongo::mutex _seenWritebacksLock;
-
- public:
- static void init( DBClientBase& conn ){
- scoped_lock lk( _cacheLock );
- WriteBackListener*& l = _cache[conn.getServerAddress()];
- if ( l )
- return;
- l = new WriteBackListener( conn.getServerAddress() );
- l->go();
- }
-
-
- static void waitFor( const OID& oid ){
- Timer t;
- for ( int i=0; i<5000; i++ ){
- {
- scoped_lock lk( _seenWritebacksLock );
- if ( _seenWritebacks.count( oid ) )
- return;
- }
- sleepmillis( 10 );
- }
- stringstream ss;
- ss << "didn't get writeback for: " << oid << " after: " << t.millis() << " ms";
- uasserted( 13403 , ss.str() );
- }
- };
-
- void waitForWriteback( const OID& oid ){
- WriteBackListener::waitFor( oid );
- }
-
- map<string,WriteBackListener*> WriteBackListener::_cache;
- mongo::mutex WriteBackListener::_cacheLock("WriteBackListener");
-
- set<OID> WriteBackListener::_seenWritebacks;
- mongo::mutex WriteBackListener::_seenWritebacksLock( "WriteBackListener::seen" );
-
- struct ConnectionShardStatus {
-
- typedef unsigned long long S;
-
- ConnectionShardStatus()
- : _mutex( "ConnectionShardStatus" ){
- }
-
- S getSequence( DBClientBase * conn , const string& ns ){
- scoped_lock lk( _mutex );
- return _map[conn][ns];
- }
-
- void setSequence( DBClientBase * conn , const string& ns , const S& s ){
- scoped_lock lk( _mutex );
- _map[conn][ns] = s;
- }
-
- void reset( DBClientBase * conn ){
- scoped_lock lk( _mutex );
- _map.erase( conn );
- }
-
- map<DBClientBase*, map<string,unsigned long long> > _map;
- mongo::mutex _mutex;
- } connectionShardStatus;
-
- void resetShardVersion( DBClientBase * conn ){
- connectionShardStatus.reset( conn );
+ r.reply( response , actualServer.size() ? actualServer : c.getServerAddress() );
+ dbcon.done();
}
-
- /**
- * @return true if had to do something
- */
- bool checkShardVersion( DBClientBase& conn , const string& ns , bool authoritative , int tryNumber ){
- // TODO: cache, optimize, etc...
-
- WriteBackListener::init( conn );
- DBConfigPtr conf = grid.getDBConfig( ns );
- if ( ! conf )
- return false;
-
- unsigned long long officialSequenceNumber = 0;
-
- ChunkManagerPtr manager;
- const bool isSharded = conf->isSharded( ns );
- if ( isSharded ){
- manager = conf->getChunkManager( ns , authoritative );
- officialSequenceNumber = manager->getSequenceNumber();
- }
-
- unsigned long long sequenceNumber = connectionShardStatus.getSequence(&conn,ns);
- if ( sequenceNumber == officialSequenceNumber ){
- return false;
- }
-
-
- ShardChunkVersion version = 0;
- if ( isSharded ){
- version = manager->getVersion( Shard::make( conn.getServerAddress() ) );
- }
-
- log(2) << " have to set shard version for conn: " << &conn << " ns:" << ns
- << " my last seq: " << sequenceNumber << " current: " << officialSequenceNumber
- << " version: " << version << " manager: " << manager.get()
- << endl;
-
- BSONObj result;
- if ( setShardVersion( conn , ns , version , authoritative , result ) ){
- // success!
- log(1) << " setShardVersion success!" << endl;
- connectionShardStatus.setSequence( &conn , ns , officialSequenceNumber );
- return true;
- }
-
- log(1) << " setShardVersion failed!\n" << result << endl;
-
- if ( result.getBoolField( "need_authoritative" ) )
- massert( 10428 , "need_authoritative set but in authoritative mode already" , ! authoritative );
-
- if ( ! authoritative ){
- checkShardVersion( conn , ns , 1 , tryNumber + 1 );
- return true;
- }
-
- if ( tryNumber < 4 ){
- log(1) << "going to retry checkShardVersion" << endl;
- sleepmillis( 10 );
- checkShardVersion( conn , ns , 1 , tryNumber + 1 );
- return true;
+ void Strategy::insert( const Shard& shard , const char * ns , const BSONObj& obj ) {
+ ShardConnection dbcon( shard , ns );
+ if ( dbcon.setVersion() ) {
+ dbcon.done();
+ throw StaleConfigException( ns , "for insert" );
}
-
- log() << " setShardVersion failed: " << result << endl;
- massert( 10429 , (string)"setShardVersion failed! " + result.jsonString() , 0 );
- return true;
+ dbcon->insert( ns , obj );
+ dbcon.done();
}
-
-
}
diff --git a/s/strategy.h b/s/strategy.h
index 2aa4434..10a5a3f 100644
--- a/s/strategy.h
+++ b/s/strategy.h
@@ -23,28 +23,25 @@
#include "request.h"
namespace mongo {
-
+
class Strategy {
public:
- Strategy(){}
+ Strategy() {}
virtual ~Strategy() {}
virtual void queryOp( Request& r ) = 0;
virtual void getMore( Request& r ) = 0;
virtual void writeOp( int op , Request& r ) = 0;
-
+
protected:
void doWrite( int op , Request& r , const Shard& shard , bool checkVersion = true );
void doQuery( Request& r , const Shard& shard );
-
+
void insert( const Shard& shard , const char * ns , const BSONObj& obj );
-
+
};
extern Strategy * SINGLE;
extern Strategy * SHARDED;
- bool setShardVersion( DBClientBase & conn , const string& ns , ShardChunkVersion version , bool authoritative , BSONObj& result );
-
- void waitForWriteback( const OID& oid );
}
diff --git a/s/strategy_shard.cpp b/s/strategy_shard.cpp
index 144bf79..2eca0c6 100644
--- a/s/strategy_shard.cpp
+++ b/s/strategy_shard.cpp
@@ -21,6 +21,7 @@
#include "chunk.h"
#include "cursors.h"
#include "stats.h"
+#include "client.h"
#include "../client/connpool.h"
#include "../db/commands.h"
@@ -28,45 +29,45 @@
// error codes 8010-8040
namespace mongo {
-
+
class ShardStrategy : public Strategy {
- virtual void queryOp( Request& r ){
+ virtual void queryOp( Request& r ) {
QueryMessage q( r.d() );
log(3) << "shard query: " << q.ns << " " << q.query << endl;
-
+
if ( q.ntoreturn == 1 && strstr(q.ns, ".$cmd") )
throw UserException( 8010 , "something is wrong, shouldn't see a command here" );
ChunkManagerPtr info = r.getChunkManager();
assert( info );
-
+
Query query( q.query );
set<Shard> shards;
info->getShardsForQuery( shards , query.getFilter() );
-
+
set<ServerAndQuery> servers;
- for ( set<Shard>::iterator i = shards.begin(); i != shards.end(); i++ ){
- servers.insert( ServerAndQuery( i->getConnString() , BSONObj() ) );
+ for ( set<Shard>::iterator i = shards.begin(); i != shards.end(); i++ ) {
+ servers.insert( ServerAndQuery( i->getConnString() , BSONObj() ) );
}
-
- if ( logLevel > 4 ){
+
+ if ( logLevel > 4 ) {
StringBuilder ss;
ss << " shard query servers: " << servers.size() << '\n';
- for ( set<ServerAndQuery>::iterator i = servers.begin(); i!=servers.end(); i++ ){
+ for ( set<ServerAndQuery>::iterator i = servers.begin(); i!=servers.end(); i++ ) {
const ServerAndQuery& s = *i;
ss << " " << s.toString() << '\n';
}
- log() << ss.str();
+ log() << ss.str() << endl;
}
ClusteredCursor * cursor = 0;
-
+
BSONObj sort = query.getSort();
-
- if ( sort.isEmpty() ){
+
+ if ( sort.isEmpty() ) {
cursor = new SerialServerClusteredCursor( servers , q );
}
else {
@@ -80,85 +81,90 @@ namespace mongo {
log(5) << " cursor type: " << cursor->type() << endl;
shardedCursorTypes.hit( cursor->type() );
-
- if ( query.isExplain() ){
+
+ if ( query.isExplain() ) {
BSONObj explain = cursor->explain();
replyToQuery( 0 , r.p() , r.m() , explain );
delete( cursor );
return;
}
- } catch(...) {
+ }
+ catch(...) {
delete cursor;
throw;
}
ShardedClientCursorPtr cc (new ShardedClientCursor( q , cursor ));
- if ( ! cc->sendNextBatch( r ) ){
+ if ( ! cc->sendNextBatch( r ) ) {
return;
}
log(6) << "storing cursor : " << cc->getId() << endl;
cursorCache.store( cc );
}
-
- virtual void getMore( Request& r ){
+
+ virtual void getMore( Request& r ) {
int ntoreturn = r.d().pullInt();
long long id = r.d().pullInt64();
log(6) << "want cursor : " << id << endl;
ShardedClientCursorPtr cursor = cursorCache.get( id );
- if ( ! cursor ){
+ if ( ! cursor ) {
log(6) << "\t invalid cursor :(" << endl;
replyToQuery( ResultFlag_CursorNotFound , r.p() , r.m() , 0 , 0 , 0 );
return;
}
-
- if ( cursor->sendNextBatch( r , ntoreturn ) ){
+
+ if ( cursor->sendNextBatch( r , ntoreturn ) ) {
// still more data
cursor->accessed();
return;
}
-
+
// we've exhausted the cursor
cursorCache.remove( id );
}
-
- void _insert( Request& r , DbMessage& d, ChunkManagerPtr manager ){
-
- while ( d.moreJSObjs() ){
+
+ void _insert( Request& r , DbMessage& d, ChunkManagerPtr manager ) {
+
+ while ( d.moreJSObjs() ) {
BSONObj o = d.nextJsObj();
- if ( ! manager->hasShardKey( o ) ){
+ if ( ! manager->hasShardKey( o ) ) {
bool bad = true;
- if ( manager->getShardKey().partOfShardKey( "_id" ) ){
+ if ( manager->getShardKey().partOfShardKey( "_id" ) ) {
BSONObjBuilder b;
b.appendOID( "_id" , 0 , true );
b.appendElements( o );
o = b.obj();
bad = ! manager->hasShardKey( o );
}
-
- if ( bad ){
+
+ if ( bad ) {
log() << "tried to insert object without shard key: " << r.getns() << " " << o << endl;
throw UserException( 8011 , "tried to insert object without shard key" );
}
-
+
}
-
+
+ // Many operations benefit from having the shard key early in the object
+ o = manager->getShardKey().moveToFront(o);
+
bool gotThrough = false;
- for ( int i=0; i<10; i++ ){
+ for ( int i=0; i<10; i++ ) {
try {
ChunkPtr c = manager->findChunk( o );
log(4) << " server:" << c->getShard().toString() << " " << o << endl;
insert( c->getShard() , r.getns() , o );
-
+
r.gotInsert();
- c->splitIfShould( o.objsize() );
+ if ( r.getClientInfo()->autoSplitOk() )
+ c->splitIfShould( o.objsize() );
gotThrough = true;
break;
}
- catch ( StaleConfigException& ){
+ catch ( StaleConfigException& ) {
log(1) << "retrying insert because of StaleConfigException: " << o << endl;
r.reset();
manager = r.getChunkManager();
@@ -168,34 +174,38 @@ namespace mongo {
assert( gotThrough );
- }
+ }
}
- void _update( Request& r , DbMessage& d, ChunkManagerPtr manager ){
+ void _update( Request& r , DbMessage& d, ChunkManagerPtr manager ) {
int flags = d.pullInt();
-
+
BSONObj query = d.nextJsObj();
+ uassert( 13506 , "$atomic not supported sharded" , query["$atomic"].eoo() );
uassert( 10201 , "invalid update" , d.moreJSObjs() );
BSONObj toupdate = d.nextJsObj();
BSONObj chunkFinder = query;
-
+
bool upsert = flags & UpdateOption_Upsert;
bool multi = flags & UpdateOption_Multi;
- uassert( 10202 , "can't mix multi and upsert and sharding" , ! ( upsert && multi ) );
+ if (upsert) {
+ uassert(8012, "can't upsert something without shard key",
+ (manager->hasShardKey(toupdate) ||
+ (toupdate.firstElement().fieldName()[0] == '$' && manager->hasShardKey(query))));
- if ( upsert && !(manager->hasShardKey(toupdate) ||
- (toupdate.firstElement().fieldName()[0] == '$' && manager->hasShardKey(query))))
- {
- throw UserException( 8012 , "can't upsert something without shard key" );
+ BSONObj key = manager->getShardKey().extractKey(query);
+ BSONForEach(e, key) {
+ uassert(13465, "shard key in upsert query must be an exact match", getGtLtOp(e) == BSONObj::Equality);
+ }
}
bool save = false;
- if ( ! manager->hasShardKey( query ) ){
- if ( multi ){
+ if ( ! manager->hasShardKey( query ) ) {
+ if ( multi ) {
}
- else if ( strcmp( query.firstElement().fieldName() , "_id" ) || query.nFields() != 1 ){
+ else if ( strcmp( query.firstElement().fieldName() , "_id" ) || query.nFields() != 1 ) {
throw UserException( 8013 , "can't do non-multi update with query that doesn't have the shard key" );
}
else {
@@ -204,50 +214,59 @@ namespace mongo {
}
}
-
- if ( ! save ){
- if ( toupdate.firstElement().fieldName()[0] == '$' ){
+
+ if ( ! save ) {
+ if ( toupdate.firstElement().fieldName()[0] == '$' ) {
BSONObjIterator ops(toupdate);
- while(ops.more()){
+ while(ops.more()) {
BSONElement op(ops.next());
if (op.type() != Object)
continue;
BSONObjIterator fields(op.embeddedObject());
- while(fields.more()){
+ while(fields.more()) {
const string field = fields.next().fieldName();
- uassert(13123, "Can't modify shard key's value", ! manager->getShardKey().partOfShardKey(field));
+ uassert(13123,
+ str::stream() << "Can't modify shard key's value field" << field
+ << " for collection: " << manager->getns(),
+ ! manager->getShardKey().partOfShardKey(field));
}
}
- } else if ( manager->hasShardKey( toupdate ) ){
- uassert( 8014, "change would move shards!", manager->getShardKey().compare( query , toupdate ) == 0 );
- } else {
- uasserted(12376, "shard key must be in update object");
+ }
+ else if ( manager->hasShardKey( toupdate ) ) {
+ uassert( 8014,
+ str::stream() << "cannot modify shard key for collection: " << manager->getns(),
+ manager->getShardKey().compare( query , toupdate ) == 0 );
+ }
+ else {
+ uasserted(12376,
+ str::stream() << "shard key must be in update object for collection: " << manager->getns() );
}
}
-
- if ( multi ){
+
+ if ( multi ) {
set<Shard> shards;
manager->getShardsForQuery( shards , chunkFinder );
int * x = (int*)(r.d().afterNS());
x[0] |= UpdateOption_Broadcast;
- for ( set<Shard>::iterator i=shards.begin(); i!=shards.end(); i++){
+ for ( set<Shard>::iterator i=shards.begin(); i!=shards.end(); i++) {
doWrite( dbUpdate , r , *i , false );
}
}
else {
int left = 5;
- while ( true ){
+ while ( true ) {
try {
ChunkPtr c = manager->findChunk( chunkFinder );
doWrite( dbUpdate , r , c->getShard() );
- c->splitIfShould( d.msg().header()->dataLen() );
+ if ( r.getClientInfo()->autoSplitOk() )
+ c->splitIfShould( d.msg().header()->dataLen() );
break;
}
- catch ( StaleConfigException& e ){
+ catch ( StaleConfigException& e ) {
if ( left <= 0 )
throw e;
left--;
- log() << "update failed b/c of StaleConfigException, retrying "
+ log() << "update failed b/c of StaleConfigException, retrying "
<< " left:" << left << " ns: " << r.getns() << " query: " << query << endl;
r.reset( false );
manager = r.getChunkManager();
@@ -256,74 +275,75 @@ namespace mongo {
}
}
-
- void _delete( Request& r , DbMessage& d, ChunkManagerPtr manager ){
+
+ void _delete( Request& r , DbMessage& d, ChunkManagerPtr manager ) {
int flags = d.pullInt();
bool justOne = flags & 1;
-
+
uassert( 10203 , "bad delete message" , d.moreJSObjs() );
BSONObj pattern = d.nextJsObj();
+ uassert( 13505 , "$atomic not supported sharded" , pattern["$atomic"].eoo() );
set<Shard> shards;
int left = 5;
-
- while ( true ){
+
+ while ( true ) {
try {
manager->getShardsForQuery( shards , pattern );
log(2) << "delete : " << pattern << " \t " << shards.size() << " justOne: " << justOne << endl;
- if ( shards.size() == 1 ){
+ if ( shards.size() == 1 ) {
doWrite( dbDelete , r , *shards.begin() );
return;
}
break;
}
- catch ( StaleConfigException& e ){
+ catch ( StaleConfigException& e ) {
if ( left <= 0 )
throw e;
left--;
- log() << "delete failed b/c of StaleConfigException, retrying "
+ log() << "delete failed b/c of StaleConfigException, retrying "
<< " left:" << left << " ns: " << r.getns() << " patt: " << pattern << endl;
r.reset( false );
shards.clear();
manager = r.getChunkManager();
}
}
-
+
if ( justOne && ! pattern.hasField( "_id" ) )
throw UserException( 8015 , "can only delete with a non-shard key pattern if can delete as many as we find" );
-
- for ( set<Shard>::iterator i=shards.begin(); i!=shards.end(); i++){
+
+ for ( set<Shard>::iterator i=shards.begin(); i!=shards.end(); i++) {
int * x = (int*)(r.d().afterNS());
x[0] |= RemoveOption_Broadcast;
doWrite( dbDelete , r , *i , false );
}
}
-
- virtual void writeOp( int op , Request& r ){
+
+ virtual void writeOp( int op , Request& r ) {
const char *ns = r.getns();
log(3) << "write: " << ns << endl;
-
+
DbMessage& d = r.d();
ChunkManagerPtr info = r.getChunkManager();
assert( info );
-
- if ( op == dbInsert ){
+
+ if ( op == dbInsert ) {
_insert( r , d , info );
}
- else if ( op == dbUpdate ){
- _update( r , d , info );
+ else if ( op == dbUpdate ) {
+ _update( r , d , info );
}
- else if ( op == dbDelete ){
+ else if ( op == dbDelete ) {
_delete( r , d , info );
}
else {
log() << "sharding can't do write op: " << op << endl;
throw UserException( 8016 , "can't do this write op on sharded collection" );
}
-
+
}
};
-
+
Strategy * SHARDED = new ShardStrategy();
}
diff --git a/s/strategy_single.cpp b/s/strategy_single.cpp
index b840c9b..b3b5502 100644
--- a/s/strategy_single.cpp
+++ b/s/strategy_single.cpp
@@ -18,117 +18,102 @@
#include "pch.h"
#include "request.h"
+#include "cursors.h"
#include "../client/connpool.h"
#include "../db/commands.h"
namespace mongo {
class SingleStrategy : public Strategy {
-
+
public:
- SingleStrategy(){
+ SingleStrategy() {
_commandsSafeToPass.insert( "$eval" );
_commandsSafeToPass.insert( "create" );
}
private:
- virtual void queryOp( Request& r ){
+ virtual void queryOp( Request& r ) {
QueryMessage q( r.d() );
-
- bool lateAssert = false;
-
+
log(3) << "single query: " << q.ns << " " << q.query << " ntoreturn: " << q.ntoreturn << endl;
-
- try {
- if ( r.isCommand() ){
-
- if ( handleSpecialNamespaces( r , q ) )
- return;
-
- int loops = 5;
- while ( true ){
- BSONObjBuilder builder;
- try {
- bool ok = Command::runAgainstRegistered(q.ns, q.query, builder);
- if ( ok ) {
- BSONObj x = builder.done();
- replyToQuery(0, r.p(), r.m(), x);
- return;
- }
- break;
- }
- catch ( StaleConfigException& e ){
- if ( loops <= 0 )
- throw e;
-
- loops--;
- log() << "retrying command: " << q.query << endl;
- ShardConnection::checkMyConnectionVersions( e.getns() );
- }
- catch ( AssertionException& e ){
- e.getInfo().append( builder , "assertion" , "assertionCode" );
- builder.append( "errmsg" , "db assertion failure" );
- builder.append( "ok" , 0 );
+
+ if ( r.isCommand() ) {
+
+ if ( handleSpecialNamespaces( r , q ) )
+ return;
+
+ int loops = 5;
+ while ( true ) {
+ BSONObjBuilder builder;
+ try {
+ bool ok = Command::runAgainstRegistered(q.ns, q.query, builder);
+ if ( ok ) {
BSONObj x = builder.done();
replyToQuery(0, r.p(), r.m(), x);
return;
}
+ break;
}
-
- string commandName = q.query.firstElement().fieldName();
+ catch ( StaleConfigException& e ) {
+ if ( loops <= 0 )
+ throw e;
- uassert(13390, "unrecognized command: " + commandName, _commandsSafeToPass.count(commandName) != 0);
- }
-
- lateAssert = true;
- doQuery( r , r.primaryShard() );
- }
- catch ( AssertionException& e ) {
- if ( lateAssert ){
- log() << "lateAssert: " << e.getInfo() << endl;
- assert( !lateAssert );
+ loops--;
+ log() << "retrying command: " << q.query << endl;
+ ShardConnection::checkMyConnectionVersions( e.getns() );
+ }
+ catch ( AssertionException& e ) {
+ e.getInfo().append( builder , "assertion" , "assertionCode" );
+ builder.append( "errmsg" , "db assertion failure" );
+ builder.append( "ok" , 0 );
+ BSONObj x = builder.done();
+ replyToQuery(0, r.p(), r.m(), x);
+ return;
+ }
}
- BSONObjBuilder err;
- e.getInfo().append( err );
- BSONObj errObj = err.done();
- replyToQuery(ResultFlag_ErrSet, r.p() , r.m() , errObj);
- return;
+ string commandName = q.query.firstElement().fieldName();
+
+ uassert(13390, "unrecognized command: " + commandName, _commandsSafeToPass.count(commandName) != 0);
}
+ doQuery( r , r.primaryShard() );
}
-
- virtual void getMore( Request& r ){
+
+ virtual void getMore( Request& r ) {
const char *ns = r.getns();
-
- log(3) << "single getmore: " << ns << endl;
- ShardConnection conn( r.primaryShard() , ns );
+ LOG(3) << "single getmore: " << ns << endl;
+
+ long long id = r.d().getInt64( 4 );
+
+ ShardConnection conn( cursorCache.getRef( id ) , ns );
Message response;
bool ok = conn->callRead( r.m() , response);
uassert( 10204 , "dbgrid: getmore: error calling db", ok);
- r.reply( response , conn->getServerAddress() );
-
+ r.reply( response , "" /*conn->getServerAddress() */ );
+
conn.done();
}
-
- void handleIndexWrite( int op , Request& r ){
-
+
+ void handleIndexWrite( int op , Request& r ) {
+
DbMessage& d = r.d();
- if ( op == dbInsert ){
- while( d.moreJSObjs() ){
+ if ( op == dbInsert ) {
+ while( d.moreJSObjs() ) {
BSONObj o = d.nextJsObj();
const char * ns = o["ns"].valuestr();
- if ( r.getConfig()->isSharded( ns ) ){
+ if ( r.getConfig()->isSharded( ns ) ) {
BSONObj newIndexKey = o["key"].embeddedObjectUserCheck();
-
- uassert( 10205 , (string)"can't use unique indexes with sharding ns:" + ns +
- " key: " + o["key"].embeddedObjectUserCheck().toString() ,
+
+ uassert( 10205 , (string)"can't use unique indexes with sharding ns:" + ns +
+ " key: " + o["key"].embeddedObjectUserCheck().toString() ,
IndexDetails::isIdIndexPattern( newIndexKey ) ||
- ! o["unique"].trueValue() ||
+ ! o["unique"].trueValue() ||
r.getConfig()->getChunkManager( ns )->getShardKey().isPrefixOf( newIndexKey ) );
ChunkManagerPtr cm = r.getConfig()->getChunkManager( ns );
@@ -145,10 +130,10 @@ namespace mongo {
r.gotInsert();
}
}
- else if ( op == dbUpdate ){
+ else if ( op == dbUpdate ) {
throw UserException( 8050 , "can't update system.indexes" );
}
- else if ( op == dbDelete ){
+ else if ( op == dbDelete ) {
// TODO
throw UserException( 8051 , "can't delete indexes on sharded collection yet" );
}
@@ -156,26 +141,26 @@ namespace mongo {
log() << "handleIndexWrite invalid write op: " << op << endl;
throw UserException( 8052 , "handleIndexWrite invalid write op" );
}
-
+
}
- virtual void writeOp( int op , Request& r ){
+ virtual void writeOp( int op , Request& r ) {
const char *ns = r.getns();
-
- if ( r.isShardingEnabled() &&
- strstr( ns , ".system.indexes" ) == strchr( ns , '.' ) &&
- strchr( ns , '.' ) ) {
+
+ if ( r.isShardingEnabled() &&
+ strstr( ns , ".system.indexes" ) == strchr( ns , '.' ) &&
+ strchr( ns , '.' ) ) {
log(1) << " .system.indexes write for: " << ns << endl;
handleIndexWrite( op , r );
return;
}
-
+
log(3) << "single write: " << ns << endl;
doWrite( op , r , r.primaryShard() );
r.gotInsert(); // Won't handle mulit-insert correctly. Not worth parsing the request.
}
- bool handleSpecialNamespaces( Request& r , QueryMessage& q ){
+ bool handleSpecialNamespaces( Request& r , QueryMessage& q ) {
const char * ns = r.getns();
ns = strstr( r.getns() , ".$cmd.sys." );
if ( ! ns )
@@ -184,29 +169,32 @@ namespace mongo {
BSONObjBuilder b;
vector<Shard> shards;
-
- if ( strcmp( ns , "inprog" ) == 0 ){
+
+ if ( strcmp( ns , "inprog" ) == 0 ) {
Shard::getAllShards( shards );
-
+
BSONArrayBuilder arr( b.subarrayStart( "inprog" ) );
- for ( unsigned i=0; i<shards.size(); i++ ){
+ for ( unsigned i=0; i<shards.size(); i++ ) {
Shard shard = shards[i];
ScopedDbConnection conn( shard );
BSONObj temp = conn->findOne( r.getns() , BSONObj() );
- if ( temp["inprog"].isABSONObj() ){
+ if ( temp["inprog"].isABSONObj() ) {
BSONObjIterator i( temp["inprog"].Obj() );
- while ( i.more() ){
+ while ( i.more() ) {
BSONObjBuilder x;
-
+
BSONObjIterator j( i.next().Obj() );
- while( j.more() ){
+ while( j.more() ) {
BSONElement e = j.next();
- if ( strcmp( e.fieldName() , "opid" ) == 0 ){
+ if ( str::equals( e.fieldName() , "opid" ) ) {
stringstream ss;
ss << shard.getName() << ':' << e.numberInt();
x.append( "opid" , ss.str() );
}
+ else if ( str::equals( e.fieldName() , "client" ) ) {
+ x.appendAs( e , "client_s" );
+ }
else {
x.append( e );
}
@@ -216,15 +204,15 @@ namespace mongo {
}
conn.done();
}
-
+
arr.done();
}
- else if ( strcmp( ns , "killop" ) == 0 ){
+ else if ( strcmp( ns , "killop" ) == 0 ) {
BSONElement e = q.query["op"];
- if ( strstr( r.getns() , "admin." ) != 0 ){
+ if ( strstr( r.getns() , "admin." ) != 0 ) {
b.append( "err" , "unauthorized" );
}
- else if ( e.type() != String ){
+ else if ( e.type() != String ) {
b.append( "err" , "bad op" );
b.append( e );
}
@@ -232,7 +220,7 @@ namespace mongo {
b.append( e );
string s = e.String();
string::size_type i = s.find( ':' );
- if ( i == string::npos ){
+ if ( i == string::npos ) {
b.append( "err" , "bad opid" );
}
else {
@@ -243,14 +231,14 @@ namespace mongo {
log() << "want to kill op: " << e << endl;
Shard s(shard);
-
+
ScopedDbConnection conn( s );
conn->findOne( r.getns() , BSON( "op" << opid ) );
conn.done();
}
}
}
- else if ( strcmp( ns , "unlock" ) == 0 ){
+ else if ( strcmp( ns , "unlock" ) == 0 ) {
b.append( "err" , "can't do unlock through mongos" );
}
else {
@@ -265,6 +253,6 @@ namespace mongo {
set<string> _commandsSafeToPass;
};
-
+
Strategy * SINGLE = new SingleStrategy();
}
diff --git a/s/util.h b/s/util.h
index 7695eda..b3f63d8 100644
--- a/s/util.h
+++ b/s/util.h
@@ -36,29 +36,30 @@ namespace mongo {
};
unsigned long long _combined;
};
-
+
ShardChunkVersion( int major=0, int minor=0 )
- : _minor(minor),_major(major){
+ : _minor(minor),_major(major) {
}
-
+
ShardChunkVersion( unsigned long long ll )
- : _combined( ll ){
+ : _combined( ll ) {
}
-
- ShardChunkVersion( const BSONElement& e ){
- if ( e.type() == Date || e.type() == Timestamp ){
+
+ ShardChunkVersion( const BSONElement& e ) {
+ if ( e.type() == Date || e.type() == Timestamp ) {
_combined = e._numberLong();
}
- else if ( e.eoo() ){
+ else if ( e.eoo() ) {
_combined = 0;
}
else {
+ _combined = 0;
log() << "ShardChunkVersion can't handle type (" << (int)(e.type()) << ") " << e << endl;
assert(0);
}
}
- void inc( bool major ){
+ void inc( bool major ) {
if ( major )
incMajor();
else
@@ -69,7 +70,7 @@ namespace mongo {
_major++;
_minor = 0;
}
-
+
void incMinor() {
_minor++;
}
@@ -82,19 +83,19 @@ namespace mongo {
return _combined > 0;
}
- string toString() const {
- stringstream ss;
- ss << _major << "|" << _minor;
- return ss.str();
+ string toString() const {
+ stringstream ss;
+ ss << _major << "|" << _minor;
+ return ss.str();
}
int majorVersion() const { return _major; }
int minorVersion() const { return _minor; }
-
+
operator unsigned long long() const { return _combined; }
-
- ShardChunkVersion& operator=( const BSONElement& elem ){
- switch ( elem.type() ){
+
+ ShardChunkVersion& operator=( const BSONElement& elem ) {
+ switch ( elem.type() ) {
case Timestamp:
case NumberLong:
case Date:
@@ -109,39 +110,39 @@ namespace mongo {
return *this;
}
};
-
- inline ostream& operator<<( ostream &s , const ShardChunkVersion& v){
+
+ inline ostream& operator<<( ostream &s , const ShardChunkVersion& v) {
s << v._major << "|" << v._minor;
return s;
}
- /**
- * your config info for a given shard/chunk is out of date
+ /**
+ * your config info for a given shard/chunk is out of date
*/
class StaleConfigException : public AssertionException {
public:
StaleConfigException( const string& ns , const string& raw , bool justConnection = false )
- : AssertionException( (string)"ns: " + ns + " " + raw , 9996 ) ,
+ : AssertionException( (string)"ns: " + ns + " " + raw , 9996 ) ,
_justConnection(justConnection) ,
- _ns(ns){
+ _ns(ns) {
}
-
- virtual ~StaleConfigException() throw(){}
-
+
+ virtual ~StaleConfigException() throw() {}
+
virtual void appendPrefix( stringstream& ss ) const { ss << "StaleConfigException: "; }
-
+
bool justConnection() const { return _justConnection; }
-
+
string getns() const { return _ns; }
- static bool parse( const string& big , string& ns , string& raw ){
+ static bool parse( const string& big , string& ns , string& raw ) {
string::size_type start = big.find( '[' );
if ( start == string::npos )
return false;
string::size_type end = big.find( ']' ,start );
if ( end == string::npos )
return false;
-
+
ns = big.substr( start + 1 , ( end - start ) - 1 );
raw = big.substr( end + 1 );
return true;
@@ -151,6 +152,7 @@ namespace mongo {
string _ns;
};
- bool checkShardVersion( DBClientBase & conn , const string& ns , bool authoritative = false , int tryNumber = 1 );
- void resetShardVersion( DBClientBase * conn );
+ extern boost::function4<bool, DBClientBase&, const string&, bool, int> checkShardVersionCB;
+ extern boost::function1<void, DBClientBase*> resetShardVersionCB;
+
}
diff --git a/s/writeback_listener.cpp b/s/writeback_listener.cpp
new file mode 100644
index 0000000..21d59d0
--- /dev/null
+++ b/s/writeback_listener.cpp
@@ -0,0 +1,254 @@
+// @file writeback_listener.cpp
+
+/**
+* Copyright (C) 2010 10gen Inc.
+*
+* This program is free software: you can redistribute it and/or modify
+* it under the terms of the GNU Affero General Public License, version 3,
+* as published by the Free Software Foundation.
+*
+* This program is distributed in the hope that it will be useful,
+* but WITHOUT ANY WARRANTY; without even the implied warranty of
+* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+* GNU Affero General Public License for more details.
+*
+* You should have received a copy of the GNU Affero General Public License
+* along with this program. If not, see <http://www.gnu.org/licenses/>.
+*/
+
+#include "pch.h"
+
+#include "../util/timer.h"
+
+#include "config.h"
+#include "grid.h"
+#include "request.h"
+#include "server.h"
+#include "shard.h"
+#include "util.h"
+#include "client.h"
+
+#include "writeback_listener.h"
+
+namespace mongo {
+
+ map<string,WriteBackListener*> WriteBackListener::_cache;
+ set<string> WriteBackListener::_seenSets;
+ mongo::mutex WriteBackListener::_cacheLock("WriteBackListener");
+
+ map<ConnectionId,WriteBackListener::WBStatus> WriteBackListener::_seenWritebacks;
+ mongo::mutex WriteBackListener::_seenWritebacksLock("WriteBackListener::seen");
+
+ WriteBackListener::WriteBackListener( const string& addr ) : _addr( addr ) {
+ log() << "creating WriteBackListener for: " << addr << endl;
+ }
+
+ /* static */
+ void WriteBackListener::init( DBClientBase& conn ) {
+
+ if ( conn.type() == ConnectionString::SYNC ) {
+ // don't want write back listeners for config servers
+ return;
+ }
+
+ if ( conn.type() != ConnectionString::SET ) {
+ init( conn.getServerAddress() );
+ return;
+ }
+
+
+ {
+ scoped_lock lk( _cacheLock );
+ if ( _seenSets.count( conn.getServerAddress() ) )
+ return;
+ }
+
+ // we want to do writebacks on all rs nodes
+ string errmsg;
+ ConnectionString cs = ConnectionString::parse( conn.getServerAddress() , errmsg );
+ uassert( 13641 , str::stream() << "can't parse host [" << conn.getServerAddress() << "]" , cs.isValid() );
+
+ vector<HostAndPort> hosts = cs.getServers();
+
+ for ( unsigned i=0; i<hosts.size(); i++ )
+ init( hosts[i].toString() );
+
+ }
+
+ /* static */
+ void WriteBackListener::init( const string& host ) {
+ scoped_lock lk( _cacheLock );
+ WriteBackListener*& l = _cache[host];
+ if ( l )
+ return;
+ l = new WriteBackListener( host );
+ l->go();
+ }
+
+ /* static */
+ BSONObj WriteBackListener::waitFor( ConnectionId connectionId, const OID& oid ) {
+ Timer t;
+ for ( int i=0; i<5000; i++ ) {
+ {
+ scoped_lock lk( _seenWritebacksLock );
+ WBStatus s = _seenWritebacks[connectionId];
+ if ( oid < s.id ) {
+ // this means we're waiting for a GLE that already passed.
+ // it should be impossible becauseonce we call GLE, no other
+ // writebacks should happen with that connection id
+ msgasserted( 13633 , str::stream() << "got writeback waitfor for older id " <<
+ " oid: " << oid << " s.id: " << s.id << " connectionId: " << connectionId );
+ }
+ else if ( oid == s.id ) {
+ return s.gle;
+ }
+
+ }
+ sleepmillis( 10 );
+ }
+ uasserted( 13403 , str::stream() << "didn't get writeback for: " << oid << " after: " << t.millis() << " ms" );
+ throw 1; // never gets here
+ }
+
+ void WriteBackListener::run() {
+ int secsToSleep = 0;
+ while ( ! inShutdown() ) {
+
+ if ( ! Shard::isAShardNode( _addr ) ) {
+ log(1) << _addr << " is not a shard node" << endl;
+ sleepsecs( 60 );
+ continue;
+ }
+
+ try {
+ ScopedDbConnection conn( _addr );
+
+ BSONObj result;
+
+ {
+ BSONObjBuilder cmd;
+ cmd.appendOID( "writebacklisten" , &serverID ); // Command will block for data
+ if ( ! conn->runCommand( "admin" , cmd.obj() , result ) ) {
+ log() << "writebacklisten command failed! " << result << endl;
+ conn.done();
+ continue;
+ }
+
+ }
+
+ log(1) << "writebacklisten result: " << result << endl;
+
+ BSONObj data = result.getObjectField( "data" );
+ if ( data.getBoolField( "writeBack" ) ) {
+ string ns = data["ns"].valuestrsafe();
+
+ ConnectionId cid = 0;
+ OID wid;
+ if ( data["connectionId"].isNumber() && data["id"].type() == jstOID ) {
+ cid = data["connectionId"].numberLong();
+ wid = data["id"].OID();
+ }
+ else {
+ warning() << "mongos/mongod version mismatch (1.7.5 is the split)" << endl;
+ }
+
+ int len; // not used, but needed for next call
+ Message m( (void*)data["msg"].binData( len ) , false );
+ massert( 10427 , "invalid writeback message" , m.header()->valid() );
+
+ DBConfigPtr db = grid.getDBConfig( ns );
+ ShardChunkVersion needVersion( data["version"] );
+
+ log(1) << "connectionId: " << cid << " writebackId: " << wid << " needVersion : " << needVersion.toString()
+ << " mine : " << db->getChunkManager( ns )->getVersion().toString() << endl;// TODO change to log(3)
+
+ if ( logLevel ) log(1) << debugString( m ) << endl;
+
+ if ( needVersion.isSet() && needVersion <= db->getChunkManager( ns )->getVersion() ) {
+ // this means when the write went originally, the version was old
+ // if we're here, it means we've already updated the config, so don't need to do again
+ //db->getChunkManager( ns , true ); // SERVER-1349
+ }
+ else {
+ // we received a writeback object that was sent to a previous version of a shard
+ // the actual shard may not have the object the writeback operation is for
+ // we need to reload the chunk manager and get the new shard versions
+ db->getChunkManager( ns , true );
+ }
+
+ // do request and then call getLastError
+ // we have to call getLastError so we can return the right fields to the user if they decide to call getLastError
+
+ BSONObj gle;
+ try {
+
+ Request r( m , 0 );
+ r.init();
+
+ ClientInfo * ci = r.getClientInfo();
+ ci->noAutoSplit();
+
+ r.process();
+
+ ci->newRequest(); // this so we flip prev and cur shards
+
+ BSONObjBuilder b;
+ if ( ! ci->getLastError( BSON( "getLastError" << 1 ) , b , true ) ) {
+ b.appendBool( "commandFailed" , true );
+ }
+ gle = b.obj();
+
+ ci->clearSinceLastGetError();
+ }
+ catch ( DBException& e ) {
+ error() << "error processing writeback: " << e << endl;
+ BSONObjBuilder b;
+ b.append( "err" , e.toString() );
+ e.getInfo().append( b );
+ gle = b.obj();
+ }
+
+ {
+ scoped_lock lk( _seenWritebacksLock );
+ WBStatus& s = _seenWritebacks[cid];
+ s.id = wid;
+ s.gle = gle;
+ }
+ }
+ else if ( result["noop"].trueValue() ) {
+ // no-op
+ }
+ else {
+ log() << "unknown writeBack result: " << result << endl;
+ }
+
+ conn.done();
+ secsToSleep = 0;
+ continue;
+ }
+ catch ( std::exception e ) {
+
+ if ( inShutdown() ) {
+ // we're shutting down, so just clean up
+ return;
+ }
+
+ log() << "WriteBackListener exception : " << e.what() << endl;
+
+ // It's possible this shard was removed
+ Shard::reloadShardInfo();
+ }
+ catch ( ... ) {
+ log() << "WriteBackListener uncaught exception!" << endl;
+ }
+ secsToSleep++;
+ sleepsecs(secsToSleep);
+ if ( secsToSleep > 10 )
+ secsToSleep = 0;
+ }
+
+ log() << "WriteBackListener exiting : address no longer in cluster " << _addr;
+
+ }
+
+} // namespace mongo
diff --git a/s/writeback_listener.h b/s/writeback_listener.h
new file mode 100644
index 0000000..7335999
--- /dev/null
+++ b/s/writeback_listener.h
@@ -0,0 +1,67 @@
+// @file writeback_listener.h
+
+/**
+* Copyright (C) 2010 10gen Inc.
+*
+* This program is free software: you can redistribute it and/or modify
+* it under the terms of the GNU Affero General Public License, version 3,
+* as published by the Free Software Foundation.
+*
+* This program is distributed in the hope that it will be useful,
+* but WITHOUT ANY WARRANTY; without even the implied warranty of
+* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
+* GNU Affero General Public License for more details.
+*
+* You should have received a copy of the GNU Affero General Public License
+* along with this program. If not, see <http://www.gnu.org/licenses/>.
+*/
+
+#pragma once
+
+#include "../pch.h"
+
+#include "../client/connpool.h"
+#include "../util/background.h"
+#include "../db/client.h"
+
+namespace mongo {
+
+ /*
+ * The writeback listener takes back write attempts that were made against a wrong shard.
+ * (Wrong here in the sense that the target chunk moved before this mongos had a chance to
+ * learn so.) It is responsible for reapplying these writes to the correct shard.
+ *
+ * Currently, there is one listener per shard.
+ */
+ class WriteBackListener : public BackgroundJob {
+ public:
+ static void init( DBClientBase& conn );
+ static void init( const string& host );
+
+ static BSONObj waitFor( ConnectionId connectionId, const OID& oid );
+
+ protected:
+ WriteBackListener( const string& addr );
+
+ string name() const { return "WriteBackListener"; }
+ void run();
+
+ private:
+ string _addr;
+
+ static mongo::mutex _cacheLock; // protects _cache
+ static map<string,WriteBackListener*> _cache; // server to listener
+ static set<string> _seenSets; // cache of set urls we've seen - note this is ever expanding for order, case, changes
+
+ struct WBStatus {
+ OID id;
+ BSONObj gle;
+ };
+
+ static mongo::mutex _seenWritebacksLock; // protects _seenWritbacks
+ static map<ConnectionId,WBStatus> _seenWritebacks; // connectionId -> last write back GLE
+ };
+
+ void waitForWriteback( const OID& oid );
+
+} // namespace mongo