From cb68885328acc28a0fc04a8a711312d6320b19f4 Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Thu, 8 Mar 2018 11:27:15 -0800 Subject: [PATCH 01/12] If backup expiration determines that force is required but the force parameter is not set, it will no longer throw an error unless the backup contains data from prior to the expire_before_version. --- fdbclient/BackupContainer.actor.cpp | 46 ++++++++++++++++++++++------- fdbclient/Knobs.cpp | 1 + fdbclient/Knobs.h | 1 + 3 files changed, 38 insertions(+), 10 deletions(-) diff --git a/fdbclient/BackupContainer.actor.cpp b/fdbclient/BackupContainer.actor.cpp index 8829072680..e708715519 100644 --- a/fdbclient/BackupContainer.actor.cpp +++ b/fdbclient/BackupContainer.actor.cpp @@ -564,7 +564,7 @@ public: // Force is required if there is not a restorable snapshot which both // - begins at or after expireEndVersion // - ends at or before restorableBeginVersion - bool forceNeeded = true; + state bool forceNeeded = true; for(KeyspaceSnapshotFile &s : desc.snapshots) { if(s.restorable.orDefault(false) && s.beginVersion >= expireEndVersion && s.endVersion <= restorableBeginVersion) { forceNeeded = false; @@ -572,9 +572,6 @@ public: } } - if(forceNeeded && !force) - throw backup_cannot_expire(); - // Get metadata state Optional expiredEnd; state Optional logBegin; @@ -631,25 +628,54 @@ public: Void _ = wait(bc->logBeginVersion().clear()); } - // Delete files - state std::vector> deletes; + // Make a list of files to delete + state std::vector toDelete; + // Move filenames out of vector then destroy it to save memory for(auto const &f : logs) { - deletes.push_back(bc->deleteFile(f.fileName)); + toDelete.push_back(std::move(f.fileName)); } + logs.clear(); + // Move filenames out of vector then destroy it to save memory for(auto const &f : ranges) { // Must recheck version because list returns data up to and including the given endVersion if(f.version < expireEndVersion) - deletes.push_back(bc->deleteFile(f.fileName)); + toDelete.push_back(std::move(f.fileName)); } + ranges.clear(); for(auto const &f : desc.snapshots) { if(f.endVersion < expireEndVersion) - deletes.push_back(bc->deleteFile(f.fileName)); + toDelete.push_back(std::move(f.fileName)); } - Void _ = wait(waitForAll(deletes)); + // If some files to delete were found AND force is needed AND the force option is NOT set, then fail + if(!toDelete.empty() && forceNeeded && !force) + throw backup_cannot_expire(); + + // Delete files, but limit parallelism because the file list could use a lot of memory and the corresponding + // delete actor states would use even more if they all existed at the same time. + state std::list> deleteFutures; + + while(!toDelete.empty() || !deleteFutures.empty()) { + + // While there are files to delete and budget in the deleteFutures list, start a delete + while(!toDelete.empty() && deleteFutures.size() < CLIENT_KNOBS->BACKUP_CONCURRENT_DELETES) { + deleteFutures.push_back(bc->deleteFile(toDelete.back())); + toDelete.pop_back(); + } + + // Wait for deletes to finish until there are only targetDeletesInFlight remaining. + // If there are no files left to start then this value is 0, otherwise it is one less + // than the delete concurrency limit. + state int targetFuturesSize = toDelete.empty() ? 0 : (CLIENT_KNOBS->BACKUP_CONCURRENT_DELETES - 1); + + while(deleteFutures.size() > targetFuturesSize) { + Void _ = wait(deleteFutures.front()); + deleteFutures.pop_front(); + } + } // Update the expiredEndVersion property. Void _ = wait(bc->expiredEndVersion().set(expireEndVersion)); diff --git a/fdbclient/Knobs.cpp b/fdbclient/Knobs.cpp index 6c6232089f..e22f4550e2 100644 --- a/fdbclient/Knobs.cpp +++ b/fdbclient/Knobs.cpp @@ -91,6 +91,7 @@ ClientKnobs::ClientKnobs(bool randomize) { init( TASKBUCKET_MAX_TASK_KEYS, 1000 ); if( randomize && BUGGIFY ) TASKBUCKET_MAX_TASK_KEYS = 20; //Backup + init( BACKUP_CONCURRENT_DELETES, 100 ); init( BACKUP_SIMULATED_LIMIT_BYTES, 1e6 ); if( randomize && BUGGIFY ) BACKUP_SIMULATED_LIMIT_BYTES = 1000; init( BACKUP_GET_RANGE_LIMIT_BYTES, 1e6 ); init( BACKUP_LOCK_BYTES, 1e8 ); diff --git a/fdbclient/Knobs.h b/fdbclient/Knobs.h index d24c6717c6..1ce0cec1ea 100644 --- a/fdbclient/Knobs.h +++ b/fdbclient/Knobs.h @@ -93,6 +93,7 @@ public: int TASKBUCKET_MAX_TASK_KEYS; // Backup + int BACKUP_CONCURRENT_DELETES; int BACKUP_SIMULATED_LIMIT_BYTES; int BACKUP_GET_RANGE_LIMIT_BYTES; int BACKUP_LOCK_BYTES; From e7309a353514e15a98ad82fb5a2023cf43a1b8bd Mon Sep 17 00:00:00 2001 From: Balachandar Namasivayam Date: Thu, 8 Mar 2018 13:53:59 -0800 Subject: [PATCH 02/12] Add trace events to print the ranges in ConsistencyCheck. --- fdbserver/workloads/ConsistencyCheck.actor.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/fdbserver/workloads/ConsistencyCheck.actor.cpp b/fdbserver/workloads/ConsistencyCheck.actor.cpp index f6a778ef32..ef86574165 100644 --- a/fdbserver/workloads/ConsistencyCheck.actor.cpp +++ b/fdbserver/workloads/ConsistencyCheck.actor.cpp @@ -682,6 +682,7 @@ struct ConsistencyCheckWorkload : TestWorkload state Key lastSampleKey; state Key lastStartSampleKey; state int64_t totalReadAmount = 0; + state int64_t rangeBytes = 0; state KeySelector begin = firstGreaterOrEqual(range.begin); @@ -725,6 +726,9 @@ struct ConsistencyCheckWorkload : TestWorkload { state GetKeyValuesReply current = rangeResult.get(); totalReadAmount += current.data.expectedSize(); + if (j == 0) { + rangeBytes += current.data.expectedSize(); + } //If we haven't encountered a valid storage server yet, then mark this as the baseline to compare against if(firstValidServer == -1) firstValidServer = j; @@ -911,6 +915,8 @@ struct ConsistencyCheckWorkload : TestWorkload } } + TraceEvent("ConsistencyCheck_CheckedRange").detail("Range", printable(range)).detail("Bytes", rangeBytes); + canSplit = canSplit && sampledBytes - splitBytes >= shardBounds.min.bytes && sampledBytes > splitBytes; //Update the size of all storage servers containing this shard From 4e9091eb75e68cbb31e10d1a722f94e8b9077d0f Mon Sep 17 00:00:00 2001 From: Balachandar Namasivayam Date: Thu, 8 Mar 2018 14:19:14 -0800 Subject: [PATCH 03/12] Add new guid for wxs --- packaging/msi/FDBInstaller.wxs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/packaging/msi/FDBInstaller.wxs b/packaging/msi/FDBInstaller.wxs index e49398eb30..0ea1497102 100644 --- a/packaging/msi/FDBInstaller.wxs +++ b/packaging/msi/FDBInstaller.wxs @@ -32,7 +32,7 @@ Date: Thu, 8 Mar 2018 18:08:41 -0800 Subject: [PATCH 04/12] fix: new cluster controllers should not consider anything failed until they have time to get failure monitoring updates fix: storage and log class machines wait 100MS before attempting to become the cluster controller --- fdbserver/ClusterController.actor.cpp | 2 +- fdbserver/LeaderElection.actor.cpp | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/fdbserver/ClusterController.actor.cpp b/fdbserver/ClusterController.actor.cpp index fc8ad74b55..79894aef0c 100644 --- a/fdbserver/ClusterController.actor.cpp +++ b/fdbserver/ClusterController.actor.cpp @@ -152,7 +152,7 @@ public: }; bool workerAvailable( WorkerInfo const& worker, bool checkStable ) { - return IFailureMonitor::failureMonitor().getState(worker.interf.storage.getEndpoint()).isAvailable() && ( !checkStable || worker.reboots < 2 ); + return ( now() - startTime < 2 * FLOW_KNOBS->SERVER_REQUEST_INTERVAL ) || ( IFailureMonitor::failureMonitor().getState(worker.interf.storage.getEndpoint()).isAvailable() && ( !checkStable || worker.reboots < 2 ) ); } std::pair getStorageWorker( RecruitStorageRequest const& req ) { diff --git a/fdbserver/LeaderElection.actor.cpp b/fdbserver/LeaderElection.actor.cpp index 0bccf08dc2..8cb90f7d00 100644 --- a/fdbserver/LeaderElection.actor.cpp +++ b/fdbserver/LeaderElection.actor.cpp @@ -82,6 +82,10 @@ ACTOR Future tryBecomeLeaderInternal( ServerCoordinators coordinators, Val state bool iAmLeader = false; state UID prevChangeID; + if( asyncProcessClass->get().machineClassFitness(ProcessClass::ClusterController) > ProcessClass::UnsetFit || asyncIsExcluded->get() ) { + Void _ = wait( delay(SERVER_KNOBS->WAIT_FOR_GOOD_RECRUITMENT_DELAY) ); + } + nominees->set( vector>( coordinators.clientLeaderServers.size() ) ); myInfo.serializedInfo = proposedSerializedInterface; From 5390af8be41c82b26a8ea78d0a72cb7ba22ba8d5 Mon Sep 17 00:00:00 2001 From: Evan Tschannen Date: Fri, 16 Feb 2018 16:01:19 -0800 Subject: [PATCH 05/12] suppress spammy logs --- fdbclient/FileBackupAgent.actor.cpp | 2 +- fdbrpc/FailureMonitor.actor.cpp | 2 +- fdbrpc/FlowTransport.actor.cpp | 16 ++++++++-------- fdbserver/DataDistribution.actor.cpp | 4 ++-- fdbserver/DataDistributionQueue.actor.cpp | 16 ++++++++++++++-- fdbserver/storageserver.actor.cpp | 6 ++---- flow/Net2.actor.cpp | 6 +++--- flow/Trace.cpp | 2 +- flow/Trace.h | 3 ++- 9 files changed, 34 insertions(+), 23 deletions(-) diff --git a/fdbclient/FileBackupAgent.actor.cpp b/fdbclient/FileBackupAgent.actor.cpp index 142a41825b..c2f559c510 100644 --- a/fdbclient/FileBackupAgent.actor.cpp +++ b/fdbclient/FileBackupAgent.actor.cpp @@ -1497,7 +1497,7 @@ namespace fileBackup { .detail("ScheduledVersion", scheduledVersion) .detail("BeginKey", range.begin.printable()) .detail("EndKey", range.end.printable()) - .suppressFor(2, true); + .suppressFor(2); } else { // This shouldn't happen because if the transaction was already done or if another execution diff --git a/fdbrpc/FailureMonitor.actor.cpp b/fdbrpc/FailureMonitor.actor.cpp index 0874eebaf2..d2520bccf0 100644 --- a/fdbrpc/FailureMonitor.actor.cpp +++ b/fdbrpc/FailureMonitor.actor.cpp @@ -92,7 +92,7 @@ void SimpleFailureMonitor::setStatus( NetworkAddress const& address, FailureStat void SimpleFailureMonitor::endpointNotFound( Endpoint const& endpoint ) { // SOMEDAY: Expiration (this "leaks" memory) - TraceEvent("EndpointNotFound").detail("Address", endpoint.address).detail("Token", endpoint.token); + TraceEvent("EndpointNotFound").detail("Address", endpoint.address).detail("Token", endpoint.token).suppressFor(1.0); endpointKnownFailed.set( endpoint, true ); } diff --git a/fdbrpc/FlowTransport.actor.cpp b/fdbrpc/FlowTransport.actor.cpp index ae1275d0a1..edeaef5f8f 100644 --- a/fdbrpc/FlowTransport.actor.cpp +++ b/fdbrpc/FlowTransport.actor.cpp @@ -369,14 +369,14 @@ struct Peer : NonCopyable { Void _ = wait( delayJittered( std::max(0.0, self->lastConnectTime+self->reconnectionDelay - now()) ) ); // Don't connect() to the same peer more than once per 2 sec self->lastConnectTime = now(); - TraceEvent("ConnectingTo", conn ? conn->getDebugID() : UID()).detail("PeerAddr", self->destination); + TraceEvent("ConnectingTo", conn ? conn->getDebugID() : UID()).detail("PeerAddr", self->destination).suppressFor(1.0); Reference _conn = wait( timeout( INetworkConnections::net()->connect(self->destination), FLOW_KNOBS->CONNECTION_MONITOR_TIMEOUT, Reference() ) ); if (_conn) { conn = _conn; - TraceEvent("ConnectionExchangingConnectPacket", conn->getDebugID()).detail("PeerAddr", self->destination); + TraceEvent("ConnectionExchangingConnectPacket", conn->getDebugID()).detail("PeerAddr", self->destination).suppressFor(1.0); self->prependConnectPacket(); } else { - TraceEvent("ConnectionTimedOut", conn ? conn->getDebugID() : UID()).detail("PeerAddr", self->destination); + TraceEvent("ConnectionTimedOut", conn ? conn->getDebugID() : UID()).detail("PeerAddr", self->destination).suppressFor(1.0); throw connection_failed(); } @@ -408,7 +408,7 @@ struct Peer : NonCopyable { bool ok = e.code() == error_code_connection_failed || e.code() == error_code_actor_cancelled || ( g_network->isSimulated() && e.code() == error_code_checksum_failed ); if(self->compatible) { - TraceEvent(ok ? SevInfo : SevError, "ConnectionClosed", conn ? conn->getDebugID() : UID()).detail("PeerAddr", self->destination).error(e, true); + TraceEvent(ok ? SevInfo : SevWarnAlways, "ConnectionClosed", conn ? conn->getDebugID() : UID()).detail("PeerAddr", self->destination).error(e, true).suppressFor(1.0); } else { TraceEvent(ok ? SevInfo : SevError, "IncompatibleConnectionClosed", conn ? conn->getDebugID() : UID()).detail("PeerAddr", self->destination).error(e, true); @@ -637,7 +637,7 @@ ACTOR static Future connectionReader( compatible = true; TraceEvent("ConnectionEstablished", conn->getDebugID()) .detail("Peer", conn->getPeerAddress()) - .detail("ConnectionId", connectionId); + .detail("ConnectionId", connectionId).suppressFor(1.0); } if(connectionId > 1) { @@ -649,7 +649,7 @@ ACTOR static Future connectionReader( peerProtocolVersion = p->protocolVersion; if (peer != nullptr) { // Outgoing connection; port information should be what we expect - TraceEvent("ConnectedOutgoing").detail("PeerAddr", NetworkAddress( p->canonicalRemoteIp, p->canonicalRemotePort ) ); + TraceEvent("ConnectedOutgoing").detail("PeerAddr", NetworkAddress( p->canonicalRemoteIp, p->canonicalRemotePort ) ).suppressFor(1.0); peer->compatible = compatible; if (!compatible) peer->transport->numIncompatibleConnections++; @@ -710,7 +710,7 @@ ACTOR static Future connectionIncoming( TransportData* self, ReferencegetDebugID()).error(e).detail("FromAddress", conn->getPeerAddress()); + TraceEvent("IncomingConnectionError", conn->getDebugID()).error(e).detail("FromAddress", conn->getPeerAddress()).suppressFor(1.0); conn->close(); return Void(); } @@ -722,7 +722,7 @@ ACTOR static Future listen( TransportData* self, NetworkAddress listenAddr try { loop { Reference conn = wait( listener->accept() ); - TraceEvent("ConnectionFrom", conn->getDebugID()).detail("FromAddress", conn->getPeerAddress()); + TraceEvent("ConnectionFrom", conn->getDebugID()).detail("FromAddress", conn->getPeerAddress()).suppressFor(1.0); incoming.add( connectionIncoming(self, conn) ); } } catch (Error& e) { diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index b0875eea7f..03124d9a1b 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -1708,7 +1708,7 @@ ACTOR Future storageRecruiter( DDTeamCollection *self, Referenceserver_info.begin(); s != self->server_info.end(); ++s) { auto serverStatus = self->server_status.get( s->second->lastKnownInterface.id() ); if( serverStatus.excludeOnRecruit() ) { - TraceEvent("DDRecruitExcl1").detail("Excluding", s->second->lastKnownInterface.address()); + TraceEvent(SevDebug, "DDRecruitExcl1").detail("Excluding", s->second->lastKnownInterface.address()); auto addr = s->second->lastKnownInterface.address(); exclusions.insert( AddressExclusion( addr.ip, addr.port ) ); } @@ -1720,7 +1720,7 @@ ACTOR Future storageRecruiter( DDTeamCollection *self, ReferenceexcludedServers.getKeys(); for(auto& s : excl) if (self->excludedServers.get(s)) { - TraceEvent("DDRecruitExcl2").detail("Excluding", s.toString()); + TraceEvent(SevDebug, "DDRecruitExcl2").detail("Excluding", s.toString()); exclusions.insert( s ); } rsr.criticalRecruitment = self->healthyTeamCount == 0; diff --git a/fdbserver/DataDistributionQueue.actor.cpp b/fdbserver/DataDistributionQueue.actor.cpp index be13eb20e0..8a13b52fd7 100644 --- a/fdbserver/DataDistributionQueue.actor.cpp +++ b/fdbserver/DataDistributionQueue.actor.cpp @@ -193,6 +193,8 @@ struct DDQueueData { PromiseStream getShardMetrics; double* lastLimited; + double lastInterval; + int suppressIntervals; DDQueueData( MasterInterface mi, MoveKeysLock lock, Database cx, TeamCollectionInterface teamCollection, Reference sABTF, PromiseStream> getAverageShardBytes, @@ -202,7 +204,7 @@ struct DDQueueData { shardsAffectedByTeamFailure( sABTF ), getAverageShardBytes( getAverageShardBytes ), mi( mi ), lock( lock ), cx( cx ), teamSize( teamSize ), durableStorageQuorum( durableStorageQuorum ), input( input ), getShardMetrics( getShardMetrics ), startMoveKeysParallelismLock( SERVER_KNOBS->DD_MOVE_KEYS_PARALLELISM ), - finishMoveKeysParallelismLock( SERVER_KNOBS->DD_MOVE_KEYS_PARALLELISM ), lastLimited(lastLimited) {} + finishMoveKeysParallelismLock( SERVER_KNOBS->DD_MOVE_KEYS_PARALLELISM ), lastLimited(lastLimited), suppressIntervals(0), lastInterval(0) {} void validate() { if( EXPENSIVE_VALIDATION ) { @@ -652,9 +654,19 @@ ACTOR Future dataDistributionRelocator( DDQueueData *self, RelocateData rd state Reference destination; try { + if(now() - self->lastInterval < 1.0) { + relocateShardInterval.severity = SevDebug; + self->suppressIntervals++; + } + TraceEvent(relocateShardInterval.begin(), masterId) .detail("KeyBegin", printable(rd.keys.begin)).detail("KeyEnd", printable(rd.keys.end)) - .detail("Priority", rd.priority).detail("RelocationID", relocateShardInterval.pairID); + .detail("Priority", rd.priority).detail("RelocationID", relocateShardInterval.pairID).detail("SuppressedEventCount", self->suppressIntervals); + + if(relocateShardInterval.severity != SevDebug) { + self->lastInterval = now(); + self->suppressIntervals = 0; + } state StorageMetrics metrics = wait( brokenPromiseToNever( self->getShardMetrics.getReply( GetMetricsRequest( rd.keys ) ) ) ); diff --git a/fdbserver/storageserver.actor.cpp b/fdbserver/storageserver.actor.cpp index 70e15a7766..30cfed948c 100644 --- a/fdbserver/storageserver.actor.cpp +++ b/fdbserver/storageserver.actor.cpp @@ -1840,7 +1840,7 @@ ACTOR Future fetchKeys( StorageServer *data, AddingShard* shard ) { break; } catch (Error& e) { - TraceEvent("FKBlockFail", data->thisServerID).detail("FKID", interval.pairID).error(e,true); + TraceEvent("FKBlockFail", data->thisServerID).detail("FKID", interval.pairID).error(e,true).suppressFor(1.0); if (e.code() == error_code_transaction_too_old){ TEST(true); // A storage server has forgotten the history data we are fetching Void _ = wait( delayJittered( FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY ) ); @@ -2290,9 +2290,7 @@ ACTOR Future update( StorageServer* data, bool* pReceivedUpdate ) if (now() - waitStartT >= .1) { TraceEvent(SevWarn, "StorageServerUpdateLag", data->thisServerID) .detail("Version", data->version.get()) - .detail("DurableVersion", data->durableVersion.get()) - //.detail("ExtraBytes", usedBytesOlderThanDesiredDurableVersion(data)) - ; + .detail("DurableVersion", data->durableVersion.get()).suppressFor(1.0); waitStartT = now(); } diff --git a/flow/Net2.actor.cpp b/flow/Net2.actor.cpp index d8c6aa8a91..602dfda5b8 100644 --- a/flow/Net2.actor.cpp +++ b/flow/Net2.actor.cpp @@ -412,15 +412,15 @@ private: boost::system::error_code error; socket.close(error); if (error) - TraceEvent(SevWarn, "N2_CloseError", id).detail("Message", error.value()); + TraceEvent(SevWarn, "N2_CloseError", id).detail("Message", error.value()).suppressFor(1.0); } void onReadError( const boost::system::error_code& error ) { - TraceEvent(SevWarn, "N2_ReadError", id).detail("Message", error.value()); + TraceEvent(SevWarn, "N2_ReadError", id).detail("Message", error.value()).suppressFor(1.0); closeSocket(); } void onWriteError( const boost::system::error_code& error ) { - TraceEvent(SevWarn, "N2_WriteError", id).detail("Message", error.value()); + TraceEvent(SevWarn, "N2_WriteError", id).detail("Message", error.value()).suppressFor(1.0); closeSocket(); } }; diff --git a/flow/Trace.cpp b/flow/Trace.cpp index 81c27ce843..520ef39347 100644 --- a/flow/Trace.cpp +++ b/flow/Trace.cpp @@ -650,7 +650,7 @@ TraceEvent::TraceEvent(Severity severity, const char* type, const Optional Date: Fri, 9 Mar 2018 10:16:34 -0800 Subject: [PATCH 06/12] suppress spammy trace events --- fdbrpc/AsyncFileEIO.actor.h | 4 ++-- fdbserver/DataDistribution.actor.cpp | 1 - fdbserver/DataDistributionQueue.actor.cpp | 2 +- fdbserver/workloads/ConsistencyCheck.actor.cpp | 12 ++++-------- 4 files changed, 7 insertions(+), 12 deletions(-) diff --git a/fdbrpc/AsyncFileEIO.actor.h b/fdbrpc/AsyncFileEIO.actor.h index a857df8aa3..3205cf9800 100644 --- a/fdbrpc/AsyncFileEIO.actor.h +++ b/fdbrpc/AsyncFileEIO.actor.h @@ -83,7 +83,7 @@ public: TraceEvent(notFound ? SevWarn : SevWarnAlways, "FileOpenError").error(e).GetLastError().detail("File", filename).detail("Flags", flags).detail("Mode", mode); throw e; } - TraceEvent("AsyncFileOpened").detail("Filename", filename).detail("fd", r->result).detail("Flags", flags); + TraceEvent("AsyncFileOpened").detail("Filename", filename).detail("fd", r->result).detail("Flags", flags).suppressFor(1.0); if ((flags & OPEN_LOCK) && !lock_fd(r->result)) { TraceEvent(SevError, "UnableToLockFile").detail("filename", filename).GetLastError(); @@ -264,7 +264,7 @@ private: state eio_req* r = eio_close(fd, 0, eio_callback, &p); Void _ = wait( p.getFuture() ); if (r->result) error( "CloseError", fd, r ); - TraceEvent("AsyncFileClosed").detail("fd", fd); + TraceEvent("AsyncFileClosed").detail("fd", fd).suppressFor(1.0); } ACTOR static Future read_impl( int fd, void* data, int length, int64_t offset ) { diff --git a/fdbserver/DataDistribution.actor.cpp b/fdbserver/DataDistribution.actor.cpp index 03124d9a1b..eedaac2fc5 100644 --- a/fdbserver/DataDistribution.actor.cpp +++ b/fdbserver/DataDistribution.actor.cpp @@ -611,7 +611,6 @@ struct DDTeamCollection { } if( foundExact || (req.wantsTrueBest && bestOption.present() ) ) { - TraceEvent("getTeam").detail("wantsVariety", req.wantsNewServers).detail("bestOption", bestOption.get()->getDesc()); ASSERT( bestOption.present() ); req.reply.send( bestOption ); return Void(); diff --git a/fdbserver/DataDistributionQueue.actor.cpp b/fdbserver/DataDistributionQueue.actor.cpp index 8a13b52fd7..db6210a8c1 100644 --- a/fdbserver/DataDistributionQueue.actor.cpp +++ b/fdbserver/DataDistributionQueue.actor.cpp @@ -702,7 +702,7 @@ ACTOR Future dataDistributionRelocator( DDQueueData *self, RelocateData rd destination->addDataInFlightToTeam( +metrics.bytes ); - TraceEvent("RelocateShardHasDestination", masterId) + TraceEvent(relocateShardInterval.severity, "RelocateShardHasDestination", masterId) .detail("PairId", relocateShardInterval.pairID) .detail("DestinationTeam", destination->getDesc()); diff --git a/fdbserver/workloads/ConsistencyCheck.actor.cpp b/fdbserver/workloads/ConsistencyCheck.actor.cpp index ef86574165..99b1af91d8 100644 --- a/fdbserver/workloads/ConsistencyCheck.actor.cpp +++ b/fdbserver/workloads/ConsistencyCheck.actor.cpp @@ -682,7 +682,6 @@ struct ConsistencyCheckWorkload : TestWorkload state Key lastSampleKey; state Key lastStartSampleKey; state int64_t totalReadAmount = 0; - state int64_t rangeBytes = 0; state KeySelector begin = firstGreaterOrEqual(range.begin); @@ -726,9 +725,6 @@ struct ConsistencyCheckWorkload : TestWorkload { state GetKeyValuesReply current = rangeResult.get(); totalReadAmount += current.data.expectedSize(); - if (j == 0) { - rangeBytes += current.data.expectedSize(); - } //If we haven't encountered a valid storage server yet, then mark this as the baseline to compare against if(firstValidServer == -1) firstValidServer = j; @@ -841,7 +837,7 @@ struct ConsistencyCheckWorkload : TestWorkload else if(!isRelocating) { TraceEvent("ConsistencyCheck_StorageServerUnavailable").detail("StorageServer", storageServers[j]).detail("ShardBegin", printable(range.begin)).detail("ShardEnd", printable(range.end)) - .detail("Address", storageServerInterfaces[j].address()).detail("GetKeyValuesToken", storageServerInterfaces[j].getKeyValues.getEndpoint().token); + .detail("Address", storageServerInterfaces[j].address()).detail("GetKeyValuesToken", storageServerInterfaces[j].getKeyValues.getEndpoint().token).suppressFor(1.0); //All shards should be available in quiscence if(self->performQuiescentChecks) @@ -915,8 +911,6 @@ struct ConsistencyCheckWorkload : TestWorkload } } - TraceEvent("ConsistencyCheck_CheckedRange").detail("Range", printable(range)).detail("Bytes", rangeBytes); - canSplit = canSplit && sampledBytes - splitBytes >= shardBounds.min.bytes && sampledBytes > splitBytes; //Update the size of all storage servers containing this shard @@ -983,7 +977,9 @@ struct ConsistencyCheckWorkload : TestWorkload } } - TraceEvent("ConsistencyCheck_ReadRange").detail("range", printable(range)).detail("bytesRead", bytesReadInRange); + if(bytesReadInRange > 0) { + TraceEvent("ConsistencyCheck_ReadRange").detail("range", printable(range)).detail("bytesRead", bytesReadInRange); + } } //SOMEDAY: when background data distribution is implemented, include this test From 3a7288924a10f47acc165373db81e2b0edbbecdc Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Fri, 9 Mar 2018 11:29:23 -0800 Subject: [PATCH 07/12] Bug fixes. During expiration, the backup container's log range metadata could be updated incorrectly if force was required and not specified or if a backup had no log begin metadata and an expire was done which covered 1 or more log file. In the latter case a backup could be left in a state where the container metadata suggests the backup has more log coverage than it actually does. --- fdbclient/BackupContainer.actor.cpp | 52 ++++++++++++++++++----------- 1 file changed, 32 insertions(+), 20 deletions(-) diff --git a/fdbclient/BackupContainer.actor.cpp b/fdbclient/BackupContainer.actor.cpp index e708715519..90e9dd41d3 100644 --- a/fdbclient/BackupContainer.actor.cpp +++ b/fdbclient/BackupContainer.actor.cpp @@ -601,33 +601,16 @@ public: newLogBeginVersion = expireEndVersion; } else { - // If the last log overlaps the expiredEnd then use the log's begin version + // If the last log overlaps the expiredEnd then use the log's begin version and move the expiredEnd + // back to match it. if(last.endVersion > expireEndVersion) { newLogBeginVersion = last.beginVersion; logs.pop_back(); + expireEndVersion = newLogBeginVersion.get(); } } } - // If we have a new log begin version then potentially update the property but definitely set - // expireEndVersion to the new log begin because we will only be deleting files up to but not - // including that version. - if(newLogBeginVersion.present()) { - expireEndVersion = newLogBeginVersion.get(); - // If the new version is greater than the existing one or the - // existing one is not present then write the new value - if(logBegin.orDefault(0) < newLogBeginVersion.get()) { - Void _ = wait(bc->logBeginVersion().set(newLogBeginVersion.get())); - } - } - else { - // Otherwise, if the old logBeginVersion is present and older than expireEndVersion then clear it because - // it refers to a version in a range we're about to delete and apparently continuity through - // expireEndVersion is broken. - if(logBegin.present() && logBegin.get() < expireEndVersion) - Void _ = wait(bc->logBeginVersion().clear()); - } - // Make a list of files to delete state std::vector toDelete; @@ -654,6 +637,35 @@ public: if(!toDelete.empty() && forceNeeded && !force) throw backup_cannot_expire(); + // We are about to start deleting files, at which point no data prior to the expire end version can be + // safely assumed to exist. The [logBegin, logEnd) range from the container's metadata describes + // a range of log versions which can be assumed to exist, so if the range of data being deleted overlaps + // that range then the metadata range must be updated. + + // If we're expiring the entire log range described by the metadata then clear both metadata values + if(logEnd.present() && logEnd.get() < expireEndVersion) { + if(logBegin.present()) + Void _ = wait(bc->logBeginVersion().clear()); + if(logEnd.present()) + Void _ = wait(bc->logEndVersion().clear()); + } + else { + // If we are expiring to a point within the metadata range then update the begin if we have a new + // log begin version (which we should!) or clear the metadata range if we do not (which would be + // repairing the metadata from an incorrect state) + if(logBegin.present() && logBegin.get() < expireEndVersion) { + if(newLogBeginVersion.present()) { + Void _ = wait(bc->logBeginVersion().set(newLogBeginVersion.get())); + } + else { + if(logBegin.present()) + Void _ = wait(bc->logBeginVersion().clear()); + if(logEnd.present()) + Void _ = wait(bc->logEndVersion().clear()); + } + } + } + // Delete files, but limit parallelism because the file list could use a lot of memory and the corresponding // delete actor states would use even more if they all existed at the same time. state std::list> deleteFutures; From bb9f51bb5ce3f908650f467edace4f13fc073ee5 Mon Sep 17 00:00:00 2001 From: "A.J. Beamon" Date: Fri, 9 Mar 2018 11:55:57 -0800 Subject: [PATCH 08/12] Don't try to extract attributes from the program start trace events if they couldn't be collected. --- fdbserver/Status.actor.cpp | 32 +++++++++++++++++--------------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/fdbserver/Status.actor.cpp b/fdbserver/Status.actor.cpp index fd1bf97df0..f218de272e 100644 --- a/fdbserver/Status.actor.cpp +++ b/fdbserver/Status.actor.cpp @@ -790,8 +790,21 @@ ACTOR static Future processStatusFetcher( if (programStarts.count(address)) { auto const& psxml = programStarts.at(address); - int64_t memLimit = parseInt64(extractAttribute(psxml, "MemoryLimit")); - memoryObj["limit_bytes"] = memLimit; + + if(psxml.size() > 0) { + int64_t memLimit = parseInt64(extractAttribute(psxml, "MemoryLimit")); + memoryObj["limit_bytes"] = memLimit; + + std::string version; + if (tryExtractAttribute(psxml, LiteralStringRef("Version"), version)) { + statusObj["version"] = version; + } + + std::string commandLine; + if (tryExtractAttribute(psxml, LiteralStringRef("CommandLine"), commandLine)) { + statusObj["command_line"] = commandLine; + } + } } // if this process address is in the machine metrics @@ -811,9 +824,10 @@ ACTOR static Future processStatusFetcher( StatusArray messages; - if (errors.count(address) && errors[address].size()) + if (errors.count(address) && errors[address].size()) { // returns status object with type and time of error messages.push_back(getError(errors.at(address))); + } // string of address used so that other fields of a NetworkAddress are not compared std::string strAddress = address.toString(); @@ -838,18 +852,6 @@ ACTOR static Future processStatusFetcher( // Get roles for the worker's address as an array of objects statusObj["roles"] = roles.getStatusForAddress(address); - if (programStarts.count(address)) { - auto const& psxml = programStarts.at(address); - - std::string version; - if (tryExtractAttribute(psxml, LiteralStringRef("Version"), version)) - statusObj["version"] = version; - - std::string commandLine; - if (tryExtractAttribute(psxml, LiteralStringRef("CommandLine"), commandLine)) - statusObj["command_line"] = commandLine; - } - if (configuration.present()){ statusObj["excluded"] = configuration.get().isExcludedServer(address); } From 9c901983f0b9c72ab0cda802a9a3e402c6113222 Mon Sep 17 00:00:00 2001 From: Stephen Atherton Date: Fri, 9 Mar 2018 12:03:10 -0800 Subject: [PATCH 09/12] Clarity improvement, resetting backup description variable because it's no longer valid due to some of its contents being std::move'd. --- fdbclient/BackupContainer.actor.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/fdbclient/BackupContainer.actor.cpp b/fdbclient/BackupContainer.actor.cpp index 90e9dd41d3..99c7da15a3 100644 --- a/fdbclient/BackupContainer.actor.cpp +++ b/fdbclient/BackupContainer.actor.cpp @@ -632,6 +632,7 @@ public: if(f.endVersion < expireEndVersion) toDelete.push_back(std::move(f.fileName)); } + desc = BackupDescription(); // If some files to delete were found AND force is needed AND the force option is NOT set, then fail if(!toDelete.empty() && forceNeeded && !force) From 66f1b615433d518df8dad855a30d109767c3b494 Mon Sep 17 00:00:00 2001 From: satherton Date: Fri, 9 Mar 2018 12:14:48 -0800 Subject: [PATCH 10/12] Update FDBInstaller.wxs New product GUID. --- packaging/msi/FDBInstaller.wxs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/packaging/msi/FDBInstaller.wxs b/packaging/msi/FDBInstaller.wxs index 0ea1497102..f1995446d2 100644 --- a/packaging/msi/FDBInstaller.wxs +++ b/packaging/msi/FDBInstaller.wxs @@ -32,7 +32,7 @@ Date: Fri, 9 Mar 2018 12:16:33 -0800 Subject: [PATCH 11/12] Updated to version 5.1.4 --- versions.target | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/versions.target b/versions.target index 10c96af7dd..1c81b16c71 100644 --- a/versions.target +++ b/versions.target @@ -1,7 +1,7 @@ - 5.1.3 + 5.1.4 5.1 From b2ef6e13584ba4c9566f7d2288b6d2c323dc3669 Mon Sep 17 00:00:00 2001 From: "A.J. Beamon" Date: Fri, 9 Mar 2018 14:17:20 -0800 Subject: [PATCH 12/12] Add missing available_bytes fields to test status schemas --- tests/fast/SidebandWithStatus.txt | 2 +- tests/rare/LargeApiCorrectnessStatus.txt | 2 +- tests/slow/DDBalanceAndRemoveStatus.txt | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/fast/SidebandWithStatus.txt b/tests/fast/SidebandWithStatus.txt index bc3d0115be..0b873659f1 100644 --- a/tests/fast/SidebandWithStatus.txt +++ b/tests/fast/SidebandWithStatus.txt @@ -5,7 +5,7 @@ testTitle=CloggedCausalConsistencyTest testName=Status testDuration=30.0 - schema={"cluster":{"layers":{"_valid":true,"_error":"some error description"},"processes":{"$map":{"fault_domain":"0ccb4e0fdbdb5583010f6b77d9d10ece","class_source":{"$enum":["command_line","configure_auto","set_class"]},"class_type":{"$enum":["unset","storage","transaction","resolution","proxy","master","test"]},"roles":[{"query_queue_max":0,"data_version_lag":12341234,"input_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"kvstore_used_bytes":12341234,"stored_bytes":12341234,"data_version":12341234,"kvstore_free_bytes":12341234,"durable_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"id":"eb84471d68c12d1d26f692a50000003f","persistent_disk_used_bytes":12341234,"role":{"$enum":["master","proxy","log","storage","resolver","cluster_controller"]},"persistent_disk_total_bytes":12341234,"queue_disk_total_bytes":12341234,"persistent_disk_free_bytes":12341234,"queue_disk_used_bytes":12341234,"queue_disk_free_bytes":12341234,"kvstore_total_bytes":12341234,"finished_queries":{"hz":0.0,"counter":0,"roughness":0.0}}],"locality":{"$map":"value"},"messages":[{"description":"abc","type":"x","name":{"$enum":["file_open_error","incorrect_cluster_file_contents","process_error","io_error","io_timeout","platform_error","storage_server_lagging","(other FDB error messages)"]},"raw_log_message":"","time":12345.12312}],"address":"1.2.3.4:1234","command_line":"-r simulation","disk":{"free_bytes":3451233456234,"reads":{"hz":0.0,"counter":0,"sectors":0},"busy":0.0,"writes":{"hz":0.0,"counter":0,"sectors":0},"total_bytes":123412341234},"version":"3.0.0","excluded":false,"memory":{"available_bytes":0,"used_bytes":0,"limit_bytes":0},"machine_id":"0ccb4e0feddb5583010f6b77d9d10ece","uptime_seconds":1234.2345,"cpu":{"usage_cores":0.0},"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"connections_closed":{"hz":0.0},"connection_errors":{"hz":0.0},"current_connections":0,"connections_established":{"hz":0.0}}}},"clients":{"count":1,"supported_versions":[{"count":1,"protocol_version":"fdb00a400050001","client_version":"3.0.0","source_version":"9430e1127b4991cbc5ab2b17f41cfffa5de07e9d","connected_clients":[{"log_group":"default","address":"127.0.0.1:9898"}]}]},"qos":{"limiting_version_lag_storage_server":0,"released_transactions_per_second":0,"transactions_per_second_limit":0,"limiting_queue_bytes_storage_server":0,"performance_limited_by":{"reason_server_id":"7f8d623d0cb9966e","description":"The database is not being saturated by the workload.","reason_id":0,"name":{"$enum":["workload","storage_server_write_queue_size","storage_server_write_bandwidth_mvcc","storage_server_readable_behind","log_server_mvcc_write_bandwidth","log_server_write_queue","storage_server_min_free_space","storage_server_min_free_space_ratio","log_server_min_free_space","log_server_min_free_space_ratio"]}},"worst_version_lag_storage_server":0,"worst_queue_bytes_log_server":460,"worst_queue_bytes_storage_server":0},"incompatible_connections":[],"database_locked":false,"generation":2,"data":{"least_operating_space_bytes_log_server":0,"average_partition_size_bytes":0,"state":{"healthy":true,"description":"","name":{"$enum":["initializing","missing_data","healing","healthy_repartitioning","healthy_removing_server","healthy_rebalancing","healthy"]},"min_replicas_remaining":0},"least_operating_space_ratio_storage_server":0.1,"max_machine_failures_without_losing_availability":0,"total_disk_used_bytes":0,"total_kv_size_bytes":0,"max_machine_failures_without_losing_data":0,"moving_data":{"in_queue_bytes":0,"total_written_bytes":0,"in_flight_bytes":0},"least_operating_space_bytes_storage_server":0,"partitions_count":2},"fault_tolerance":{"max_machine_failures_without_losing_availability":0,"max_machine_failures_without_losing_data":0},"messages":[{"reasons":[{"description":"Blah."}],"unreachable_processes":[{"address":"1.2.3.4:1234"}],"name":{"$enum":["unreachable_master_worker","unreadable_configuration","client_issues","unreachable_processes","immediate_priority_transaction_start_probe_timeout","batch_priority_transaction_start_probe_timeout","transaction_start_probe_timeout","read_probe_timeout","commit_probe_timeout","storage_servers_error","status_incomplete","layer_status_incomplete","database_availability_timeout"]},"issues":[{"name":{"$enum":["incorrect_cluster_file_contents"]},"description":"Cluster file contents do not match current cluster connection string. Verify cluster file is writable and has not been overwritten externally."}],"description":"abc"}],"database_available":true,"recovery_state":{"required_proxies":1,"name":{"$enum":["reading_coordinated_state","locking_coordinated_state","locking_old_transaction_servers","reading_transaction_system_state","configuration_missing","configuration_never_created","configuration_invalid","recruiting_transaction_servers","initializing_transaction_servers","recovery_transaction","writing_coordinated_state","fully_recovered"]},"missing_logs":"7f8d623d0cb9966e","required_resolvers":1,"required_logs":3,"description":"Recovery complete."},"workload":{"operations":{"writes":{"hz":0.0,"counter":0,"roughness":0.0},"reads":{"hz":0.0}},"bytes":{"written":{"hz":0.0,"counter":0,"roughness":0.0}},"transactions":{"started":{"hz":0.0,"counter":0,"roughness":0.0},"conflicted":{"hz":0.0,"counter":0,"roughness":0.0},"committed":{"hz":0.0,"counter":0,"roughness":0.0}}},"cluster_controller_timestamp":1415650089,"protocol_version":"fdb00a400050001","configuration":{"resolvers":1,"redundancy":{"factor":{"$enum":["single","double","triple","custom","two_datacenter","three_datacenter","three_data_hall","fast_recovery_double","fast_recovery_triple"]}},"logs":2,"storage_policy":"(zoneid^3x1)","storage_engine":{"$enum":["ssd","ssd-1","ssd-2","memory","custom"]},"coordinators_count":1,"excluded_servers":[{"address":"10.0.4.1"}],"proxies":5,"tlog_policy":"(zoneid^2x1)"},"latency_probe":{"immediate_priority_transaction_start_seconds":0.0,"transaction_start_seconds":0.0,"batch_priority_transaction_start_seconds":0.0,"read_seconds":7,"commit_seconds":0.02},"machines":{"$map":{"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"tcp_segments_retransmitted":{"hz":0.0}},"locality":{"$map":"value"},"memory":{"free_bytes":0,"committed_bytes":0,"total_bytes":0},"contributing_workers":4,"datacenter_id":"6344abf1813eb05b","excluded":false,"address":"1.2.3.4","machine_id":"6344abf1813eb05b","cpu":{"logical_core_utilization":0.4}}},"old_logs":[{"log_write_anti_quorum":0,"log_fault_tolerance":2,"logs":[{"healthy":true,"id":"7f8d623d0cb9966e","address":"1.2.3.4:1234"}],"log_replication_factor":3}]},"client":{"coordinators":{"coordinators":[{"reachable":true,"address":"127.0.0.1:4701"}],"quorum_reachable":true},"cluster_file":{"path":"/etc/foundationdb/fdb.cluster","up_to_date":true},"messages":[{"name":{"$enum":["inconsistent_cluster_file","unreachable_cluster_controller","no_cluster_controller","status_incomplete_client","status_incomplete_coordinators","status_incomplete_error","status_incomplete_timeout","status_incomplete_cluster","quorum_not_reachable"]},"description":"The cluster file is not up to date."}],"timestamp":1415650089,"database_status":{"available":true,"healthy":true}}} + schema={"cluster":{"layers":{"_valid":true,"_error":"some error description"},"processes":{"$map":{"fault_domain":"0ccb4e0fdbdb5583010f6b77d9d10ece","class_source":{"$enum":["command_line","configure_auto","set_class"]},"class_type":{"$enum":["unset","storage","transaction","resolution","proxy","master","test"]},"roles":[{"query_queue_max":0,"data_version_lag":12341234,"input_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"kvstore_used_bytes":12341234,"stored_bytes":12341234,"data_version":12341234,"kvstore_free_bytes":12341234,"durable_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"id":"eb84471d68c12d1d26f692a50000003f","persistent_disk_used_bytes":12341234,"role":{"$enum":["master","proxy","log","storage","resolver","cluster_controller"]},"queue_disk_available_bytes":12341234,"persistent_disk_total_bytes":12341234,"kvstore_available_bytes":12341234,"queue_disk_total_bytes":12341234,"persistent_disk_free_bytes":12341234,"queue_disk_used_bytes":12341234,"queue_disk_free_bytes":12341234,"kvstore_total_bytes":12341234,"finished_queries":{"hz":0.0,"counter":0,"roughness":0.0}}],"locality":{"$map":"value"},"messages":[{"description":"abc","type":"x","name":{"$enum":["file_open_error","incorrect_cluster_file_contents","process_error","io_error","io_timeout","platform_error","storage_server_lagging","(other FDB error messages)"]},"raw_log_message":"","time":12345.12312}],"address":"1.2.3.4:1234","command_line":"-r simulation","disk":{"free_bytes":3451233456234,"reads":{"hz":0.0,"counter":0,"sectors":0},"busy":0.0,"writes":{"hz":0.0,"counter":0,"sectors":0},"total_bytes":123412341234},"version":"3.0.0","excluded":false,"memory":{"available_bytes":0,"used_bytes":0,"limit_bytes":0},"machine_id":"0ccb4e0feddb5583010f6b77d9d10ece","uptime_seconds":1234.2345,"cpu":{"usage_cores":0.0},"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"connections_closed":{"hz":0.0},"connection_errors":{"hz":0.0},"current_connections":0,"connections_established":{"hz":0.0}}}},"clients":{"count":1,"supported_versions":[{"count":1,"protocol_version":"fdb00a400050001","client_version":"3.0.0","source_version":"9430e1127b4991cbc5ab2b17f41cfffa5de07e9d","connected_clients":[{"log_group":"default","address":"127.0.0.1:9898"}]}]},"qos":{"limiting_version_lag_storage_server":0,"released_transactions_per_second":0,"transactions_per_second_limit":0,"limiting_queue_bytes_storage_server":0,"performance_limited_by":{"reason_server_id":"7f8d623d0cb9966e","description":"The database is not being saturated by the workload.","reason_id":0,"name":{"$enum":["workload","storage_server_write_queue_size","storage_server_write_bandwidth_mvcc","storage_server_readable_behind","log_server_mvcc_write_bandwidth","log_server_write_queue","storage_server_min_free_space","storage_server_min_free_space_ratio","log_server_min_free_space","log_server_min_free_space_ratio"]}},"worst_version_lag_storage_server":0,"worst_queue_bytes_log_server":460,"worst_queue_bytes_storage_server":0},"incompatible_connections":[],"database_locked":false,"generation":2,"data":{"least_operating_space_bytes_log_server":0,"average_partition_size_bytes":0,"state":{"healthy":true,"description":"","name":{"$enum":["initializing","missing_data","healing","healthy_repartitioning","healthy_removing_server","healthy_rebalancing","healthy"]},"min_replicas_remaining":0},"least_operating_space_ratio_storage_server":0.1,"max_machine_failures_without_losing_availability":0,"total_disk_used_bytes":0,"total_kv_size_bytes":0,"max_machine_failures_without_losing_data":0,"moving_data":{"in_queue_bytes":0,"total_written_bytes":0,"in_flight_bytes":0},"least_operating_space_bytes_storage_server":0,"partitions_count":2},"fault_tolerance":{"max_machine_failures_without_losing_availability":0,"max_machine_failures_without_losing_data":0},"messages":[{"reasons":[{"description":"Blah."}],"unreachable_processes":[{"address":"1.2.3.4:1234"}],"name":{"$enum":["unreachable_master_worker","unreadable_configuration","client_issues","unreachable_processes","immediate_priority_transaction_start_probe_timeout","batch_priority_transaction_start_probe_timeout","transaction_start_probe_timeout","read_probe_timeout","commit_probe_timeout","storage_servers_error","status_incomplete","layer_status_incomplete","database_availability_timeout"]},"issues":[{"name":{"$enum":["incorrect_cluster_file_contents"]},"description":"Cluster file contents do not match current cluster connection string. Verify cluster file is writable and has not been overwritten externally."}],"description":"abc"}],"database_available":true,"recovery_state":{"required_proxies":1,"name":{"$enum":["reading_coordinated_state","locking_coordinated_state","locking_old_transaction_servers","reading_transaction_system_state","configuration_missing","configuration_never_created","configuration_invalid","recruiting_transaction_servers","initializing_transaction_servers","recovery_transaction","writing_coordinated_state","fully_recovered"]},"missing_logs":"7f8d623d0cb9966e","required_resolvers":1,"required_logs":3,"description":"Recovery complete."},"workload":{"operations":{"writes":{"hz":0.0,"counter":0,"roughness":0.0},"reads":{"hz":0.0}},"bytes":{"written":{"hz":0.0,"counter":0,"roughness":0.0}},"transactions":{"started":{"hz":0.0,"counter":0,"roughness":0.0},"conflicted":{"hz":0.0,"counter":0,"roughness":0.0},"committed":{"hz":0.0,"counter":0,"roughness":0.0}}},"cluster_controller_timestamp":1415650089,"protocol_version":"fdb00a400050001","configuration":{"resolvers":1,"redundancy":{"factor":{"$enum":["single","double","triple","custom","two_datacenter","three_datacenter","three_data_hall","fast_recovery_double","fast_recovery_triple"]}},"logs":2,"storage_policy":"(zoneid^3x1)","storage_engine":{"$enum":["ssd","ssd-1","ssd-2","memory","custom"]},"coordinators_count":1,"excluded_servers":[{"address":"10.0.4.1"}],"proxies":5,"tlog_policy":"(zoneid^2x1)"},"latency_probe":{"immediate_priority_transaction_start_seconds":0.0,"transaction_start_seconds":0.0,"batch_priority_transaction_start_seconds":0.0,"read_seconds":7,"commit_seconds":0.02},"machines":{"$map":{"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"tcp_segments_retransmitted":{"hz":0.0}},"locality":{"$map":"value"},"memory":{"free_bytes":0,"committed_bytes":0,"total_bytes":0},"contributing_workers":4,"datacenter_id":"6344abf1813eb05b","excluded":false,"address":"1.2.3.4","machine_id":"6344abf1813eb05b","cpu":{"logical_core_utilization":0.4}}},"old_logs":[{"log_write_anti_quorum":0,"log_fault_tolerance":2,"logs":[{"healthy":true,"id":"7f8d623d0cb9966e","address":"1.2.3.4:1234"}],"log_replication_factor":3}]},"client":{"coordinators":{"coordinators":[{"reachable":true,"address":"127.0.0.1:4701"}],"quorum_reachable":true},"cluster_file":{"path":"/etc/foundationdb/fdb.cluster","up_to_date":true},"messages":[{"name":{"$enum":["inconsistent_cluster_file","unreachable_cluster_controller","no_cluster_controller","status_incomplete_client","status_incomplete_coordinators","status_incomplete_error","status_incomplete_timeout","status_incomplete_cluster","quorum_not_reachable"]},"description":"The cluster file is not up to date."}],"timestamp":1415650089,"database_status":{"available":true,"healthy":true}}} testName=RandomClogging testDuration=30.0 diff --git a/tests/rare/LargeApiCorrectnessStatus.txt b/tests/rare/LargeApiCorrectnessStatus.txt index fbb67df02d..8e4195a140 100644 --- a/tests/rare/LargeApiCorrectnessStatus.txt +++ b/tests/rare/LargeApiCorrectnessStatus.txt @@ -24,4 +24,4 @@ testTitle=ApiCorrectnessTest testName=Status testDuration=30.0 - schema={"cluster":{"layers":{"_valid":true,"_error":"some error description"},"processes":{"$map":{"fault_domain":"0ccb4e0fdbdb5583010f6b77d9d10ece","class_source":{"$enum":["command_line","configure_auto","set_class"]},"class_type":{"$enum":["unset","storage","transaction","resolution","proxy","master","test"]},"roles":[{"query_queue_max":0,"data_version_lag":12341234,"input_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"kvstore_used_bytes":12341234,"stored_bytes":12341234,"data_version":12341234,"kvstore_free_bytes":12341234,"durable_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"id":"eb84471d68c12d1d26f692a50000003f","persistent_disk_used_bytes":12341234,"role":{"$enum":["master","proxy","log","storage","resolver","cluster_controller"]},"persistent_disk_total_bytes":12341234,"queue_disk_total_bytes":12341234,"persistent_disk_free_bytes":12341234,"queue_disk_used_bytes":12341234,"queue_disk_free_bytes":12341234,"kvstore_total_bytes":12341234,"finished_queries":{"hz":0.0,"counter":0,"roughness":0.0}}],"locality":{"$map":"value"},"messages":[{"description":"abc","type":"x","name":{"$enum":["file_open_error","incorrect_cluster_file_contents","process_error","io_error","io_timeout","platform_error","storage_server_lagging","(other FDB error messages)"]},"raw_log_message":"","time":12345.12312}],"address":"1.2.3.4:1234","command_line":"-r simulation","disk":{"free_bytes":3451233456234,"reads":{"hz":0.0,"counter":0,"sectors":0},"busy":0.0,"writes":{"hz":0.0,"counter":0,"sectors":0},"total_bytes":123412341234},"version":"3.0.0","excluded":false,"memory":{"available_bytes":0,"used_bytes":0,"limit_bytes":0},"machine_id":"0ccb4e0feddb5583010f6b77d9d10ece","uptime_seconds":1234.2345,"cpu":{"usage_cores":0.0},"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"connections_closed":{"hz":0.0},"connection_errors":{"hz":0.0},"current_connections":0,"connections_established":{"hz":0.0}}}},"clients":{"count":1,"supported_versions":[{"count":1,"protocol_version":"fdb00a400050001","client_version":"3.0.0","source_version":"9430e1127b4991cbc5ab2b17f41cfffa5de07e9d","connected_clients":[{"log_group":"default","address":"127.0.0.1:9898"}]}]},"qos":{"limiting_version_lag_storage_server":0,"released_transactions_per_second":0,"transactions_per_second_limit":0,"limiting_queue_bytes_storage_server":0,"performance_limited_by":{"reason_server_id":"7f8d623d0cb9966e","description":"The database is not being saturated by the workload.","reason_id":0,"name":{"$enum":["workload","storage_server_write_queue_size","storage_server_write_bandwidth_mvcc","storage_server_readable_behind","log_server_mvcc_write_bandwidth","log_server_write_queue","storage_server_min_free_space","storage_server_min_free_space_ratio","log_server_min_free_space","log_server_min_free_space_ratio"]}},"worst_version_lag_storage_server":0,"worst_queue_bytes_log_server":460,"worst_queue_bytes_storage_server":0},"incompatible_connections":[],"database_locked":false,"generation":2,"data":{"least_operating_space_bytes_log_server":0,"average_partition_size_bytes":0,"state":{"healthy":true,"description":"","name":{"$enum":["initializing","missing_data","healing","healthy_repartitioning","healthy_removing_server","healthy_rebalancing","healthy"]},"min_replicas_remaining":0},"least_operating_space_ratio_storage_server":0.1,"max_machine_failures_without_losing_availability":0,"total_disk_used_bytes":0,"total_kv_size_bytes":0,"max_machine_failures_without_losing_data":0,"moving_data":{"in_queue_bytes":0,"total_written_bytes":0,"in_flight_bytes":0},"least_operating_space_bytes_storage_server":0,"partitions_count":2},"fault_tolerance":{"max_machine_failures_without_losing_availability":0,"max_machine_failures_without_losing_data":0},"messages":[{"reasons":[{"description":"Blah."}],"unreachable_processes":[{"address":"1.2.3.4:1234"}],"name":{"$enum":["unreachable_master_worker","unreadable_configuration","client_issues","unreachable_processes","immediate_priority_transaction_start_probe_timeout","batch_priority_transaction_start_probe_timeout","transaction_start_probe_timeout","read_probe_timeout","commit_probe_timeout","storage_servers_error","status_incomplete","layer_status_incomplete","database_availability_timeout"]},"issues":[{"name":{"$enum":["incorrect_cluster_file_contents"]},"description":"Cluster file contents do not match current cluster connection string. Verify cluster file is writable and has not been overwritten externally."}],"description":"abc"}],"database_available":true,"recovery_state":{"required_proxies":1,"name":{"$enum":["reading_coordinated_state","locking_coordinated_state","locking_old_transaction_servers","reading_transaction_system_state","configuration_missing","configuration_never_created","configuration_invalid","recruiting_transaction_servers","initializing_transaction_servers","recovery_transaction","writing_coordinated_state","fully_recovered"]},"missing_logs":"7f8d623d0cb9966e","required_resolvers":1,"required_logs":3,"description":"Recovery complete."},"workload":{"operations":{"writes":{"hz":0.0,"counter":0,"roughness":0.0},"reads":{"hz":0.0}},"bytes":{"written":{"hz":0.0,"counter":0,"roughness":0.0}},"transactions":{"started":{"hz":0.0,"counter":0,"roughness":0.0},"conflicted":{"hz":0.0,"counter":0,"roughness":0.0},"committed":{"hz":0.0,"counter":0,"roughness":0.0}}},"cluster_controller_timestamp":1415650089,"protocol_version":"fdb00a400050001","configuration":{"resolvers":1,"redundancy":{"factor":{"$enum":["single","double","triple","custom","two_datacenter","three_datacenter","three_data_hall","fast_recovery_double","fast_recovery_triple"]}},"logs":2,"storage_policy":"(zoneid^3x1)","storage_engine":{"$enum":["ssd","ssd-1","ssd-2","memory","custom"]},"coordinators_count":1,"excluded_servers":[{"address":"10.0.4.1"}],"proxies":5,"tlog_policy":"(zoneid^2x1)"},"latency_probe":{"immediate_priority_transaction_start_seconds":0.0,"transaction_start_seconds":0.0,"batch_priority_transaction_start_seconds":0.0,"read_seconds":7,"commit_seconds":0.02},"machines":{"$map":{"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"tcp_segments_retransmitted":{"hz":0.0}},"locality":{"$map":"value"},"memory":{"free_bytes":0,"committed_bytes":0,"total_bytes":0},"contributing_workers":4,"datacenter_id":"6344abf1813eb05b","excluded":false,"address":"1.2.3.4","machine_id":"6344abf1813eb05b","cpu":{"logical_core_utilization":0.4}}},"old_logs":[{"log_write_anti_quorum":0,"log_fault_tolerance":2,"logs":[{"healthy":true,"id":"7f8d623d0cb9966e","address":"1.2.3.4:1234"}],"log_replication_factor":3}]},"client":{"coordinators":{"coordinators":[{"reachable":true,"address":"127.0.0.1:4701"}],"quorum_reachable":true},"cluster_file":{"path":"/etc/foundationdb/fdb.cluster","up_to_date":true},"messages":[{"name":{"$enum":["inconsistent_cluster_file","unreachable_cluster_controller","no_cluster_controller","status_incomplete_client","status_incomplete_coordinators","status_incomplete_error","status_incomplete_timeout","status_incomplete_cluster","quorum_not_reachable"]},"description":"The cluster file is not up to date."}],"timestamp":1415650089,"database_status":{"available":true,"healthy":true}}} + schema={"cluster":{"layers":{"_valid":true,"_error":"some error description"},"processes":{"$map":{"fault_domain":"0ccb4e0fdbdb5583010f6b77d9d10ece","class_source":{"$enum":["command_line","configure_auto","set_class"]},"class_type":{"$enum":["unset","storage","transaction","resolution","proxy","master","test"]},"roles":[{"query_queue_max":0,"data_version_lag":12341234,"input_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"kvstore_used_bytes":12341234,"stored_bytes":12341234,"data_version":12341234,"kvstore_free_bytes":12341234,"durable_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"id":"eb84471d68c12d1d26f692a50000003f","persistent_disk_used_bytes":12341234,"role":{"$enum":["master","proxy","log","storage","resolver","cluster_controller"]},"queue_disk_available_bytes":12341234,"persistent_disk_total_bytes":12341234,"kvstore_available_bytes":12341234,"queue_disk_total_bytes":12341234,"persistent_disk_free_bytes":12341234,"queue_disk_used_bytes":12341234,"queue_disk_free_bytes":12341234,"kvstore_total_bytes":12341234,"finished_queries":{"hz":0.0,"counter":0,"roughness":0.0}}],"locality":{"$map":"value"},"messages":[{"description":"abc","type":"x","name":{"$enum":["file_open_error","incorrect_cluster_file_contents","process_error","io_error","io_timeout","platform_error","storage_server_lagging","(other FDB error messages)"]},"raw_log_message":"","time":12345.12312}],"address":"1.2.3.4:1234","command_line":"-r simulation","disk":{"free_bytes":3451233456234,"reads":{"hz":0.0,"counter":0,"sectors":0},"busy":0.0,"writes":{"hz":0.0,"counter":0,"sectors":0},"total_bytes":123412341234},"version":"3.0.0","excluded":false,"memory":{"available_bytes":0,"used_bytes":0,"limit_bytes":0},"machine_id":"0ccb4e0feddb5583010f6b77d9d10ece","uptime_seconds":1234.2345,"cpu":{"usage_cores":0.0},"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"connections_closed":{"hz":0.0},"connection_errors":{"hz":0.0},"current_connections":0,"connections_established":{"hz":0.0}}}},"clients":{"count":1,"supported_versions":[{"count":1,"protocol_version":"fdb00a400050001","client_version":"3.0.0","source_version":"9430e1127b4991cbc5ab2b17f41cfffa5de07e9d","connected_clients":[{"log_group":"default","address":"127.0.0.1:9898"}]}]},"qos":{"limiting_version_lag_storage_server":0,"released_transactions_per_second":0,"transactions_per_second_limit":0,"limiting_queue_bytes_storage_server":0,"performance_limited_by":{"reason_server_id":"7f8d623d0cb9966e","description":"The database is not being saturated by the workload.","reason_id":0,"name":{"$enum":["workload","storage_server_write_queue_size","storage_server_write_bandwidth_mvcc","storage_server_readable_behind","log_server_mvcc_write_bandwidth","log_server_write_queue","storage_server_min_free_space","storage_server_min_free_space_ratio","log_server_min_free_space","log_server_min_free_space_ratio"]}},"worst_version_lag_storage_server":0,"worst_queue_bytes_log_server":460,"worst_queue_bytes_storage_server":0},"incompatible_connections":[],"database_locked":false,"generation":2,"data":{"least_operating_space_bytes_log_server":0,"average_partition_size_bytes":0,"state":{"healthy":true,"description":"","name":{"$enum":["initializing","missing_data","healing","healthy_repartitioning","healthy_removing_server","healthy_rebalancing","healthy"]},"min_replicas_remaining":0},"least_operating_space_ratio_storage_server":0.1,"max_machine_failures_without_losing_availability":0,"total_disk_used_bytes":0,"total_kv_size_bytes":0,"max_machine_failures_without_losing_data":0,"moving_data":{"in_queue_bytes":0,"total_written_bytes":0,"in_flight_bytes":0},"least_operating_space_bytes_storage_server":0,"partitions_count":2},"fault_tolerance":{"max_machine_failures_without_losing_availability":0,"max_machine_failures_without_losing_data":0},"messages":[{"reasons":[{"description":"Blah."}],"unreachable_processes":[{"address":"1.2.3.4:1234"}],"name":{"$enum":["unreachable_master_worker","unreadable_configuration","client_issues","unreachable_processes","immediate_priority_transaction_start_probe_timeout","batch_priority_transaction_start_probe_timeout","transaction_start_probe_timeout","read_probe_timeout","commit_probe_timeout","storage_servers_error","status_incomplete","layer_status_incomplete","database_availability_timeout"]},"issues":[{"name":{"$enum":["incorrect_cluster_file_contents"]},"description":"Cluster file contents do not match current cluster connection string. Verify cluster file is writable and has not been overwritten externally."}],"description":"abc"}],"database_available":true,"recovery_state":{"required_proxies":1,"name":{"$enum":["reading_coordinated_state","locking_coordinated_state","locking_old_transaction_servers","reading_transaction_system_state","configuration_missing","configuration_never_created","configuration_invalid","recruiting_transaction_servers","initializing_transaction_servers","recovery_transaction","writing_coordinated_state","fully_recovered"]},"missing_logs":"7f8d623d0cb9966e","required_resolvers":1,"required_logs":3,"description":"Recovery complete."},"workload":{"operations":{"writes":{"hz":0.0,"counter":0,"roughness":0.0},"reads":{"hz":0.0}},"bytes":{"written":{"hz":0.0,"counter":0,"roughness":0.0}},"transactions":{"started":{"hz":0.0,"counter":0,"roughness":0.0},"conflicted":{"hz":0.0,"counter":0,"roughness":0.0},"committed":{"hz":0.0,"counter":0,"roughness":0.0}}},"cluster_controller_timestamp":1415650089,"protocol_version":"fdb00a400050001","configuration":{"resolvers":1,"redundancy":{"factor":{"$enum":["single","double","triple","custom","two_datacenter","three_datacenter","three_data_hall","fast_recovery_double","fast_recovery_triple"]}},"logs":2,"storage_policy":"(zoneid^3x1)","storage_engine":{"$enum":["ssd","ssd-1","ssd-2","memory","custom"]},"coordinators_count":1,"excluded_servers":[{"address":"10.0.4.1"}],"proxies":5,"tlog_policy":"(zoneid^2x1)"},"latency_probe":{"immediate_priority_transaction_start_seconds":0.0,"transaction_start_seconds":0.0,"batch_priority_transaction_start_seconds":0.0,"read_seconds":7,"commit_seconds":0.02},"machines":{"$map":{"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"tcp_segments_retransmitted":{"hz":0.0}},"locality":{"$map":"value"},"memory":{"free_bytes":0,"committed_bytes":0,"total_bytes":0},"contributing_workers":4,"datacenter_id":"6344abf1813eb05b","excluded":false,"address":"1.2.3.4","machine_id":"6344abf1813eb05b","cpu":{"logical_core_utilization":0.4}}},"old_logs":[{"log_write_anti_quorum":0,"log_fault_tolerance":2,"logs":[{"healthy":true,"id":"7f8d623d0cb9966e","address":"1.2.3.4:1234"}],"log_replication_factor":3}]},"client":{"coordinators":{"coordinators":[{"reachable":true,"address":"127.0.0.1:4701"}],"quorum_reachable":true},"cluster_file":{"path":"/etc/foundationdb/fdb.cluster","up_to_date":true},"messages":[{"name":{"$enum":["inconsistent_cluster_file","unreachable_cluster_controller","no_cluster_controller","status_incomplete_client","status_incomplete_coordinators","status_incomplete_error","status_incomplete_timeout","status_incomplete_cluster","quorum_not_reachable"]},"description":"The cluster file is not up to date."}],"timestamp":1415650089,"database_status":{"available":true,"healthy":true}}} diff --git a/tests/slow/DDBalanceAndRemoveStatus.txt b/tests/slow/DDBalanceAndRemoveStatus.txt index c256d4f917..ccd6c0bac2 100644 --- a/tests/slow/DDBalanceAndRemoveStatus.txt +++ b/tests/slow/DDBalanceAndRemoveStatus.txt @@ -43,4 +43,4 @@ testTitle=DDBalance_test testName=Status testDuration=30.0 - schema={"cluster":{"layers":{"_valid":true,"_error":"some error description"},"processes":{"$map":{"fault_domain":"0ccb4e0fdbdb5583010f6b77d9d10ece","class_source":{"$enum":["command_line","configure_auto","set_class"]},"class_type":{"$enum":["unset","storage","transaction","resolution","proxy","master","test"]},"roles":[{"query_queue_max":0,"data_version_lag":12341234,"input_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"kvstore_used_bytes":12341234,"stored_bytes":12341234,"data_version":12341234,"kvstore_free_bytes":12341234,"durable_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"id":"eb84471d68c12d1d26f692a50000003f","persistent_disk_used_bytes":12341234,"role":{"$enum":["master","proxy","log","storage","resolver","cluster_controller"]},"persistent_disk_total_bytes":12341234,"queue_disk_total_bytes":12341234,"persistent_disk_free_bytes":12341234,"queue_disk_used_bytes":12341234,"queue_disk_free_bytes":12341234,"kvstore_total_bytes":12341234,"finished_queries":{"hz":0.0,"counter":0,"roughness":0.0}}],"locality":{"$map":"value"},"messages":[{"description":"abc","type":"x","name":{"$enum":["file_open_error","incorrect_cluster_file_contents","process_error","io_error","io_timeout","platform_error","storage_server_lagging","(other FDB error messages)"]},"raw_log_message":"","time":12345.12312}],"address":"1.2.3.4:1234","command_line":"-r simulation","disk":{"free_bytes":3451233456234,"reads":{"hz":0.0,"counter":0,"sectors":0},"busy":0.0,"writes":{"hz":0.0,"counter":0,"sectors":0},"total_bytes":123412341234},"version":"3.0.0","excluded":false,"memory":{"available_bytes":0,"used_bytes":0,"limit_bytes":0},"machine_id":"0ccb4e0feddb5583010f6b77d9d10ece","uptime_seconds":1234.2345,"cpu":{"usage_cores":0.0},"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"connections_closed":{"hz":0.0},"connection_errors":{"hz":0.0},"current_connections":0,"connections_established":{"hz":0.0}}}},"clients":{"count":1,"supported_versions":[{"count":1,"protocol_version":"fdb00a400050001","client_version":"3.0.0","source_version":"9430e1127b4991cbc5ab2b17f41cfffa5de07e9d","connected_clients":[{"log_group":"default","address":"127.0.0.1:9898"}]}]},"qos":{"limiting_version_lag_storage_server":0,"released_transactions_per_second":0,"transactions_per_second_limit":0,"limiting_queue_bytes_storage_server":0,"performance_limited_by":{"reason_server_id":"7f8d623d0cb9966e","description":"The database is not being saturated by the workload.","reason_id":0,"name":{"$enum":["workload","storage_server_write_queue_size","storage_server_write_bandwidth_mvcc","storage_server_readable_behind","log_server_mvcc_write_bandwidth","log_server_write_queue","storage_server_min_free_space","storage_server_min_free_space_ratio","log_server_min_free_space","log_server_min_free_space_ratio"]}},"worst_version_lag_storage_server":0,"worst_queue_bytes_log_server":460,"worst_queue_bytes_storage_server":0},"incompatible_connections":[],"database_locked":false,"generation":2,"data":{"least_operating_space_bytes_log_server":0,"average_partition_size_bytes":0,"state":{"healthy":true,"description":"","name":{"$enum":["initializing","missing_data","healing","healthy_repartitioning","healthy_removing_server","healthy_rebalancing","healthy"]},"min_replicas_remaining":0},"least_operating_space_ratio_storage_server":0.1,"max_machine_failures_without_losing_availability":0,"total_disk_used_bytes":0,"total_kv_size_bytes":0,"max_machine_failures_without_losing_data":0,"moving_data":{"in_queue_bytes":0,"total_written_bytes":0,"in_flight_bytes":0},"least_operating_space_bytes_storage_server":0,"partitions_count":2},"fault_tolerance":{"max_machine_failures_without_losing_availability":0,"max_machine_failures_without_losing_data":0},"messages":[{"reasons":[{"description":"Blah."}],"unreachable_processes":[{"address":"1.2.3.4:1234"}],"name":{"$enum":["unreachable_master_worker","unreadable_configuration","client_issues","unreachable_processes","immediate_priority_transaction_start_probe_timeout","batch_priority_transaction_start_probe_timeout","transaction_start_probe_timeout","read_probe_timeout","commit_probe_timeout","storage_servers_error","status_incomplete","layer_status_incomplete","database_availability_timeout"]},"issues":[{"name":{"$enum":["incorrect_cluster_file_contents"]},"description":"Cluster file contents do not match current cluster connection string. Verify cluster file is writable and has not been overwritten externally."}],"description":"abc"}],"database_available":true,"recovery_state":{"required_proxies":1,"name":{"$enum":["reading_coordinated_state","locking_coordinated_state","locking_old_transaction_servers","reading_transaction_system_state","configuration_missing","configuration_never_created","configuration_invalid","recruiting_transaction_servers","initializing_transaction_servers","recovery_transaction","writing_coordinated_state","fully_recovered"]},"missing_logs":"7f8d623d0cb9966e","required_resolvers":1,"required_logs":3,"description":"Recovery complete."},"workload":{"operations":{"writes":{"hz":0.0,"counter":0,"roughness":0.0},"reads":{"hz":0.0}},"bytes":{"written":{"hz":0.0,"counter":0,"roughness":0.0}},"transactions":{"started":{"hz":0.0,"counter":0,"roughness":0.0},"conflicted":{"hz":0.0,"counter":0,"roughness":0.0},"committed":{"hz":0.0,"counter":0,"roughness":0.0}}},"cluster_controller_timestamp":1415650089,"protocol_version":"fdb00a400050001","configuration":{"resolvers":1,"redundancy":{"factor":{"$enum":["single","double","triple","custom","two_datacenter","three_datacenter","three_data_hall","fast_recovery_double","fast_recovery_triple"]}},"logs":2,"storage_policy":"(zoneid^3x1)","storage_engine":{"$enum":["ssd","ssd-1","ssd-2","memory","custom"]},"coordinators_count":1,"excluded_servers":[{"address":"10.0.4.1"}],"proxies":5,"tlog_policy":"(zoneid^2x1)"},"latency_probe":{"immediate_priority_transaction_start_seconds":0.0,"transaction_start_seconds":0.0,"batch_priority_transaction_start_seconds":0.0,"read_seconds":7,"commit_seconds":0.02},"machines":{"$map":{"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"tcp_segments_retransmitted":{"hz":0.0}},"locality":{"$map":"value"},"memory":{"free_bytes":0,"committed_bytes":0,"total_bytes":0},"contributing_workers":4,"datacenter_id":"6344abf1813eb05b","excluded":false,"address":"1.2.3.4","machine_id":"6344abf1813eb05b","cpu":{"logical_core_utilization":0.4}}},"old_logs":[{"log_write_anti_quorum":0,"log_fault_tolerance":2,"logs":[{"healthy":true,"id":"7f8d623d0cb9966e","address":"1.2.3.4:1234"}],"log_replication_factor":3}]},"client":{"coordinators":{"coordinators":[{"reachable":true,"address":"127.0.0.1:4701"}],"quorum_reachable":true},"cluster_file":{"path":"/etc/foundationdb/fdb.cluster","up_to_date":true},"messages":[{"name":{"$enum":["inconsistent_cluster_file","unreachable_cluster_controller","no_cluster_controller","status_incomplete_client","status_incomplete_coordinators","status_incomplete_error","status_incomplete_timeout","status_incomplete_cluster","quorum_not_reachable"]},"description":"The cluster file is not up to date."}],"timestamp":1415650089,"database_status":{"available":true,"healthy":true}}} + schema={"cluster":{"layers":{"_valid":true,"_error":"some error description"},"processes":{"$map":{"fault_domain":"0ccb4e0fdbdb5583010f6b77d9d10ece","class_source":{"$enum":["command_line","configure_auto","set_class"]},"class_type":{"$enum":["unset","storage","transaction","resolution","proxy","master","test"]},"roles":[{"query_queue_max":0,"data_version_lag":12341234,"input_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"kvstore_used_bytes":12341234,"stored_bytes":12341234,"data_version":12341234,"kvstore_free_bytes":12341234,"durable_bytes":{"hz":0.0,"counter":0,"roughness":0.0},"id":"eb84471d68c12d1d26f692a50000003f","persistent_disk_used_bytes":12341234,"role":{"$enum":["master","proxy","log","storage","resolver","cluster_controller"]},"queue_disk_available_bytes":12341234,"persistent_disk_total_bytes":12341234,"kvstore_available_bytes":12341234,"queue_disk_total_bytes":12341234,"persistent_disk_free_bytes":12341234,"queue_disk_used_bytes":12341234,"queue_disk_free_bytes":12341234,"kvstore_total_bytes":12341234,"finished_queries":{"hz":0.0,"counter":0,"roughness":0.0}}],"locality":{"$map":"value"},"messages":[{"description":"abc","type":"x","name":{"$enum":["file_open_error","incorrect_cluster_file_contents","process_error","io_error","io_timeout","platform_error","storage_server_lagging","(other FDB error messages)"]},"raw_log_message":"","time":12345.12312}],"address":"1.2.3.4:1234","command_line":"-r simulation","disk":{"free_bytes":3451233456234,"reads":{"hz":0.0,"counter":0,"sectors":0},"busy":0.0,"writes":{"hz":0.0,"counter":0,"sectors":0},"total_bytes":123412341234},"version":"3.0.0","excluded":false,"memory":{"available_bytes":0,"used_bytes":0,"limit_bytes":0},"machine_id":"0ccb4e0feddb5583010f6b77d9d10ece","uptime_seconds":1234.2345,"cpu":{"usage_cores":0.0},"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"connections_closed":{"hz":0.0},"connection_errors":{"hz":0.0},"current_connections":0,"connections_established":{"hz":0.0}}}},"clients":{"count":1,"supported_versions":[{"count":1,"protocol_version":"fdb00a400050001","client_version":"3.0.0","source_version":"9430e1127b4991cbc5ab2b17f41cfffa5de07e9d","connected_clients":[{"log_group":"default","address":"127.0.0.1:9898"}]}]},"qos":{"limiting_version_lag_storage_server":0,"released_transactions_per_second":0,"transactions_per_second_limit":0,"limiting_queue_bytes_storage_server":0,"performance_limited_by":{"reason_server_id":"7f8d623d0cb9966e","description":"The database is not being saturated by the workload.","reason_id":0,"name":{"$enum":["workload","storage_server_write_queue_size","storage_server_write_bandwidth_mvcc","storage_server_readable_behind","log_server_mvcc_write_bandwidth","log_server_write_queue","storage_server_min_free_space","storage_server_min_free_space_ratio","log_server_min_free_space","log_server_min_free_space_ratio"]}},"worst_version_lag_storage_server":0,"worst_queue_bytes_log_server":460,"worst_queue_bytes_storage_server":0},"incompatible_connections":[],"database_locked":false,"generation":2,"data":{"least_operating_space_bytes_log_server":0,"average_partition_size_bytes":0,"state":{"healthy":true,"description":"","name":{"$enum":["initializing","missing_data","healing","healthy_repartitioning","healthy_removing_server","healthy_rebalancing","healthy"]},"min_replicas_remaining":0},"least_operating_space_ratio_storage_server":0.1,"max_machine_failures_without_losing_availability":0,"total_disk_used_bytes":0,"total_kv_size_bytes":0,"max_machine_failures_without_losing_data":0,"moving_data":{"in_queue_bytes":0,"total_written_bytes":0,"in_flight_bytes":0},"least_operating_space_bytes_storage_server":0,"partitions_count":2},"fault_tolerance":{"max_machine_failures_without_losing_availability":0,"max_machine_failures_without_losing_data":0},"messages":[{"reasons":[{"description":"Blah."}],"unreachable_processes":[{"address":"1.2.3.4:1234"}],"name":{"$enum":["unreachable_master_worker","unreadable_configuration","client_issues","unreachable_processes","immediate_priority_transaction_start_probe_timeout","batch_priority_transaction_start_probe_timeout","transaction_start_probe_timeout","read_probe_timeout","commit_probe_timeout","storage_servers_error","status_incomplete","layer_status_incomplete","database_availability_timeout"]},"issues":[{"name":{"$enum":["incorrect_cluster_file_contents"]},"description":"Cluster file contents do not match current cluster connection string. Verify cluster file is writable and has not been overwritten externally."}],"description":"abc"}],"database_available":true,"recovery_state":{"required_proxies":1,"name":{"$enum":["reading_coordinated_state","locking_coordinated_state","locking_old_transaction_servers","reading_transaction_system_state","configuration_missing","configuration_never_created","configuration_invalid","recruiting_transaction_servers","initializing_transaction_servers","recovery_transaction","writing_coordinated_state","fully_recovered"]},"missing_logs":"7f8d623d0cb9966e","required_resolvers":1,"required_logs":3,"description":"Recovery complete."},"workload":{"operations":{"writes":{"hz":0.0,"counter":0,"roughness":0.0},"reads":{"hz":0.0}},"bytes":{"written":{"hz":0.0,"counter":0,"roughness":0.0}},"transactions":{"started":{"hz":0.0,"counter":0,"roughness":0.0},"conflicted":{"hz":0.0,"counter":0,"roughness":0.0},"committed":{"hz":0.0,"counter":0,"roughness":0.0}}},"cluster_controller_timestamp":1415650089,"protocol_version":"fdb00a400050001","configuration":{"resolvers":1,"redundancy":{"factor":{"$enum":["single","double","triple","custom","two_datacenter","three_datacenter","three_data_hall","fast_recovery_double","fast_recovery_triple"]}},"logs":2,"storage_policy":"(zoneid^3x1)","storage_engine":{"$enum":["ssd","ssd-1","ssd-2","memory","custom"]},"coordinators_count":1,"excluded_servers":[{"address":"10.0.4.1"}],"proxies":5,"tlog_policy":"(zoneid^2x1)"},"latency_probe":{"immediate_priority_transaction_start_seconds":0.0,"transaction_start_seconds":0.0,"batch_priority_transaction_start_seconds":0.0,"read_seconds":7,"commit_seconds":0.02},"machines":{"$map":{"network":{"megabits_sent":{"hz":0.0},"megabits_received":{"hz":0.0},"tcp_segments_retransmitted":{"hz":0.0}},"locality":{"$map":"value"},"memory":{"free_bytes":0,"committed_bytes":0,"total_bytes":0},"contributing_workers":4,"datacenter_id":"6344abf1813eb05b","excluded":false,"address":"1.2.3.4","machine_id":"6344abf1813eb05b","cpu":{"logical_core_utilization":0.4}}},"old_logs":[{"log_write_anti_quorum":0,"log_fault_tolerance":2,"logs":[{"healthy":true,"id":"7f8d623d0cb9966e","address":"1.2.3.4:1234"}],"log_replication_factor":3}]},"client":{"coordinators":{"coordinators":[{"reachable":true,"address":"127.0.0.1:4701"}],"quorum_reachable":true},"cluster_file":{"path":"/etc/foundationdb/fdb.cluster","up_to_date":true},"messages":[{"name":{"$enum":["inconsistent_cluster_file","unreachable_cluster_controller","no_cluster_controller","status_incomplete_client","status_incomplete_coordinators","status_incomplete_error","status_incomplete_timeout","status_incomplete_cluster","quorum_not_reachable"]},"description":"The cluster file is not up to date."}],"timestamp":1415650089,"database_status":{"available":true,"healthy":true}}}