2017-05-26 04:48:44 +08:00
/*
* Status . actor . cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013 - 2018 Apple Inc . and the FoundationDB project authors
2018-02-22 02:25:11 +08:00
*
2017-05-26 04:48:44 +08:00
* Licensed under the Apache License , Version 2.0 ( the " License " ) ;
* you may not use this file except in compliance with the License .
* You may obtain a copy of the License at
2018-02-22 02:25:11 +08:00
*
2017-05-26 04:48:44 +08:00
* http : //www.apache.org/licenses/LICENSE-2.0
2018-02-22 02:25:11 +08:00
*
2017-05-26 04:48:44 +08:00
* Unless required by applicable law or agreed to in writing , software
* distributed under the License is distributed on an " AS IS " BASIS ,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND , either express or implied .
* See the License for the specific language governing permissions and
* limitations under the License .
*/
2019-05-05 01:52:02 +08:00
# include <cinttypes>
2018-10-20 01:30:13 +08:00
# include "fdbserver/Status.h"
2017-05-26 04:48:44 +08:00
# include "flow/Trace.h"
2019-02-18 07:41:16 +08:00
# include "fdbclient/NativeAPI.actor.h"
2017-05-26 04:48:44 +08:00
# include "fdbclient/SystemData.h"
# include "fdbclient/ReadYourWrites.h"
2019-02-18 11:13:26 +08:00
# include "fdbserver/WorkerInterface.actor.h"
2018-10-20 01:30:13 +08:00
# include "fdbserver/ClusterRecruitmentInterface.h"
2017-05-26 04:48:44 +08:00
# include <time.h>
2018-10-20 01:30:13 +08:00
# include "fdbserver/CoordinationInterface.h"
2019-03-06 02:29:37 +08:00
# include "fdbserver/DataDistribution.actor.h"
2017-05-26 04:48:44 +08:00
# include "flow/UnitTest.h"
2018-10-20 01:30:13 +08:00
# include "fdbserver/QuietDatabase.h"
# include "fdbserver/RecoveryState.h"
2018-09-11 10:01:24 +08:00
# include "fdbclient/JsonBuilder.h"
2018-08-11 06:18:24 +08:00
# include "flow/actorcompiler.h" // This must be the last #include.
2018-09-08 22:15:28 +08:00
2019-07-24 10:22:44 +08:00
void setIssues ( ProcessIssuesMap & issueMap , NetworkAddress const & addr , VectorRef < StringRef > const & issues ,
Optional < UID > & issueID ) {
if ( issues . size ( ) ) {
auto & e = issueMap [ addr ] ;
e . first = issues ;
e . second = deterministicRandom ( ) - > randomUniqueID ( ) ;
issueID = e . second ;
} else {
issueMap . erase ( addr ) ;
issueID = Optional < UID > ( ) ;
}
}
void removeIssues ( ProcessIssuesMap & issueMap , NetworkAddress const & addr , Optional < UID > & issueID ) {
if ( ! issueID . present ( ) ) {
return ;
}
if ( issueMap . count ( addr ) & & issueMap [ addr ] . second = = issueID . get ( ) ) {
issueMap . erase ( addr ) ;
}
}
2017-05-26 04:48:44 +08:00
const char * RecoveryStatus : : names [ ] = {
" 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 " ,
2018-07-05 15:08:51 +08:00
" writing_coordinated_state " , " accepting_commits " , " all_logs_recruited " , " storage_recovered " , " fully_recovered "
2017-05-26 04:48:44 +08:00
} ;
static_assert ( sizeof ( RecoveryStatus : : names ) = = sizeof ( RecoveryStatus : : names [ 0 ] ) * RecoveryStatus : : END , " RecoveryStatus::names[] size " ) ;
const char * RecoveryStatus : : descriptions [ ] = {
// reading_coordinated_state
" Requesting information from coordination servers. Verify that a majority of coordination server processes are active. " ,
// locking_coordinated_state
" Locking coordination state. Verify that a majority of coordination server processes are active. " ,
// locking_old_transaction_servers
2017-11-03 01:47:51 +08:00
" Locking old transaction servers. Verify that at least one transaction server from the previous generation is running. " ,
2017-05-26 04:48:44 +08:00
// reading_transaction_system_state
" Recovering transaction server state. Verify that the transaction server processes are active. " ,
// configuration_missing
" There appears to be a database, but its configuration does not appear to be initialized. " ,
// configuration_never_created
" The coordinator(s) have no record of this database. Either the coordinator addresses are incorrect, the coordination state on those machines is missing, or no database has been created. " ,
// configuration_invalid
" The database configuration is invalid. Set a new, valid configuration to recover the database. " ,
// recruiting_transaction_servers
" Recruiting new transaction servers. " ,
// initializing_transaction_servers
" Initializing new transaction servers and recovering transaction logs. " ,
// recovery_transaction
" Performing recovery transaction. " ,
// writing_coordinated_state
" Writing coordinated state. Verify that a majority of coordination server processes are active. " ,
2018-07-05 15:08:51 +08:00
// accepting_commits
" Accepting commits. " ,
// all_logs_recruited
" Accepting commits. All logs recruited. " ,
// storage_recovered
" Accepting commits. All storage servers are reading from the new logs. " ,
2017-05-26 04:48:44 +08:00
// fully_recovered
2018-07-05 15:08:51 +08:00
" Recovery complete. "
2017-05-26 04:48:44 +08:00
} ;
static_assert ( sizeof ( RecoveryStatus : : descriptions ) = = sizeof ( RecoveryStatus : : descriptions [ 0 ] ) * RecoveryStatus : : END , " RecoveryStatus::descriptions[] size " ) ;
// From Ratekeeper.actor.cpp
extern int limitReasonEnd ;
extern const char * limitReasonName [ ] ;
extern const char * limitReasonDesc [ ] ;
2019-02-08 05:39:22 +08:00
struct WorkerEvents : std : : map < NetworkAddress , TraceEventFields > { } ;
typedef std : : map < std : : string , TraceEventFields > EventMap ;
2017-05-26 04:48:44 +08:00
2018-05-03 01:44:38 +08:00
ACTOR static Future < Optional < TraceEventFields > > latestEventOnWorker ( WorkerInterface worker , std : : string eventName ) {
2017-05-26 04:48:44 +08:00
try {
EventLogRequest req = eventName . size ( ) > 0 ? EventLogRequest ( Standalone < StringRef > ( eventName ) ) : EventLogRequest ( ) ;
2018-05-03 01:44:38 +08:00
ErrorOr < TraceEventFields > eventTrace = wait ( errorOr ( timeoutError ( worker . eventLogRequest . getReply ( req ) , 2.0 ) ) ) ;
2017-05-26 04:48:44 +08:00
if ( eventTrace . isError ( ) ) {
2018-05-03 01:44:38 +08:00
return Optional < TraceEventFields > ( ) ;
2017-05-26 04:48:44 +08:00
}
2018-05-03 01:44:38 +08:00
return eventTrace . get ( ) ;
2017-05-26 04:48:44 +08:00
}
catch ( Error & e ) {
if ( e . code ( ) = = error_code_actor_cancelled )
throw ;
2018-05-03 01:44:38 +08:00
return Optional < TraceEventFields > ( ) ;
2017-05-26 04:48:44 +08:00
}
}
2019-03-09 00:25:07 +08:00
ACTOR static Future < Optional < std : : pair < WorkerEvents , std : : set < std : : string > > > > latestEventOnWorkers ( std : : vector < WorkerDetails > workers , std : : string eventName ) {
2017-05-26 04:48:44 +08:00
try {
2018-05-03 01:44:38 +08:00
state vector < Future < ErrorOr < TraceEventFields > > > eventTraces ;
2017-05-26 04:48:44 +08:00
for ( int c = 0 ; c < workers . size ( ) ; c + + ) {
EventLogRequest req = eventName . size ( ) > 0 ? EventLogRequest ( Standalone < StringRef > ( eventName ) ) : EventLogRequest ( ) ;
2019-03-09 00:25:07 +08:00
eventTraces . push_back ( errorOr ( timeoutError ( workers [ c ] . interf . eventLogRequest . getReply ( req ) , 2.0 ) ) ) ;
2017-05-26 04:48:44 +08:00
}
2018-08-11 04:57:10 +08:00
wait ( waitForAll ( eventTraces ) ) ;
2017-05-26 04:48:44 +08:00
std : : set < std : : string > failed ;
WorkerEvents results ;
for ( int i = 0 ; i < eventTraces . size ( ) ; i + + ) {
2018-06-27 05:37:21 +08:00
const ErrorOr < TraceEventFields > & v = eventTraces [ i ] . get ( ) ;
2017-05-26 04:48:44 +08:00
if ( v . isError ( ) ) {
2019-03-09 00:25:07 +08:00
failed . insert ( workers [ i ] . interf . address ( ) . toString ( ) ) ;
results [ workers [ i ] . interf . address ( ) ] = TraceEventFields ( ) ;
2017-05-26 04:48:44 +08:00
}
else {
2019-03-09 00:25:07 +08:00
results [ workers [ i ] . interf . address ( ) ] = v . get ( ) ;
2017-05-26 04:48:44 +08:00
}
}
std : : pair < WorkerEvents , std : : set < std : : string > > val ;
val . first = results ;
val . second = failed ;
return val ;
}
catch ( Error & e ) {
ASSERT ( e . code ( ) = = error_code_actor_cancelled ) ; // All errors should be filtering through the errorOr actor above
throw ;
}
}
2019-03-09 00:25:07 +08:00
static Future < Optional < std : : pair < WorkerEvents , std : : set < std : : string > > > > latestErrorOnWorkers ( std : : vector < WorkerDetails > workers ) {
2017-05-26 04:48:44 +08:00
return latestEventOnWorkers ( workers , " " ) ;
}
2019-03-09 00:25:07 +08:00
static Optional < WorkerDetails > getWorker ( std : : vector < WorkerDetails > const & workers , NetworkAddress const & address ) {
2017-05-26 04:48:44 +08:00
try {
for ( int c = 0 ; c < workers . size ( ) ; c + + )
2019-03-09 00:25:07 +08:00
if ( address = = workers [ c ] . interf . address ( ) )
2017-05-26 04:48:44 +08:00
return workers [ c ] ;
2019-03-09 00:25:07 +08:00
return Optional < WorkerDetails > ( ) ;
2017-05-26 04:48:44 +08:00
}
2019-03-27 00:58:54 +08:00
catch ( Error & ) {
2019-03-09 00:25:07 +08:00
return Optional < WorkerDetails > ( ) ;
2017-05-26 04:48:44 +08:00
}
}
2019-03-09 00:25:07 +08:00
static Optional < WorkerDetails > getWorker ( std : : map < NetworkAddress , WorkerDetails > const & workersMap , NetworkAddress const & address ) {
2017-05-26 04:48:44 +08:00
auto itr = workersMap . find ( address ) ;
if ( itr = = workersMap . end ( ) ) {
2019-03-09 00:25:07 +08:00
return Optional < WorkerDetails > ( ) ;
2017-05-26 04:48:44 +08:00
}
return itr - > second ;
}
2018-08-31 16:21:24 +08:00
class StatusCounter {
2018-12-01 02:46:04 +08:00
public :
StatusCounter ( ) : hz ( 0 ) , roughness ( 0 ) , counter ( 0 ) { }
StatusCounter ( double hz , double roughness , int64_t counter ) : hz ( hz ) , roughness ( roughness ) , counter ( counter ) { }
StatusCounter ( const std : : string & parsableText ) {
parseText ( parsableText ) ;
}
2017-05-26 04:48:44 +08:00
2018-12-01 02:46:04 +08:00
StatusCounter & parseText ( const std : : string & parsableText ) {
2019-05-05 01:52:02 +08:00
sscanf ( parsableText . c_str ( ) , " %lf %lf % " SCNd64 " " , & hz , & roughness , & counter ) ;
2018-12-01 02:46:04 +08:00
return * this ;
}
2017-05-26 04:48:44 +08:00
2018-12-01 02:46:04 +08:00
StatusCounter & updateValues ( const StatusCounter & statusCounter ) {
double hzNew = hz + statusCounter . hz ;
double roughnessNew = ( hz + statusCounter . hz ) ? ( roughness * hz + statusCounter . roughness * statusCounter . hz ) / ( hz + statusCounter . hz ) : 0.0 ;
int64_t counterNew = counter + statusCounter . counter ;
hz = hzNew ;
roughness = roughnessNew ;
counter = counterNew ;
return * this ;
}
2018-08-31 16:21:24 +08:00
2018-12-01 02:46:04 +08:00
JsonBuilderObject getStatus ( ) const {
JsonBuilderObject statusObject ;
statusObject [ " hz " ] = hz ;
statusObject [ " roughness " ] = roughness ;
statusObject [ " counter " ] = counter ;
return statusObject ;
}
2018-08-31 16:21:24 +08:00
2018-12-01 02:46:04 +08:00
double getHz ( ) {
return hz ;
}
double getRoughness ( ) {
return roughness ;
}
int64_t getCounter ( ) {
return counter ;
}
protected :
double hz ;
double roughness ;
int64_t counter ;
2018-08-31 16:21:24 +08:00
} ;
2017-05-26 04:48:44 +08:00
2018-09-09 06:44:48 +08:00
static JsonBuilderObject getLocalityInfo ( const LocalityData & locality ) {
JsonBuilderObject localityObj ;
2017-05-26 04:48:44 +08:00
for ( auto it = locality . _data . begin ( ) ; it ! = locality . _data . end ( ) ; it + + ) {
if ( it - > second . present ( ) ) {
2018-09-10 13:47:12 +08:00
localityObj [ it - > first ] = it - > second . get ( ) ;
2017-05-26 04:48:44 +08:00
}
else {
2018-09-10 13:47:12 +08:00
localityObj [ it - > first ] = JsonBuilder ( ) ;
2017-05-26 04:48:44 +08:00
}
}
return localityObj ;
}
2018-09-09 06:44:48 +08:00
static JsonBuilderObject getError ( const TraceEventFields & errorFields ) {
JsonBuilderObject statusObj ;
2017-05-26 04:48:44 +08:00
try {
2018-05-03 01:44:38 +08:00
if ( errorFields . size ( ) ) {
double time = atof ( errorFields . getValue ( " Time " ) . c_str ( ) ) ;
2017-05-26 04:48:44 +08:00
statusObj [ " time " ] = time ;
2018-05-03 01:44:38 +08:00
statusObj [ " raw_log_message " ] = errorFields . toString ( ) ;
2017-05-26 04:48:44 +08:00
2018-05-03 01:44:38 +08:00
std : : string type = errorFields . getValue ( " Type " ) ;
2017-05-26 04:48:44 +08:00
statusObj [ " type " ] = type ;
std : : string description = type ;
std : : string errorName ;
2018-05-03 01:44:38 +08:00
if ( errorFields . tryGetValue ( " Error " , errorName ) ) {
2017-05-26 04:48:44 +08:00
statusObj [ " name " ] = errorName ;
description + = " : " + errorName ;
}
else
statusObj [ " name " ] = " process_error " ;
struct tm * timeinfo ;
time_t t = ( time_t ) time ;
timeinfo = localtime ( & t ) ;
char buffer [ 128 ] ;
strftime ( buffer , 128 , " %c " , timeinfo ) ;
description + = " at " + std : : string ( buffer ) ;
statusObj [ " description " ] = description ;
}
}
catch ( Error & e ) {
2018-05-03 01:44:38 +08:00
TraceEvent ( SevError , " StatusGetErrorError " ) . error ( e ) . detail ( " RawError " , errorFields . toString ( ) ) ;
2017-05-26 04:48:44 +08:00
}
return statusObj ;
}
2019-03-09 00:25:07 +08:00
static JsonBuilderObject machineStatusFetcher ( WorkerEvents mMetrics , vector < WorkerDetails > workers , Optional < DatabaseConfiguration > configuration , std : : set < std : : string > * incomplete_reasons ) {
2018-09-09 06:44:48 +08:00
JsonBuilderObject machineMap ;
2017-05-26 04:48:44 +08:00
double metric ;
int failed = 0 ;
// map from machine networkAddress to datacenter ID
2018-05-03 01:44:38 +08:00
std : : map < NetworkAddress , std : : string > dcIds ;
2017-05-26 04:48:44 +08:00
std : : map < NetworkAddress , LocalityData > locality ;
2018-08-31 16:21:24 +08:00
std : : map < std : : string , bool > notExcludedMap ;
std : : map < std : : string , int32_t > workerContribMap ;
2018-09-09 06:44:48 +08:00
std : : map < std : : string , JsonBuilderObject > machineJsonMap ;
2017-05-26 04:48:44 +08:00
2019-02-08 05:39:22 +08:00
for ( auto const & worker : workers ) {
2019-03-09 00:25:07 +08:00
locality [ worker . interf . address ( ) ] = worker . interf . locality ;
if ( worker . interf . locality . dcId ( ) . present ( ) )
dcIds [ worker . interf . address ( ) ] = worker . interf . locality . dcId ( ) . get ( ) . printable ( ) ;
2017-05-26 04:48:44 +08:00
}
for ( auto it = mMetrics . begin ( ) ; it ! = mMetrics . end ( ) ; it + + ) {
if ( ! it - > second . size ( ) ) {
continue ;
}
2018-09-09 06:44:48 +08:00
JsonBuilderObject statusObj ; // Represents the status for a machine
2018-06-27 05:37:21 +08:00
const TraceEventFields & event = it - > second ;
2017-05-26 04:48:44 +08:00
try {
2019-02-27 10:04:03 +08:00
std : : string address = it - > first . ip . toString ( ) ;
2017-05-26 04:48:44 +08:00
// We will use the "physical" caluculated machine ID here to limit exposure to machineID repurposing
2018-05-03 01:44:38 +08:00
std : : string machineId = event . getValue ( " MachineID " ) ;
2017-05-26 04:48:44 +08:00
// If this machine ID does not already exist in the machineMap, add it
2018-09-06 13:51:15 +08:00
if ( machineJsonMap . count ( machineId ) = = 0 ) {
2017-05-26 04:48:44 +08:00
statusObj [ " machine_id " ] = machineId ;
if ( dcIds . count ( it - > first ) ) {
statusObj [ " datacenter_id " ] = dcIds [ it - > first ] ;
}
if ( locality . count ( it - > first ) ) {
statusObj [ " locality " ] = getLocalityInfo ( locality [ it - > first ] ) ;
}
statusObj [ " address " ] = address ;
2018-09-09 06:44:48 +08:00
JsonBuilderObject memoryObj ;
2018-09-10 14:19:00 +08:00
memoryObj . setKeyRawNumber ( " total_bytes " , event . getValue ( " TotalMemory " ) ) ;
memoryObj . setKeyRawNumber ( " committed_bytes " , event . getValue ( " CommittedMemory " ) ) ;
memoryObj . setKeyRawNumber ( " free_bytes " , event . getValue ( " AvailableMemory " ) ) ;
2017-05-26 04:48:44 +08:00
statusObj [ " memory " ] = memoryObj ;
2018-09-09 06:44:48 +08:00
JsonBuilderObject cpuObj ;
2019-06-27 05:03:02 +08:00
double cpuSeconds = event . getDouble ( " CPUSeconds " ) ;
2019-03-01 01:53:16 +08:00
double elapsed = event . getDouble ( " Elapsed " ) ;
2017-05-26 04:48:44 +08:00
if ( elapsed > 0 ) {
2019-06-27 05:03:02 +08:00
cpuObj [ " logical_core_utilization " ] = std : : max ( 0.0 , std : : min ( cpuSeconds / elapsed , 1.0 ) ) ;
2017-05-26 04:48:44 +08:00
}
statusObj [ " cpu " ] = cpuObj ;
2018-09-09 06:44:48 +08:00
JsonBuilderObject networkObj ;
2018-09-10 14:19:00 +08:00
networkObj [ " megabits_sent " ] = JsonBuilderObject ( ) . setKeyRawNumber ( " hz " , event . getValue ( " MbpsSent " ) ) ;
networkObj [ " megabits_received " ] = JsonBuilderObject ( ) . setKeyRawNumber ( " hz " , event . getValue ( " MbpsReceived " ) ) ;
2017-05-26 04:48:44 +08:00
2019-03-01 01:53:16 +08:00
metric = event . getDouble ( " RetransSegs " ) ;
2018-09-09 06:44:48 +08:00
JsonBuilderObject retransSegsObj ;
2017-05-26 04:48:44 +08:00
if ( elapsed > 0 ) {
retransSegsObj [ " hz " ] = metric / elapsed ;
}
networkObj [ " tcp_segments_retransmitted " ] = retransSegsObj ;
statusObj [ " network " ] = networkObj ;
if ( configuration . present ( ) ) {
2018-08-31 16:21:24 +08:00
notExcludedMap [ machineId ] = true ; // Will be set to false below if this or any later process is not excluded
2017-05-26 04:48:44 +08:00
}
2018-08-31 16:21:24 +08:00
workerContribMap [ machineId ] = 0 ;
machineJsonMap [ machineId ] = statusObj ;
2017-05-26 04:48:44 +08:00
}
2018-08-31 16:21:24 +08:00
if ( configuration . present ( ) & & ! configuration . get ( ) . isExcludedServer ( it - > first ) )
notExcludedMap [ machineId ] = false ;
workerContribMap [ machineId ] + + ;
2017-05-26 04:48:44 +08:00
}
2019-03-27 00:58:54 +08:00
catch ( Error & ) {
2017-05-26 04:48:44 +08:00
+ + failed ;
}
}
2018-08-31 16:21:24 +08:00
// Add the status json for each machine with tracked values
2018-09-06 03:22:04 +08:00
for ( auto & mapPair : machineJsonMap ) {
2018-08-31 16:21:24 +08:00
auto & machineId = mapPair . first ;
2018-09-06 03:22:04 +08:00
auto & jsonItem = machineJsonMap [ machineId ] ;
jsonItem [ " excluded " ] = notExcludedMap [ machineId ] ;
jsonItem [ " contributing_workers " ] = workerContribMap [ machineId ] ;
machineMap [ machineId ] = jsonItem ;
2018-08-31 16:21:24 +08:00
}
2017-05-26 04:48:44 +08:00
if ( failed > 0 )
incomplete_reasons - > insert ( " Cannot retrieve all machine status information. " ) ;
return machineMap ;
}
2019-08-16 04:42:39 +08:00
JsonBuilderObject getLagObject ( int64_t versions ) {
JsonBuilderObject lag ;
lag [ " versions " ] = versions ;
lag [ " seconds " ] = versions / ( double ) SERVER_KNOBS - > VERSIONS_PER_SECOND ;
return lag ;
}
2017-05-26 04:48:44 +08:00
struct MachineMemoryInfo {
double memoryUsage ;
double numProcesses ;
MachineMemoryInfo ( ) : memoryUsage ( 0 ) , numProcesses ( 0 ) { }
bool valid ( ) { return memoryUsage > = 0 ; }
void invalidate ( ) { memoryUsage = - 1 ; }
} ;
struct RolesInfo {
2018-09-09 06:44:48 +08:00
std : : multimap < NetworkAddress , JsonBuilderObject > roles ;
2019-02-08 05:39:22 +08:00
JsonBuilderObject addLatencyBandInfo ( TraceEventFields const & metrics ) {
JsonBuilderObject latency ;
std : : map < std : : string , JsonBuilderObject > bands ;
for ( auto itr = metrics . begin ( ) ; itr ! = metrics . end ( ) ; + + itr ) {
std : : string band ;
if ( itr - > first . substr ( 0 , 4 ) = = " Band " ) {
band = itr - > first . substr ( 4 ) ;
}
else if ( itr - > first = = " Filtered " ) {
band = " filtered " ;
}
else {
continue ;
}
latency [ band ] = StatusCounter ( itr - > second ) . getCounter ( ) ;
}
return latency ;
}
2018-09-09 06:44:48 +08:00
JsonBuilderObject & addRole ( NetworkAddress address , std : : string const & role , UID id ) {
JsonBuilderObject obj ;
2017-05-26 04:48:44 +08:00
obj [ " id " ] = id . shortString ( ) ;
obj [ " role " ] = role ;
2018-08-31 16:21:24 +08:00
return roles . insert ( std : : make_pair ( address , obj ) ) - > second ;
2017-05-26 04:48:44 +08:00
}
2019-02-08 05:39:22 +08:00
JsonBuilderObject & addRole ( std : : string const & role , StorageServerInterface & iface , EventMap const & metrics , Version maxTLogVersion , double * pDataLagSeconds ) {
2018-09-09 06:44:48 +08:00
JsonBuilderObject obj ;
2018-08-31 16:21:24 +08:00
double dataLagSeconds = - 1.0 ;
2017-05-26 04:48:44 +08:00
obj [ " id " ] = iface . id ( ) . shortString ( ) ;
obj [ " role " ] = role ;
try {
2019-02-08 05:39:22 +08:00
TraceEventFields const & storageMetrics = metrics . at ( " StorageMetrics " ) ;
obj . setKeyRawNumber ( " stored_bytes " , storageMetrics . getValue ( " BytesStored " ) ) ;
obj . setKeyRawNumber ( " kvstore_used_bytes " , storageMetrics . getValue ( " KvstoreBytesUsed " ) ) ;
obj . setKeyRawNumber ( " kvstore_free_bytes " , storageMetrics . getValue ( " KvstoreBytesFree " ) ) ;
obj . setKeyRawNumber ( " kvstore_available_bytes " , storageMetrics . getValue ( " KvstoreBytesAvailable " ) ) ;
obj . setKeyRawNumber ( " kvstore_total_bytes " , storageMetrics . getValue ( " KvstoreBytesTotal " ) ) ;
2019-01-10 10:03:54 +08:00
obj . setKeyRawNumber ( " kvstore_total_size " , storageMetrics . getValue ( " KvstoreSizeTotal " ) ) ;
obj . setKeyRawNumber ( " kvstore_total_nodes " , storageMetrics . getValue ( " KvstoreNodeTotal " ) ) ;
obj . setKeyRawNumber ( " kvstore_inline_keys " , storageMetrics . getValue ( " KvstoreInlineKey " ) ) ;
2019-02-08 05:39:22 +08:00
obj [ " input_bytes " ] = StatusCounter ( storageMetrics . getValue ( " BytesInput " ) ) . getStatus ( ) ;
obj [ " durable_bytes " ] = StatusCounter ( storageMetrics . getValue ( " BytesDurable " ) ) . getStatus ( ) ;
obj . setKeyRawNumber ( " query_queue_max " , storageMetrics . getValue ( " QueryQueueMax " ) ) ;
obj [ " total_queries " ] = StatusCounter ( storageMetrics . getValue ( " QueryQueue " ) ) . getStatus ( ) ;
obj [ " finished_queries " ] = StatusCounter ( storageMetrics . getValue ( " FinishedQueries " ) ) . getStatus ( ) ;
obj [ " bytes_queried " ] = StatusCounter ( storageMetrics . getValue ( " BytesQueried " ) ) . getStatus ( ) ;
obj [ " keys_queried " ] = StatusCounter ( storageMetrics . getValue ( " RowsQueried " ) ) . getStatus ( ) ;
obj [ " mutation_bytes " ] = StatusCounter ( storageMetrics . getValue ( " MutationBytes " ) ) . getStatus ( ) ;
obj [ " mutations " ] = StatusCounter ( storageMetrics . getValue ( " Mutations " ) ) . getStatus ( ) ;
2019-06-16 06:02:43 +08:00
obj . setKeyRawNumber ( " local_rate " , storageMetrics . getValue ( " LocalRate " ) ) ;
2019-02-08 05:39:22 +08:00
2019-03-01 01:53:16 +08:00
Version version = storageMetrics . getInt64 ( " Version " ) ;
Version durableVersion = storageMetrics . getInt64 ( " DurableVersion " ) ;
2018-09-28 05:33:21 +08:00
2017-05-26 04:48:44 +08:00
obj [ " data_version " ] = version ;
2018-09-28 05:33:21 +08:00
obj [ " durable_version " ] = durableVersion ;
2017-05-26 04:48:44 +08:00
2019-03-01 01:53:16 +08:00
int64_t versionLag = storageMetrics . getInt64 ( " VersionLag " ) ;
2017-05-26 04:48:44 +08:00
if ( maxTLogVersion > 0 ) {
2018-06-22 06:59:43 +08:00
// It's possible that the storage server hasn't talked to the logs recently, in which case it may not be aware of how far behind it is.
// To account for that, we also compute the version difference between each storage server and the tlog with the largest version.
//
// Because this data is only logged periodically, this difference will likely be an overestimate for the lag. We subtract off the logging interval
// in order to make this estimate a bounded underestimate instead.
versionLag = std : : max < int64_t > ( versionLag , maxTLogVersion - version - SERVER_KNOBS - > STORAGE_LOGGING_DELAY * SERVER_KNOBS - > VERSIONS_PER_SECOND ) ;
2017-05-26 04:48:44 +08:00
}
2019-02-08 05:39:22 +08:00
TraceEventFields const & readLatencyMetrics = metrics . at ( " ReadLatencyMetrics " ) ;
if ( readLatencyMetrics . size ( ) ) {
obj [ " read_latency_bands " ] = addLatencyBandInfo ( readLatencyMetrics ) ;
}
2019-08-16 04:42:39 +08:00
obj [ " data_lag " ] = getLagObject ( versionLag ) ;
obj [ " durability_lag " ] = getLagObject ( version - durableVersion ) ;
2018-06-22 06:59:43 +08:00
2017-05-26 04:48:44 +08:00
} catch ( Error & e ) {
if ( e . code ( ) ! = error_code_attribute_not_found )
throw e ;
}
2018-12-01 02:46:04 +08:00
if ( pDataLagSeconds ) {
2018-08-31 16:21:24 +08:00
* pDataLagSeconds = dataLagSeconds ;
2018-12-01 02:46:04 +08:00
}
2018-08-31 16:21:24 +08:00
return roles . insert ( std : : make_pair ( iface . address ( ) , obj ) ) - > second ;
2017-05-26 04:48:44 +08:00
}
2019-02-08 05:39:22 +08:00
JsonBuilderObject & addRole ( std : : string const & role , TLogInterface & iface , EventMap const & metrics , Version * pMetricVersion ) {
2018-09-09 06:44:48 +08:00
JsonBuilderObject obj ;
2018-08-31 16:21:24 +08:00
Version metricVersion = 0 ;
2017-05-26 04:48:44 +08:00
obj [ " id " ] = iface . id ( ) . shortString ( ) ;
obj [ " role " ] = role ;
try {
2019-02-08 05:39:22 +08:00
TraceEventFields const & tlogMetrics = metrics . at ( " TLogMetrics " ) ;
obj . setKeyRawNumber ( " kvstore_used_bytes " , tlogMetrics . getValue ( " KvstoreBytesUsed " ) ) ;
obj . setKeyRawNumber ( " kvstore_free_bytes " , tlogMetrics . getValue ( " KvstoreBytesFree " ) ) ;
obj . setKeyRawNumber ( " kvstore_available_bytes " , tlogMetrics . getValue ( " KvstoreBytesAvailable " ) ) ;
obj . setKeyRawNumber ( " kvstore_total_bytes " , tlogMetrics . getValue ( " KvstoreBytesTotal " ) ) ;
obj . setKeyRawNumber ( " queue_disk_used_bytes " , tlogMetrics . getValue ( " QueueDiskBytesUsed " ) ) ;
obj . setKeyRawNumber ( " queue_disk_free_bytes " , tlogMetrics . getValue ( " QueueDiskBytesFree " ) ) ;
obj . setKeyRawNumber ( " queue_disk_available_bytes " , tlogMetrics . getValue ( " QueueDiskBytesAvailable " ) ) ;
obj . setKeyRawNumber ( " queue_disk_total_bytes " , tlogMetrics . getValue ( " QueueDiskBytesTotal " ) ) ;
obj [ " input_bytes " ] = StatusCounter ( tlogMetrics . getValue ( " BytesInput " ) ) . getStatus ( ) ;
obj [ " durable_bytes " ] = StatusCounter ( tlogMetrics . getValue ( " BytesDurable " ) ) . getStatus ( ) ;
2019-03-01 01:53:16 +08:00
metricVersion = tlogMetrics . getInt64 ( " Version " ) ;
2018-08-31 16:21:24 +08:00
obj [ " data_version " ] = metricVersion ;
2017-05-26 04:48:44 +08:00
} catch ( Error & e ) {
if ( e . code ( ) ! = error_code_attribute_not_found )
throw e ;
}
2018-08-31 16:21:24 +08:00
if ( pMetricVersion )
* pMetricVersion = metricVersion ;
return roles . insert ( std : : make_pair ( iface . address ( ) , obj ) ) - > second ;
2017-05-26 04:48:44 +08:00
}
2019-02-08 05:39:22 +08:00
JsonBuilderObject & addRole ( std : : string const & role , MasterProxyInterface & iface , EventMap const & metrics ) {
2018-12-01 02:46:04 +08:00
JsonBuilderObject obj ;
obj [ " id " ] = iface . id ( ) . shortString ( ) ;
obj [ " role " ] = role ;
try {
2019-02-08 05:39:22 +08:00
TraceEventFields const & grvLatencyMetrics = metrics . at ( " GRVLatencyMetrics " ) ;
if ( grvLatencyMetrics . size ( ) ) {
obj [ " grv_latency_bands " ] = addLatencyBandInfo ( grvLatencyMetrics ) ;
2018-12-01 02:46:04 +08:00
}
2019-02-08 05:39:22 +08:00
TraceEventFields const & commitLatencyMetrics = metrics . at ( " CommitLatencyMetrics " ) ;
if ( commitLatencyMetrics . size ( ) ) {
obj [ " commit_latency_bands " ] = addLatencyBandInfo ( commitLatencyMetrics ) ;
2019-01-19 08:18:34 +08:00
}
2018-12-01 02:46:04 +08:00
} catch ( Error & e ) {
if ( e . code ( ) ! = error_code_attribute_not_found ) {
throw e ;
}
}
return roles . insert ( std : : make_pair ( iface . address ( ) , obj ) ) - > second ;
}
2017-05-26 04:48:44 +08:00
template < class InterfaceType >
2018-09-09 06:44:48 +08:00
JsonBuilderObject & addRole ( std : : string const & role , InterfaceType & iface ) {
2017-05-26 04:48:44 +08:00
return addRole ( iface . address ( ) , role , iface . id ( ) ) ;
}
2019-08-16 05:42:49 +08:00
JsonBuilderObject & addCoordinatorRole ( NetworkAddress addr ) {
JsonBuilderObject obj ;
obj [ " role " ] = " coordinator " ;
return roles . insert ( std : : make_pair ( addr , obj ) ) - > second ;
}
2018-09-09 06:44:48 +08:00
JsonBuilderArray getStatusForAddress ( NetworkAddress a ) {
JsonBuilderArray v ;
2017-05-26 04:48:44 +08:00
auto it = roles . lower_bound ( a ) ;
while ( it ! = roles . end ( ) & & it - > first = = a ) {
v . push_back ( it - > second ) ;
+ + it ;
}
return v ;
}
} ;
2018-09-09 06:44:48 +08:00
ACTOR static Future < JsonBuilderObject > processStatusFetcher (
2019-09-11 05:27:22 +08:00
Reference < AsyncVar < CachedSerialization < ServerDBInfo > > > db , std : : vector < WorkerDetails > workers , WorkerEvents pMetrics ,
2019-07-12 05:53:00 +08:00
WorkerEvents mMetrics , WorkerEvents nMetrics , WorkerEvents errors , WorkerEvents traceFileOpenErrors ,
2019-08-16 05:42:49 +08:00
WorkerEvents programStarts , std : : map < std : : string , std : : vector < JsonBuilderObject > > processIssues ,
2019-03-22 23:29:41 +08:00
vector < std : : pair < StorageServerInterface , EventMap > > storageServers ,
vector < std : : pair < TLogInterface , EventMap > > tLogs , vector < std : : pair < MasterProxyInterface , EventMap > > proxies ,
2020-02-05 02:26:18 +08:00
ServerCoordinators coordinators , Database cx , Optional < DatabaseConfiguration > configuration ,
2019-08-16 05:42:49 +08:00
Optional < Key > healthyZone , std : : set < std : : string > * incomplete_reasons ) {
2017-05-26 04:48:44 +08:00
2018-09-09 06:44:48 +08:00
state JsonBuilderObject processMap ;
2017-05-26 04:48:44 +08:00
// construct a map from a process address to a status object containing a trace file open error
// this is later added to the messages subsection
2018-09-09 06:44:48 +08:00
state std : : map < std : : string , JsonBuilderObject > tracefileOpenErrorMap ;
2017-05-26 04:48:44 +08:00
state WorkerEvents : : iterator traceFileErrorsItr ;
for ( traceFileErrorsItr = traceFileOpenErrors . begin ( ) ; traceFileErrorsItr ! = traceFileOpenErrors . end ( ) ; + + traceFileErrorsItr ) {
2018-08-11 04:57:10 +08:00
wait ( yield ( ) ) ;
2017-05-26 04:48:44 +08:00
if ( traceFileErrorsItr - > second . size ( ) ) {
try {
2018-05-03 01:44:38 +08:00
// Have event fields, parse it and turn it into a message object describing the trace file opening error
2018-06-27 05:37:21 +08:00
const TraceEventFields & event = traceFileErrorsItr - > second ;
2018-05-03 01:44:38 +08:00
std : : string fileName = event . getValue ( " Filename " ) ;
2018-09-09 06:44:48 +08:00
JsonBuilderObject msgObj = JsonString : : makeMessage ( " file_open_error " , format ( " Could not open file '%s' (%s). " , fileName . c_str ( ) , event . getValue ( " Error " ) . c_str ( ) ) . c_str ( ) ) ;
2017-05-26 04:48:44 +08:00
msgObj [ " file_name " ] = fileName ;
// Map the address of the worker to the error message object
tracefileOpenErrorMap [ traceFileErrorsItr - > first . toString ( ) ] = msgObj ;
}
catch ( Error & e ) {
incomplete_reasons - > insert ( " file_open_error details could not be retrieved " ) ;
}
}
}
state std : : map < Optional < Standalone < StringRef > > , MachineMemoryInfo > machineMemoryUsage ;
2019-03-09 00:25:07 +08:00
state std : : vector < WorkerDetails > : : iterator workerItr ;
2017-05-26 04:48:44 +08:00
for ( workerItr = workers . begin ( ) ; workerItr ! = workers . end ( ) ; + + workerItr ) {
2018-08-11 04:57:10 +08:00
wait ( yield ( ) ) ;
2019-03-09 00:25:07 +08:00
state std : : map < Optional < Standalone < StringRef > > , MachineMemoryInfo > : : iterator memInfo = machineMemoryUsage . insert ( std : : make_pair ( workerItr - > interf . locality . machineId ( ) , MachineMemoryInfo ( ) ) ) . first ;
2017-05-26 04:48:44 +08:00
try {
2019-03-09 00:25:07 +08:00
ASSERT ( pMetrics . count ( workerItr - > interf . address ( ) ) ) ;
const TraceEventFields & processMetrics = pMetrics [ workerItr - > interf . address ( ) ] ;
2017-05-26 04:48:44 +08:00
if ( memInfo - > second . valid ( ) ) {
if ( processMetrics . size ( ) > 0 ) {
2019-03-01 01:53:16 +08:00
memInfo - > second . memoryUsage + = processMetrics . getDouble ( " Memory " ) ;
2017-05-26 04:48:44 +08:00
+ + memInfo - > second . numProcesses ;
}
else
memInfo - > second . invalidate ( ) ;
}
}
catch ( Error & e ) {
memInfo - > second . invalidate ( ) ;
}
}
state RolesInfo roles ;
2019-09-11 05:27:22 +08:00
roles . addRole ( " master " , db - > get ( ) . read ( ) . master ) ;
roles . addRole ( " cluster_controller " , db - > get ( ) . read ( ) . clusterInterface . clientInterface ) ;
2017-05-26 04:48:44 +08:00
2019-09-11 05:27:22 +08:00
if ( db - > get ( ) . read ( ) . distributor . present ( ) ) {
roles . addRole ( " data_distributor " , db - > get ( ) . read ( ) . distributor . get ( ) ) ;
2019-03-26 06:11:29 +08:00
}
2019-09-11 05:27:22 +08:00
if ( db - > get ( ) . read ( ) . ratekeeper . present ( ) ) {
roles . addRole ( " ratekeeper " , db - > get ( ) . read ( ) . ratekeeper . get ( ) ) ;
2019-03-26 06:11:29 +08:00
}
2019-09-11 05:27:22 +08:00
for ( auto & tLogSet : db - > get ( ) . read ( ) . logSystemConfig . tLogs ) {
2019-05-03 07:16:25 +08:00
for ( auto & it : tLogSet . logRouters ) {
if ( it . present ( ) ) {
roles . addRole ( " router " , it . interf ( ) ) ;
}
}
}
2019-09-11 05:27:22 +08:00
for ( auto & old : db - > get ( ) . read ( ) . logSystemConfig . oldTLogs ) {
2019-05-03 07:16:25 +08:00
for ( auto & tLogSet : old . tLogs ) {
for ( auto & it : tLogSet . logRouters ) {
if ( it . present ( ) ) {
roles . addRole ( " router " , it . interf ( ) ) ;
}
}
}
}
2019-08-16 05:42:49 +08:00
for ( auto & coordinator : coordinators . ccf - > getConnectionString ( ) . coordinators ( ) ) {
roles . addCoordinatorRole ( coordinator ) ;
}
2019-02-08 05:39:22 +08:00
state std : : vector < std : : pair < MasterProxyInterface , EventMap > > : : iterator proxy ;
2018-12-01 02:46:04 +08:00
for ( proxy = proxies . begin ( ) ; proxy ! = proxies . end ( ) ; + + proxy ) {
roles . addRole ( " proxy " , proxy - > first , proxy - > second ) ;
2019-01-25 03:43:26 +08:00
wait ( yield ( ) ) ;
2017-05-26 04:48:44 +08:00
}
2019-02-08 05:39:22 +08:00
state std : : vector < std : : pair < TLogInterface , EventMap > > : : iterator log ;
2017-05-26 04:48:44 +08:00
state Version maxTLogVersion = 0 ;
2018-08-31 16:21:24 +08:00
// Get largest TLog version
2017-05-26 04:48:44 +08:00
for ( log = tLogs . begin ( ) ; log ! = tLogs . end ( ) ; + + log ) {
2018-09-06 13:51:15 +08:00
Version tLogVersion = 0 ;
2018-09-09 06:44:48 +08:00
roles . addRole ( " log " , log - > first , log - > second , & tLogVersion ) ;
2018-09-06 13:51:15 +08:00
maxTLogVersion = std : : max ( maxTLogVersion , tLogVersion ) ;
2018-08-11 04:57:10 +08:00
wait ( yield ( ) ) ;
2017-05-26 04:48:44 +08:00
}
2019-02-08 05:39:22 +08:00
state std : : vector < std : : pair < StorageServerInterface , EventMap > > : : iterator ss ;
2018-08-08 02:02:09 +08:00
state std : : map < NetworkAddress , double > ssLag ;
2018-08-31 16:21:24 +08:00
state double lagSeconds ;
2017-05-26 04:48:44 +08:00
for ( ss = storageServers . begin ( ) ; ss ! = storageServers . end ( ) ; + + ss ) {
2018-09-09 06:44:48 +08:00
roles . addRole ( " storage " , ss - > first , ss - > second , maxTLogVersion , & lagSeconds ) ;
2018-08-31 16:21:24 +08:00
if ( lagSeconds ! = - 1.0 ) {
2018-08-08 02:02:09 +08:00
ssLag [ ss - > first . address ( ) ] = lagSeconds ;
2017-05-26 04:48:44 +08:00
}
2018-08-11 04:57:10 +08:00
wait ( yield ( ) ) ;
2017-05-26 04:48:44 +08:00
}
state std : : vector < ResolverInterface > : : const_iterator res ;
2019-09-11 05:27:22 +08:00
state std : : vector < ResolverInterface > resolvers = db - > get ( ) . read ( ) . resolvers ;
2017-05-26 04:48:44 +08:00
for ( res = resolvers . begin ( ) ; res ! = resolvers . end ( ) ; + + res ) {
roles . addRole ( " resolver " , * res ) ;
2018-08-11 04:57:10 +08:00
wait ( yield ( ) ) ;
2017-05-26 04:48:44 +08:00
}
for ( workerItr = workers . begin ( ) ; workerItr ! = workers . end ( ) ; + + workerItr ) {
2018-08-11 04:57:10 +08:00
wait ( yield ( ) ) ;
2018-09-09 06:44:48 +08:00
state JsonBuilderObject statusObj ;
2017-05-26 04:48:44 +08:00
try {
2019-03-09 00:25:07 +08:00
ASSERT ( pMetrics . count ( workerItr - > interf . address ( ) ) ) ;
2017-05-26 04:48:44 +08:00
2019-03-09 00:25:07 +08:00
NetworkAddress address = workerItr - > interf . address ( ) ;
2019-06-27 05:03:02 +08:00
const TraceEventFields & processMetrics = pMetrics [ workerItr - > interf . address ( ) ] ;
2017-05-26 04:48:44 +08:00
statusObj [ " address " ] = address . toString ( ) ;
2018-09-09 06:44:48 +08:00
JsonBuilderObject memoryObj ;
2017-05-26 04:48:44 +08:00
2019-06-27 05:03:02 +08:00
if ( processMetrics . size ( ) > 0 ) {
std : : string zoneID = processMetrics . getValue ( " ZoneID " ) ;
2017-05-26 04:48:44 +08:00
statusObj [ " fault_domain " ] = zoneID ;
2019-04-03 05:15:51 +08:00
if ( healthyZone . present ( ) & & healthyZone = = workerItr - > interf . locality . zoneId ( ) ) {
statusObj [ " under_maintenance " ] = true ;
}
2017-05-26 04:48:44 +08:00
2019-06-27 05:03:02 +08:00
std : : string MachineID = processMetrics . getValue ( " MachineID " ) ;
2017-05-26 04:48:44 +08:00
statusObj [ " machine_id " ] = MachineID ;
2019-03-09 00:25:07 +08:00
statusObj [ " locality " ] = getLocalityInfo ( workerItr - > interf . locality ) ;
2017-05-26 04:48:44 +08:00
2019-06-27 05:03:02 +08:00
statusObj . setKeyRawNumber ( " uptime_seconds " , processMetrics . getValue ( " UptimeSeconds " ) ) ;
2017-05-26 04:48:44 +08:00
// rates are calculated over the last elapsed seconds
2019-06-27 05:03:02 +08:00
double processMetricsElapsed = processMetrics . getDouble ( " Elapsed " ) ;
double cpuSeconds = processMetrics . getDouble ( " CPUSeconds " ) ;
double diskIdleSeconds = processMetrics . getDouble ( " DiskIdleSeconds " ) ;
double diskReads = processMetrics . getDouble ( " DiskReads " ) ;
double diskWrites = processMetrics . getDouble ( " DiskWrites " ) ;
2017-05-26 04:48:44 +08:00
2018-09-09 06:44:48 +08:00
JsonBuilderObject diskObj ;
2019-06-27 05:03:02 +08:00
if ( processMetricsElapsed > 0 ) {
2018-09-09 06:44:48 +08:00
JsonBuilderObject cpuObj ;
2019-06-27 05:03:02 +08:00
cpuObj [ " usage_cores " ] = std : : max ( 0.0 , cpuSeconds / processMetricsElapsed ) ;
2017-05-26 04:48:44 +08:00
statusObj [ " cpu " ] = cpuObj ;
2019-06-27 05:03:02 +08:00
diskObj [ " busy " ] = std : : max ( 0.0 , std : : min ( ( processMetricsElapsed - diskIdleSeconds ) / processMetricsElapsed , 1.0 ) ) ;
2017-05-26 04:48:44 +08:00
2018-09-09 06:44:48 +08:00
JsonBuilderObject readsObj ;
2019-06-27 05:03:02 +08:00
readsObj . setKeyRawNumber ( " counter " , processMetrics . getValue ( " DiskReadsCount " ) ) ;
if ( processMetricsElapsed > 0 )
readsObj [ " hz " ] = diskReads / processMetricsElapsed ;
readsObj . setKeyRawNumber ( " sectors " , processMetrics . getValue ( " DiskReadSectors " ) ) ;
2017-05-26 04:48:44 +08:00
2018-09-09 06:44:48 +08:00
JsonBuilderObject writesObj ;
2019-06-27 05:03:02 +08:00
writesObj . setKeyRawNumber ( " counter " , processMetrics . getValue ( " DiskWritesCount " ) ) ;
if ( processMetricsElapsed > 0 )
writesObj [ " hz " ] = diskWrites / processMetricsElapsed ;
writesObj . setKeyRawNumber ( " sectors " , processMetrics . getValue ( " DiskWriteSectors " ) ) ;
2017-05-26 04:48:44 +08:00
diskObj [ " reads " ] = readsObj ;
diskObj [ " writes " ] = writesObj ;
}
2019-06-27 05:03:02 +08:00
diskObj . setKeyRawNumber ( " total_bytes " , processMetrics . getValue ( " DiskTotalBytes " ) ) ;
diskObj . setKeyRawNumber ( " free_bytes " , processMetrics . getValue ( " DiskFreeBytes " ) ) ;
2017-05-26 04:48:44 +08:00
statusObj [ " disk " ] = diskObj ;
2018-09-09 06:44:48 +08:00
JsonBuilderObject networkObj ;
2017-05-26 04:48:44 +08:00
2019-06-27 05:03:02 +08:00
networkObj . setKeyRawNumber ( " current_connections " , processMetrics . getValue ( " CurrentConnections " ) ) ;
2018-09-09 06:44:48 +08:00
JsonBuilderObject connections_established ;
2019-06-27 05:03:02 +08:00
connections_established . setKeyRawNumber ( " hz " , processMetrics . getValue ( " ConnectionsEstablished " ) ) ;
2017-05-26 04:48:44 +08:00
networkObj [ " connections_established " ] = connections_established ;
2018-09-09 06:44:48 +08:00
JsonBuilderObject connections_closed ;
2019-06-27 05:03:02 +08:00
connections_closed . setKeyRawNumber ( " hz " , processMetrics . getValue ( " ConnectionsClosed " ) ) ;
2017-05-26 04:48:44 +08:00
networkObj [ " connections_closed " ] = connections_closed ;
2018-09-09 06:44:48 +08:00
JsonBuilderObject connection_errors ;
2019-06-27 05:03:02 +08:00
connection_errors . setKeyRawNumber ( " hz " , processMetrics . getValue ( " ConnectionErrors " ) ) ;
2017-05-26 04:48:44 +08:00
networkObj [ " connection_errors " ] = connection_errors ;
2018-09-09 06:44:48 +08:00
JsonBuilderObject megabits_sent ;
2019-06-27 05:03:02 +08:00
megabits_sent . setKeyRawNumber ( " hz " , processMetrics . getValue ( " MbpsSent " ) ) ;
2017-05-26 04:48:44 +08:00
networkObj [ " megabits_sent " ] = megabits_sent ;
2018-09-09 06:44:48 +08:00
JsonBuilderObject megabits_received ;
2019-06-27 05:03:02 +08:00
megabits_received . setKeyRawNumber ( " hz " , processMetrics . getValue ( " MbpsReceived " ) ) ;
2017-05-26 04:48:44 +08:00
networkObj [ " megabits_received " ] = megabits_received ;
statusObj [ " network " ] = networkObj ;
2019-06-27 05:03:02 +08:00
memoryObj . setKeyRawNumber ( " used_bytes " , processMetrics . getValue ( " Memory " ) ) ;
memoryObj . setKeyRawNumber ( " unused_allocated_memory " , processMetrics . getValue ( " UnusedAllocatedMemory " ) ) ;
2017-05-26 04:48:44 +08:00
}
if ( programStarts . count ( address ) ) {
auto const & psxml = programStarts . at ( address ) ;
2018-03-10 03:55:57 +08:00
if ( psxml . size ( ) > 0 ) {
2018-09-10 14:19:00 +08:00
memoryObj . setKeyRawNumber ( " limit_bytes " , psxml . getValue ( " MemoryLimit " ) ) ;
2018-03-10 03:55:57 +08:00
std : : string version ;
2018-05-03 01:44:38 +08:00
if ( psxml . tryGetValue ( " Version " , version ) ) {
2018-03-10 03:55:57 +08:00
statusObj [ " version " ] = version ;
}
std : : string commandLine ;
2018-05-03 01:44:38 +08:00
if ( psxml . tryGetValue ( " CommandLine " , commandLine ) ) {
2018-03-10 03:55:57 +08:00
statusObj [ " command_line " ] = commandLine ;
}
}
2017-05-26 04:48:44 +08:00
}
// if this process address is in the machine metrics
if ( mMetrics . count ( address ) & & mMetrics [ address ] . size ( ) ) {
double availableMemory ;
2019-03-01 01:53:16 +08:00
availableMemory = mMetrics [ address ] . getDouble ( " AvailableMemory " ) ;
2017-05-26 04:48:44 +08:00
2019-03-09 00:25:07 +08:00
auto machineMemInfo = machineMemoryUsage [ workerItr - > interf . locality . machineId ( ) ] ;
2017-05-26 04:48:44 +08:00
if ( machineMemInfo . valid ( ) ) {
ASSERT ( machineMemInfo . numProcesses > 0 ) ;
int64_t memory = ( availableMemory + machineMemInfo . memoryUsage ) / machineMemInfo . numProcesses ;
memoryObj [ " available_bytes " ] = std : : max < int64_t > ( memory , 0 ) ;
}
}
statusObj [ " memory " ] = memoryObj ;
2018-09-09 06:44:48 +08:00
JsonBuilderArray messages ;
2017-05-26 04:48:44 +08:00
2018-03-10 03:55:57 +08:00
if ( errors . count ( address ) & & errors [ address ] . size ( ) ) {
2017-05-26 04:48:44 +08:00
// returns status object with type and time of error
messages . push_back ( getError ( errors . at ( address ) ) ) ;
2018-03-10 03:55:57 +08:00
}
2017-05-26 04:48:44 +08:00
// string of address used so that other fields of a NetworkAddress are not compared
std : : string strAddress = address . toString ( ) ;
// If this process has a process issue, identified by strAddress, then add it to messages array
2019-03-22 23:29:41 +08:00
for ( auto issue : processIssues [ strAddress ] ) {
messages . push_back ( issue ) ;
2017-05-26 04:48:44 +08:00
}
// If this process had a trace file open error, identified by strAddress, then add it to messages array
if ( tracefileOpenErrorMap . count ( strAddress ) ) {
messages . push_back ( tracefileOpenErrorMap [ strAddress ] ) ;
}
2018-08-08 02:02:09 +08:00
if ( ssLag [ address ] > = 60 ) {
2018-08-31 16:21:24 +08:00
messages . push_back ( JsonString : : makeMessage ( " storage_server_lagging " , format ( " Storage server lagging by %ld seconds. " , ( int64_t ) ssLag [ address ] ) . c_str ( ) ) ) ;
2017-05-26 04:48:44 +08:00
}
// Store the message array into the status object that represents the worker process
statusObj [ " messages " ] = messages ;
// Get roles for the worker's address as an array of objects
statusObj [ " roles " ] = roles . getStatusForAddress ( address ) ;
if ( configuration . present ( ) ) {
statusObj [ " excluded " ] = configuration . get ( ) . isExcludedServer ( address ) ;
}
2019-03-09 00:25:07 +08:00
statusObj [ " class_type " ] = workerItr - > processClass . toString ( ) ;
statusObj [ " class_source " ] = workerItr - > processClass . sourceString ( ) ;
2019-03-11 13:58:15 +08:00
if ( workerItr - > degraded ) {
statusObj [ " degraded " ] = true ;
}
2019-06-27 05:03:02 +08:00
const TraceEventFields & networkMetrics = nMetrics [ workerItr - > interf . address ( ) ] ;
double networkMetricsElapsed = networkMetrics . getDouble ( " Elapsed " ) ;
try {
double runLoopBusy = networkMetrics . getDouble ( " PriorityBusy1 " ) ;
statusObj [ " run_loop_busy " ] = runLoopBusy / networkMetricsElapsed ;
}
catch ( Error & e ) {
// This should only happen very early in the process lifetime before priority bin info has been populated
incomplete_reasons - > insert ( " Cannot retrieve run loop busyness. " ) ;
}
2017-05-26 04:48:44 +08:00
}
catch ( Error & e ) {
// Something strange occurred, process list is incomplete but what was built so far, if anything, will be returned.
incomplete_reasons - > insert ( " Cannot retrieve all process status information. " ) ;
}
2019-03-09 00:25:07 +08:00
processMap [ printable ( workerItr - > interf . locality . processId ( ) ) ] = statusObj ;
2017-05-26 04:48:44 +08:00
}
return processMap ;
}
2019-07-26 08:15:31 +08:00
struct ClientStats {
int count ;
std : : set < std : : pair < NetworkAddress , Key > > examples ;
2017-05-26 04:48:44 +08:00
2019-07-26 08:15:31 +08:00
ClientStats ( ) : count ( 0 ) { }
} ;
static JsonBuilderObject clientStatusFetcher ( std : : map < NetworkAddress , std : : pair < double , OpenDatabaseRequest > > * clientStatusMap ) {
JsonBuilderObject clientStatus ;
2017-05-26 04:48:44 +08:00
2019-07-26 08:15:31 +08:00
int64_t clientCount = 0 ;
std : : map < Key , ClientStats > issues ;
std : : map < Standalone < ClientVersionRef > , ClientStats > supportedVersions ;
std : : map < Key , ClientStats > maxSupportedProtocol ;
2019-09-06 01:20:50 +08:00
for ( auto iter = clientStatusMap - > begin ( ) ; iter ! = clientStatusMap - > end ( ) ; ) {
if ( now ( ) - iter - > second . first < 2 * SERVER_KNOBS - > COORDINATOR_REGISTER_INTERVAL ) {
2019-07-26 08:15:31 +08:00
clientCount + = iter - > second . second . clientCount ;
for ( auto & it : iter - > second . second . issues ) {
auto & issue = issues [ it . item ] ;
issue . count + = it . count ;
issue . examples . insert ( it . examples . begin ( ) , it . examples . end ( ) ) ;
}
for ( auto & it : iter - > second . second . supportedVersions ) {
auto & version = supportedVersions [ it . item ] ;
version . count + = it . count ;
version . examples . insert ( it . examples . begin ( ) , it . examples . end ( ) ) ;
}
for ( auto & it : iter - > second . second . maxProtocolSupported ) {
auto & protocolVersion = maxSupportedProtocol [ it . item ] ;
protocolVersion . count + = it . count ;
protocolVersion . examples . insert ( it . examples . begin ( ) , it . examples . end ( ) ) ;
}
2019-09-06 01:20:50 +08:00
+ + iter ;
2019-07-26 08:15:31 +08:00
} else {
iter = clientStatusMap - > erase ( iter ) ;
2017-05-26 04:48:44 +08:00
}
}
2019-07-26 08:15:31 +08:00
clientStatus [ " count " ] = clientCount ;
2018-09-09 06:44:48 +08:00
JsonBuilderArray versionsArray = JsonBuilderArray ( ) ;
2019-07-26 08:15:31 +08:00
for ( auto & cv : supportedVersions ) {
2018-09-09 06:44:48 +08:00
JsonBuilderObject ver ;
2019-07-26 08:15:31 +08:00
ver [ " count " ] = ( int64_t ) cv . second . count ;
2017-05-26 04:48:44 +08:00
ver [ " client_version " ] = cv . first . clientVersion . toString ( ) ;
ver [ " protocol_version " ] = cv . first . protocolVersion . toString ( ) ;
ver [ " source_version " ] = cv . first . sourceVersion . toString ( ) ;
2018-09-09 06:44:48 +08:00
JsonBuilderArray clients = JsonBuilderArray ( ) ;
2019-07-26 08:15:31 +08:00
for ( auto & client : cv . second . examples ) {
2018-09-09 06:44:48 +08:00
JsonBuilderObject cli ;
2019-07-26 08:15:31 +08:00
cli [ " address " ] = client . first . toString ( ) ;
cli [ " log_group " ] = client . second . toString ( ) ;
2017-09-29 07:31:29 +08:00
clients . push_back ( cli ) ;
2017-05-26 04:48:44 +08:00
}
2019-07-26 08:15:31 +08:00
auto iter = maxSupportedProtocol . find ( cv . first . protocolVersion ) ;
if ( iter ! = maxSupportedProtocol . end ( ) ) {
JsonBuilderArray maxClients = JsonBuilderArray ( ) ;
for ( auto & client : iter - > second . examples ) {
JsonBuilderObject cli ;
cli [ " address " ] = client . first . toString ( ) ;
cli [ " log_group " ] = client . second . toString ( ) ;
2019-08-14 06:54:40 +08:00
maxClients . push_back ( cli ) ;
2019-07-26 08:15:31 +08:00
}
ver [ " max_protocol_count " ] = iter - > second . count ;
ver [ " max_protocol_clients " ] = maxClients ;
2019-07-27 04:23:56 +08:00
maxSupportedProtocol . erase ( cv . first . protocolVersion ) ;
2019-07-26 08:15:31 +08:00
}
2017-11-02 01:41:30 +08:00
ver [ " connected_clients " ] = clients ;
2017-05-26 04:48:44 +08:00
versionsArray . push_back ( ver ) ;
}
if ( versionsArray . size ( ) > 0 ) {
clientStatus [ " supported_versions " ] = versionsArray ;
}
return clientStatus ;
}
2019-03-09 00:25:07 +08:00
ACTOR static Future < JsonBuilderObject > recoveryStateStatusFetcher ( WorkerDetails mWorker , int workerCount , std : : set < std : : string > * incomplete_reasons , int * statusCode ) {
2018-09-09 06:44:48 +08:00
state JsonBuilderObject message ;
2017-05-26 04:48:44 +08:00
try {
2019-03-09 00:25:07 +08:00
TraceEventFields md = wait ( timeoutError ( mWorker . interf . eventLogRequest . getReply ( EventLogRequest ( LiteralStringRef ( " MasterRecoveryState " ) ) ) , 1.0 ) ) ;
2019-03-01 01:53:16 +08:00
state int mStatusCode = md . getInt ( " StatusCode " ) ;
2017-05-26 04:48:44 +08:00
if ( mStatusCode < 0 | | mStatusCode > = RecoveryStatus : : END )
throw attribute_not_found ( ) ;
2018-08-31 16:21:24 +08:00
message = JsonString : : makeMessage ( RecoveryStatus : : names [ mStatusCode ] , RecoveryStatus : : descriptions [ mStatusCode ] ) ;
* statusCode = mStatusCode ;
2017-05-26 04:48:44 +08:00
// Add additional metadata for certain statuses
if ( mStatusCode = = RecoveryStatus : : recruiting_transaction_servers ) {
2018-05-03 01:44:38 +08:00
int requiredLogs = atoi ( md . getValue ( " RequiredTLogs " ) . c_str ( ) ) ;
int requiredProxies = atoi ( md . getValue ( " RequiredProxies " ) . c_str ( ) ) ;
int requiredResolvers = atoi ( md . getValue ( " RequiredResolvers " ) . c_str ( ) ) ;
2017-05-26 04:48:44 +08:00
//int requiredProcesses = std::max(requiredLogs, std::max(requiredResolvers, requiredProxies));
//int requiredMachines = std::max(requiredLogs, 1);
message [ " required_logs " ] = requiredLogs ;
message [ " required_proxies " ] = requiredProxies ;
message [ " required_resolvers " ] = requiredResolvers ;
2017-10-25 07:28:50 +08:00
} else if ( mStatusCode = = RecoveryStatus : : locking_old_transaction_servers ) {
2018-05-03 01:44:38 +08:00
message [ " missing_logs " ] = md . getValue ( " MissingIDs " ) . c_str ( ) ;
2017-05-26 04:48:44 +08:00
}
// TODO: time_in_recovery: 0.5
// time_in_state: 0.1
} catch ( Error & e ) {
if ( e . code ( ) = = error_code_actor_cancelled )
throw ;
}
// If recovery status name is not know, status is incomplete
2018-09-06 13:51:15 +08:00
if ( message . empty ( ) ) {
2017-05-26 04:48:44 +08:00
incomplete_reasons - > insert ( " Recovery Status unavailable. " ) ;
2018-09-06 13:51:15 +08:00
}
2017-05-26 04:48:44 +08:00
return message ;
}
ACTOR static Future < double > doGrvProbe ( Transaction * tr , Optional < FDBTransactionOptions : : Option > priority = Optional < FDBTransactionOptions : : Option > ( ) ) {
state double start = timer_monotonic ( ) ;
loop {
try {
2017-08-29 08:16:46 +08:00
tr - > setOption ( FDBTransactionOptions : : LOCK_AWARE ) ;
2017-05-26 04:48:44 +08:00
if ( priority . present ( ) ) {
tr - > setOption ( priority . get ( ) ) ;
}
2019-02-13 08:07:17 +08:00
wait ( success ( tr - > getReadVersion ( ) ) ) ;
2017-05-26 04:48:44 +08:00
return timer_monotonic ( ) - start ;
}
catch ( Error & e ) {
2018-08-11 04:57:10 +08:00
wait ( tr - > onError ( e ) ) ;
2017-05-26 04:48:44 +08:00
}
}
}
ACTOR static Future < double > doReadProbe ( Future < double > grvProbe , Transaction * tr ) {
ErrorOr < double > grv = wait ( errorOr ( grvProbe ) ) ;
if ( grv . isError ( ) ) {
throw grv . getError ( ) ;
}
state double start = timer_monotonic ( ) ;
loop {
2017-08-29 08:16:46 +08:00
tr - > setOption ( FDBTransactionOptions : : LOCK_AWARE ) ;
2017-05-26 04:48:44 +08:00
try {
Optional < Standalone < StringRef > > _ = wait ( tr - > get ( LiteralStringRef ( " \xff /StatusJsonTestKey62793 " ) ) ) ;
return timer_monotonic ( ) - start ;
}
catch ( Error & e ) {
2018-08-11 04:57:10 +08:00
wait ( tr - > onError ( e ) ) ;
2017-05-26 04:48:44 +08:00
tr - > setOption ( FDBTransactionOptions : : PRIORITY_SYSTEM_IMMEDIATE ) ;
}
}
}
ACTOR static Future < double > doCommitProbe ( Future < double > grvProbe , Transaction * sourceTr , Transaction * tr ) {
ErrorOr < double > grv = wait ( errorOr ( grvProbe ) ) ;
if ( grv . isError ( ) ) {
throw grv . getError ( ) ;
}
ASSERT ( sourceTr - > getReadVersion ( ) . isReady ( ) ) ;
tr - > setVersion ( sourceTr - > getReadVersion ( ) . get ( ) ) ;
state double start = timer_monotonic ( ) ;
loop {
try {
2017-08-29 08:16:46 +08:00
tr - > setOption ( FDBTransactionOptions : : LOCK_AWARE ) ;
2017-05-26 04:48:44 +08:00
tr - > setOption ( FDBTransactionOptions : : PRIORITY_SYSTEM_IMMEDIATE ) ;
tr - > makeSelfConflicting ( ) ;
2018-08-11 04:57:10 +08:00
wait ( tr - > commit ( ) ) ;
2017-05-26 04:48:44 +08:00
return timer_monotonic ( ) - start ;
}
catch ( Error & e ) {
2018-08-11 04:57:10 +08:00
wait ( tr - > onError ( e ) ) ;
2017-05-26 04:48:44 +08:00
}
}
}
2018-09-09 06:44:48 +08:00
ACTOR static Future < Void > doProbe ( Future < double > probe , int timeoutSeconds , const char * prefix , const char * description , JsonBuilderObject * probeObj , JsonBuilderArray * messages , std : : set < std : : string > * incomplete_reasons , bool * isAvailable = nullptr ) {
2017-05-26 04:48:44 +08:00
choose {
when ( ErrorOr < double > result = wait ( errorOr ( probe ) ) ) {
if ( result . isError ( ) ) {
2018-09-06 13:51:15 +08:00
if ( isAvailable ! = nullptr ) {
* isAvailable = false ;
}
2017-05-26 04:48:44 +08:00
incomplete_reasons - > insert ( format ( " Unable to retrieve latency probe information (%s: %s). " , description , result . getError ( ) . what ( ) ) ) ;
}
else {
( * probeObj ) [ format ( " %s_seconds " , prefix ) . c_str ( ) ] = result . get ( ) ;
}
}
2018-08-11 04:57:10 +08:00
when ( wait ( delay ( timeoutSeconds ) ) ) {
2018-09-06 13:51:15 +08:00
if ( isAvailable ! = nullptr ) {
* isAvailable = false ;
}
2018-08-31 16:21:24 +08:00
messages - > push_back ( JsonString : : makeMessage ( format ( " %s_probe_timeout " , prefix ) . c_str ( ) , format ( " Unable to %s after %d seconds. " , description , timeoutSeconds ) . c_str ( ) ) ) ;
2017-05-26 04:48:44 +08:00
}
}
return Void ( ) ;
}
2018-09-09 06:44:48 +08:00
ACTOR static Future < JsonBuilderObject > latencyProbeFetcher ( Database cx , JsonBuilderArray * messages , std : : set < std : : string > * incomplete_reasons , bool * isAvailable ) {
2017-05-26 04:48:44 +08:00
state Transaction trImmediate ( cx ) ;
state Transaction trDefault ( cx ) ;
state Transaction trBatch ( cx ) ;
state Transaction trWrite ( cx ) ;
2018-09-09 06:44:48 +08:00
state JsonBuilderObject statusObj ;
2017-05-26 04:48:44 +08:00
try {
Future < double > immediateGrvProbe = doGrvProbe ( & trImmediate , FDBTransactionOptions : : PRIORITY_SYSTEM_IMMEDIATE ) ;
Future < double > defaultGrvProbe = doGrvProbe ( & trDefault ) ;
Future < double > batchGrvProbe = doGrvProbe ( & trBatch , FDBTransactionOptions : : PRIORITY_BATCH ) ;
Future < double > readProbe = doReadProbe ( immediateGrvProbe , & trImmediate ) ;
Future < double > commitProbe = doCommitProbe ( immediateGrvProbe , & trImmediate , & trWrite ) ;
int timeoutSeconds = 5 ;
std : : vector < Future < Void > > probes ;
2018-09-06 13:51:15 +08:00
probes . push_back ( doProbe ( immediateGrvProbe , timeoutSeconds , " immediate_priority_transaction_start " , " start immediate priority transaction " , & statusObj , messages , incomplete_reasons , isAvailable ) ) ;
2017-05-26 04:48:44 +08:00
probes . push_back ( doProbe ( defaultGrvProbe , timeoutSeconds , " transaction_start " , " start default priority transaction " , & statusObj , messages , incomplete_reasons ) ) ;
probes . push_back ( doProbe ( batchGrvProbe , timeoutSeconds , " batch_priority_transaction_start " , " start batch priority transaction " , & statusObj , messages , incomplete_reasons ) ) ;
2018-09-06 13:51:15 +08:00
probes . push_back ( doProbe ( readProbe , timeoutSeconds , " read " , " read " , & statusObj , messages , incomplete_reasons , isAvailable ) ) ;
probes . push_back ( doProbe ( commitProbe , timeoutSeconds , " commit " , " commit " , & statusObj , messages , incomplete_reasons , isAvailable ) ) ;
2017-05-26 04:48:44 +08:00
2018-08-11 04:57:10 +08:00
wait ( waitForAll ( probes ) ) ;
2017-05-26 04:48:44 +08:00
}
catch ( Error & e ) {
incomplete_reasons - > insert ( format ( " Unable to retrieve latency probe information (%s). " , e . what ( ) ) ) ;
}
return statusObj ;
}
2019-10-03 08:06:19 +08:00
ACTOR static Future < Void > consistencyCheckStatusFetcher ( Database cx , JsonBuilderArray * messages , std : : set < std : : string > * incomplete_reasons ) {
try {
state Transaction tr ( cx ) ;
loop {
try {
tr . setOption ( FDBTransactionOptions : : PRIORITY_SYSTEM_IMMEDIATE ) ;
tr . setOption ( FDBTransactionOptions : : LOCK_AWARE ) ;
tr . setOption ( FDBTransactionOptions : : ACCESS_SYSTEM_KEYS ) ;
Optional < Value > ccSuspendVal = wait ( timeoutError ( BUGGIFY ? Never ( ) : tr . get ( fdbShouldConsistencyCheckBeSuspended ) , 5.0 ) ) ;
bool ccSuspend = ccSuspendVal . present ( ) ? BinaryReader : : fromStringRef < bool > ( ccSuspendVal . get ( ) , Unversioned ( ) ) : false ;
if ( ccSuspend ) {
messages - > push_back ( JsonString : : makeMessage ( " consistencycheck_disabled " , " Consistency checker is disabled. " ) ) ;
}
break ;
} catch ( Error & e ) {
if ( e . code ( ) = = error_code_timed_out ) {
messages - > push_back ( JsonString : : makeMessage ( " consistencycheck_suspendkey_fetch_timeout " ,
format ( " Timed out trying to fetch `%s` from the database. " , printable ( fdbShouldConsistencyCheckBeSuspended ) . c_str ( ) ) . c_str ( ) ) ) ;
2019-06-21 12:38:45 +08:00
break ;
2019-10-03 08:06:19 +08:00
}
wait ( tr . onError ( e ) ) ;
}
}
} catch ( Error & e ) {
incomplete_reasons - > insert ( format ( " Unable to retrieve consistency check settings (%s). " , e . what ( ) ) ) ;
}
return Void ( ) ;
}
2019-10-23 02:58:40 +08:00
struct LogRangeAndUID {
KeyRange range ;
UID destID ;
LogRangeAndUID ( KeyRange const & range , UID const & destID ) : range ( range ) , destID ( destID ) { }
2020-02-05 02:26:18 +08:00
bool operator < ( LogRangeAndUID const & r ) const {
2019-10-23 02:58:40 +08:00
if ( range . begin ! = r . range . begin ) return range . begin < r . range . begin ;
if ( range . end ! = r . range . end ) return range . end < r . range . end ;
2020-02-05 02:26:18 +08:00
return destID < r . destID ;
2019-10-23 02:58:40 +08:00
}
} ;
2019-10-03 08:06:19 +08:00
ACTOR static Future < Void > logRangeWarningFetcher ( Database cx , JsonBuilderArray * messages , std : : set < std : : string > * incomplete_reasons ) {
try {
state Transaction tr ( cx ) ;
2019-10-23 08:06:54 +08:00
state Future < Void > timeoutFuture = timeoutError ( Future < Void > ( Never ( ) ) , 5.0 ) ;
2019-10-03 08:06:19 +08:00
loop {
try {
tr . setOption ( FDBTransactionOptions : : PRIORITY_SYSTEM_IMMEDIATE ) ;
2019-10-23 07:33:44 +08:00
tr . setOption ( FDBTransactionOptions : : LOCK_AWARE ) ;
tr . setOption ( FDBTransactionOptions : : ACCESS_SYSTEM_KEYS ) ;
2019-10-03 08:06:19 +08:00
2019-10-23 02:58:40 +08:00
state Future < Standalone < RangeResultRef > > existingDestUidValues = tr . getRange ( KeyRangeRef ( destUidLookupPrefix , strinc ( destUidLookupPrefix ) ) , CLIENT_KNOBS - > TOO_MANY ) ;
state Future < Standalone < RangeResultRef > > existingLogRanges = tr . getRange ( logRangesRange , CLIENT_KNOBS - > TOO_MANY ) ;
wait ( ( success ( existingDestUidValues ) & & success ( existingLogRanges ) ) | | timeoutFuture ) ;
std : : set < LogRangeAndUID > loggingRanges ;
for ( auto & it : existingLogRanges . get ( ) ) {
Key logDestination ;
UID logUid ;
KeyRef logRangeBegin = logRangesDecodeKey ( it . key , & logUid ) ;
2019-10-23 04:27:52 +08:00
Key logRangeEnd = logRangesDecodeValue ( it . value , & logDestination ) ;
2019-10-23 02:58:40 +08:00
loggingRanges . insert ( LogRangeAndUID ( KeyRangeRef ( logRangeBegin , logRangeEnd ) , logUid ) ) ;
}
2019-10-03 08:06:19 +08:00
2019-10-17 11:17:09 +08:00
std : : set < std : : pair < Key , Key > > existingRanges ;
2019-10-23 02:58:40 +08:00
for ( auto & it : existingDestUidValues . get ( ) ) {
2019-10-03 08:06:19 +08:00
KeyRange range = BinaryReader : : fromStringRef < KeyRange > ( it . key . removePrefix ( destUidLookupPrefix ) , IncludeVersion ( ) ) ;
2019-10-23 02:58:40 +08:00
UID logUid = BinaryReader : : fromStringRef < UID > ( it . value , Unversioned ( ) ) ;
if ( loggingRanges . count ( LogRangeAndUID ( range , logUid ) ) ) {
std : : pair < Key , Key > rangePair = std : : make_pair ( range . begin , range . end ) ;
if ( existingRanges . count ( rangePair ) ) {
messages - > push_back ( JsonString : : makeMessage ( " duplicate_mutation_streams " , format ( " Backup and DR are not sharing the same stream of mutations for `%s` - `%s` " , printable ( range . begin ) . c_str ( ) , printable ( range . end ) . c_str ( ) ) . c_str ( ) ) ) ;
break ;
}
existingRanges . insert ( rangePair ) ;
} else {
2019-10-23 07:33:44 +08:00
//This cleanup is done during status, because it should only be required once after upgrading to 6.2.7 or later.
//There is no other good location to detect that the metadata is mismatched.
TraceEvent ( SevWarnAlways , " CleaningDestUidLookup " ) . detail ( " K " , it . key . printable ( ) ) . detail ( " V " , it . value . printable ( ) ) ;
2019-10-23 02:58:40 +08:00
tr . clear ( it . key ) ;
2019-07-16 03:33:18 +08:00
}
2019-06-21 12:38:45 +08:00
}
2019-10-23 04:32:09 +08:00
wait ( tr . commit ( ) | | timeoutFuture ) ;
2019-10-03 08:06:19 +08:00
break ;
} catch ( Error & e ) {
if ( e . code ( ) = = error_code_timed_out ) {
messages - > push_back ( JsonString : : makeMessage ( " duplicate_mutation_fetch_timeout " ,
format ( " Timed out trying to fetch `%s` from the database. " , printable ( destUidLookupPrefix ) . c_str ( ) ) . c_str ( ) ) ) ;
break ;
2019-06-21 12:38:45 +08:00
}
2019-10-03 08:06:19 +08:00
wait ( tr . onError ( e ) ) ;
2019-06-21 12:38:45 +08:00
}
}
2019-10-03 08:06:19 +08:00
} catch ( Error & e ) {
incomplete_reasons - > insert ( format ( " Unable to retrieve log ranges (%s). " , e . what ( ) ) ) ;
2019-06-21 12:38:45 +08:00
}
return Void ( ) ;
}
2019-04-03 05:15:51 +08:00
struct LoadConfigurationResult {
bool fullReplication ;
Optional < Key > healthyZone ;
double healthyZoneSeconds ;
2019-07-12 05:53:00 +08:00
bool rebalanceDDIgnored ;
2019-08-30 09:41:34 +08:00
bool dataDistributionDisabled ;
2019-04-03 05:15:51 +08:00
2019-08-30 09:41:34 +08:00
LoadConfigurationResult ( ) : fullReplication ( true ) , healthyZoneSeconds ( 0 ) , rebalanceDDIgnored ( false ) , dataDistributionDisabled ( false ) { }
2019-04-03 05:15:51 +08:00
} ;
ACTOR static Future < std : : pair < Optional < DatabaseConfiguration > , Optional < LoadConfigurationResult > > > loadConfiguration ( Database cx , JsonBuilderArray * messages , std : : set < std : : string > * status_incomplete_reasons ) {
2017-05-26 04:48:44 +08:00
state Optional < DatabaseConfiguration > result ;
2019-04-03 05:15:51 +08:00
state Optional < LoadConfigurationResult > loadResult ;
2017-05-26 04:48:44 +08:00
state Transaction tr ( cx ) ;
state Future < Void > getConfTimeout = delay ( 5.0 ) ;
loop {
tr . setOption ( FDBTransactionOptions : : PRIORITY_SYSTEM_IMMEDIATE ) ;
tr . setOption ( FDBTransactionOptions : : CAUSAL_READ_RISKY ) ;
try {
choose {
when ( Standalone < RangeResultRef > res = wait ( tr . getRange ( configKeys , SERVER_KNOBS - > CONFIGURATION_ROWS_TO_FETCH ) ) ) {
DatabaseConfiguration configuration ;
if ( res . size ( ) = = SERVER_KNOBS - > CONFIGURATION_ROWS_TO_FETCH ) {
status_incomplete_reasons - > insert ( " Too many configuration parameters set. " ) ;
}
else {
2018-01-23 03:40:08 +08:00
configuration . fromKeyValues ( ( VectorRef < KeyValueRef > ) res ) ;
2017-05-26 04:48:44 +08:00
}
result = configuration ;
}
2018-08-11 04:57:10 +08:00
when ( wait ( getConfTimeout ) ) {
2018-06-15 12:14:18 +08:00
if ( ! result . present ( ) ) {
2018-08-31 16:21:24 +08:00
messages - > push_back ( JsonString : : makeMessage ( " unreadable_configuration " , " Unable to read database configuration. " ) ) ;
2018-06-15 12:14:18 +08:00
} else {
2018-08-31 16:21:24 +08:00
messages - > push_back ( JsonString : : makeMessage ( " full_replication_timeout " , " Unable to read datacenter replicas. " ) ) ;
2018-06-15 12:14:18 +08:00
}
break ;
}
}
ASSERT ( result . present ( ) ) ;
state std : : vector < Future < Optional < Value > > > replicasFutures ;
for ( auto & region : result . get ( ) . regions ) {
replicasFutures . push_back ( tr . get ( datacenterReplicasKeyFor ( region . dcId ) ) ) ;
}
2019-04-03 05:46:56 +08:00
state Future < Optional < Value > > healthyZoneValue = tr . get ( healthyZoneKey ) ;
2019-07-12 05:53:00 +08:00
state Future < Optional < Value > > rebalanceDDIgnored = tr . get ( rebalanceDDIgnoreKey ) ;
2019-08-30 09:41:34 +08:00
state Future < Optional < Value > > ddModeKey = tr . get ( dataDistributionModeKey ) ;
2018-06-15 12:14:18 +08:00
choose {
2019-08-30 09:41:34 +08:00
when ( wait ( waitForAll ( replicasFutures ) & & success ( healthyZoneValue ) & & success ( rebalanceDDIgnored ) & & success ( ddModeKey ) ) ) {
2018-06-15 12:14:18 +08:00
int unreplicated = 0 ;
for ( int i = 0 ; i < result . get ( ) . regions . size ( ) ; i + + ) {
if ( ! replicasFutures [ i ] . get ( ) . present ( ) | | decodeDatacenterReplicasValue ( replicasFutures [ i ] . get ( ) . get ( ) ) < result . get ( ) . storageTeamSize ) {
unreplicated + + ;
}
}
2019-04-03 05:15:51 +08:00
LoadConfigurationResult res ;
res . fullReplication = ( ! unreplicated | | ( result . get ( ) . usableRegions = = 1 & & unreplicated < result . get ( ) . regions . size ( ) ) ) ;
2019-04-03 05:46:56 +08:00
if ( healthyZoneValue . get ( ) . present ( ) ) {
auto healthyZone = decodeHealthyZoneValue ( healthyZoneValue . get ( ) . get ( ) ) ;
2019-08-30 09:41:34 +08:00
if ( healthyZone . first = = ignoreSSFailuresZoneString ) {
res . healthyZone = healthyZone . first ;
}
else if ( healthyZone . second > tr . getReadVersion ( ) . get ( ) ) {
2019-04-03 05:46:56 +08:00
res . healthyZone = healthyZone . first ;
res . healthyZoneSeconds = ( healthyZone . second - tr . getReadVersion ( ) . get ( ) ) / CLIENT_KNOBS - > CORE_VERSIONSPERSECOND ;
}
2019-04-03 05:15:51 +08:00
}
2019-07-12 05:53:00 +08:00
res . rebalanceDDIgnored = rebalanceDDIgnored . get ( ) . present ( ) ;
2019-08-30 09:41:34 +08:00
if ( ddModeKey . get ( ) . present ( ) ) {
BinaryReader rd ( ddModeKey . get ( ) . get ( ) , Unversioned ( ) ) ;
int currentMode ;
rd > > currentMode ;
if ( currentMode = = 0 ) {
res . dataDistributionDisabled = true ;
}
}
2019-04-03 05:15:51 +08:00
loadResult = res ;
2018-06-15 12:14:18 +08:00
}
2018-08-11 04:57:10 +08:00
when ( wait ( getConfTimeout ) ) {
2018-08-31 16:21:24 +08:00
messages - > push_back ( JsonString : : makeMessage ( " full_replication_timeout " , " Unable to read datacenter replicas. " ) ) ;
2017-05-26 04:48:44 +08:00
}
}
break ;
}
catch ( Error & e ) {
2018-08-11 04:57:10 +08:00
wait ( tr . onError ( e ) ) ;
2017-05-26 04:48:44 +08:00
}
}
2019-04-03 05:15:51 +08:00
return std : : make_pair ( result , loadResult ) ;
2017-05-26 04:48:44 +08:00
}
2018-09-09 06:44:48 +08:00
static JsonBuilderObject configurationFetcher ( Optional < DatabaseConfiguration > conf , ServerCoordinators coordinators , std : : set < std : : string > * incomplete_reasons ) {
JsonBuilderObject statusObj ;
2017-05-26 04:48:44 +08:00
try {
if ( conf . present ( ) ) {
DatabaseConfiguration configuration = conf . get ( ) ;
2018-09-09 06:44:48 +08:00
statusObj . addContents ( configuration . toJSON ( ) ) ;
2017-05-26 04:48:44 +08:00
2018-09-09 06:44:48 +08:00
JsonBuilderArray excludedServersArr ;
2017-05-26 04:48:44 +08:00
std : : set < AddressExclusion > excludedServers = configuration . getExcludedServers ( ) ;
for ( std : : set < AddressExclusion > : : iterator it = excludedServers . begin ( ) ; it ! = excludedServers . end ( ) ; it + + ) {
2018-09-09 06:44:48 +08:00
JsonBuilderObject statusObj ;
2017-05-26 04:48:44 +08:00
statusObj [ " address " ] = it - > toString ( ) ;
excludedServersArr . push_back ( statusObj ) ;
}
statusObj [ " excluded_servers " ] = excludedServersArr ;
}
2018-03-06 11:27:46 +08:00
vector < ClientLeaderRegInterface > coordinatorLeaderServers = coordinators . clientLeaderServers ;
int count = coordinatorLeaderServers . size ( ) ;
statusObj [ " coordinators_count " ] = count ;
2017-05-26 04:48:44 +08:00
}
2019-03-27 00:58:54 +08:00
catch ( Error & ) {
2017-05-26 04:48:44 +08:00
incomplete_reasons - > insert ( " Could not retrieve all configuration status information. " ) ;
}
return statusObj ;
}
2019-10-11 01:36:35 +08:00
ACTOR static Future < JsonBuilderObject > dataStatusFetcher ( WorkerDetails ddWorker , DatabaseConfiguration configuration , int * minReplicasRemaining ) {
2018-09-09 06:44:48 +08:00
state JsonBuilderObject statusObjData ;
2017-05-26 04:48:44 +08:00
try {
2018-05-03 01:44:38 +08:00
std : : vector < Future < TraceEventFields > > futures ;
2017-05-26 04:48:44 +08:00
// TODO: Should this be serial?
2019-03-09 00:25:07 +08:00
futures . push_back ( timeoutError ( ddWorker . interf . eventLogRequest . getReply ( EventLogRequest ( LiteralStringRef ( " DDTrackerStarting " ) ) ) , 1.0 ) ) ;
futures . push_back ( timeoutError ( ddWorker . interf . eventLogRequest . getReply ( EventLogRequest ( LiteralStringRef ( " DDTrackerStats " ) ) ) , 1.0 ) ) ;
futures . push_back ( timeoutError ( ddWorker . interf . eventLogRequest . getReply ( EventLogRequest ( LiteralStringRef ( " MovingData " ) ) ) , 1.0 ) ) ;
futures . push_back ( timeoutError ( ddWorker . interf . eventLogRequest . getReply ( EventLogRequest ( LiteralStringRef ( " TotalDataInFlight " ) ) ) , 1.0 ) ) ;
futures . push_back ( timeoutError ( ddWorker . interf . eventLogRequest . getReply ( EventLogRequest ( LiteralStringRef ( " TotalDataInFlightRemote " ) ) ) , 1.0 ) ) ;
2017-05-26 04:48:44 +08:00
2018-05-03 01:44:38 +08:00
std : : vector < TraceEventFields > dataInfo = wait ( getAll ( futures ) ) ;
2017-05-26 04:48:44 +08:00
2018-05-03 01:44:38 +08:00
TraceEventFields startingStats = dataInfo [ 0 ] ;
2018-08-10 04:16:09 +08:00
TraceEventFields dataStats = dataInfo [ 1 ] ;
2017-05-26 04:48:44 +08:00
2018-05-03 01:44:38 +08:00
if ( startingStats . size ( ) & & startingStats . getValue ( " State " ) ! = " Active " ) {
2018-09-09 06:44:48 +08:00
JsonBuilderObject stateSectionObj ;
2017-05-26 04:48:44 +08:00
stateSectionObj [ " name " ] = " initializing " ;
stateSectionObj [ " description " ] = " (Re)initializing automatic data distribution " ;
2018-08-10 04:16:09 +08:00
statusObjData [ " state " ] = stateSectionObj ;
return statusObjData ;
}
TraceEventFields md = dataInfo [ 2 ] ;
// If we have a MovingData message, parse it.
2019-05-08 08:06:54 +08:00
int64_t partitionsInFlight = 0 ;
int movingHighestPriority = 1000 ;
2018-08-10 04:16:09 +08:00
if ( md . size ( ) )
{
2019-03-01 01:53:16 +08:00
int64_t partitionsInQueue = md . getInt64 ( " InQueue " ) ;
int64_t averagePartitionSize = md . getInt64 ( " AverageShardSize " ) ;
2019-05-08 08:06:54 +08:00
partitionsInFlight = md . getInt64 ( " InFlight " ) ;
movingHighestPriority = md . getInt ( " HighestPriority " ) ;
2018-08-10 04:16:09 +08:00
if ( averagePartitionSize > = 0 ) {
2018-09-09 06:44:48 +08:00
JsonBuilderObject moving_data ;
2018-08-10 04:16:09 +08:00
moving_data [ " in_queue_bytes " ] = partitionsInQueue * averagePartitionSize ;
moving_data [ " in_flight_bytes " ] = partitionsInFlight * averagePartitionSize ;
2018-09-10 14:19:00 +08:00
moving_data . setKeyRawNumber ( " total_written_bytes " , md . getValue ( " BytesWritten " ) ) ;
2019-05-08 08:06:54 +08:00
moving_data [ " highest_priority " ] = movingHighestPriority ;
2018-08-10 04:16:09 +08:00
// TODO: moving_data["rate_bytes"] = makeCounter(hz, c, r);
statusObjData [ " moving_data " ] = moving_data ;
statusObjData [ " average_partition_size_bytes " ] = averagePartitionSize ;
}
2017-05-26 04:48:44 +08:00
}
2018-08-10 04:16:09 +08:00
if ( dataStats . size ( ) )
{
2018-09-10 14:19:00 +08:00
statusObjData . setKeyRawNumber ( " total_kv_size_bytes " , dataStats . getValue ( " TotalSizeBytes " ) ) ;
2019-09-28 13:39:19 +08:00
statusObjData . setKeyRawNumber ( " system_kv_size_bytes " , dataStats . getValue ( " SystemSizeBytes " ) ) ;
2018-09-10 14:19:00 +08:00
statusObjData . setKeyRawNumber ( " partitions_count " , dataStats . getValue ( " Shards " ) ) ;
2018-08-10 04:16:09 +08:00
}
2018-09-09 06:44:48 +08:00
JsonBuilderArray teamTrackers ;
2018-08-10 04:16:09 +08:00
for ( int i = 0 ; i < 2 ; i + + ) {
TraceEventFields inFlight = dataInfo [ 3 + i ] ;
if ( ! inFlight . size ( ) ) {
continue ;
}
2019-10-11 01:36:35 +08:00
int replicas = configuration . storageTeamSize ;
2019-03-01 01:53:16 +08:00
bool primary = inFlight . getInt ( " Primary " ) ;
int highestPriority = inFlight . getInt ( " HighestPriority " ) ;
2019-07-20 09:32:05 +08:00
2019-10-12 08:50:43 +08:00
if ( movingHighestPriority < SERVER_KNOBS - > PRIORITY_TEAM_REDUNDANT ) {
2019-05-08 08:06:54 +08:00
highestPriority = movingHighestPriority ;
2019-07-20 09:32:05 +08:00
} else if ( partitionsInFlight > 0 ) {
2019-10-12 08:50:43 +08:00
highestPriority = std : : max < int > ( highestPriority , SERVER_KNOBS - > PRIORITY_MERGE_SHARD ) ;
2019-05-08 08:06:54 +08:00
}
2018-08-10 04:16:09 +08:00
2018-09-09 06:44:48 +08:00
JsonBuilderObject team_tracker ;
2018-08-10 04:16:09 +08:00
team_tracker [ " primary " ] = primary ;
2018-09-10 14:19:00 +08:00
team_tracker . setKeyRawNumber ( " in_flight_bytes " , inFlight . getValue ( " TotalBytes " ) ) ;
team_tracker . setKeyRawNumber ( " unhealthy_servers " , inFlight . getValue ( " UnhealthyServers " ) ) ;
2018-08-10 04:16:09 +08:00
2018-09-09 06:44:48 +08:00
JsonBuilderObject stateSectionObj ;
2019-10-12 08:50:43 +08:00
if ( highestPriority > = SERVER_KNOBS - > PRIORITY_TEAM_0_LEFT ) {
2018-08-10 04:16:09 +08:00
stateSectionObj [ " healthy " ] = false ;
stateSectionObj [ " name " ] = " missing_data " ;
stateSectionObj [ " description " ] = " No replicas remain of some data " ;
stateSectionObj [ " min_replicas_remaining " ] = 0 ;
2019-10-11 01:36:35 +08:00
replicas = 0 ;
2018-08-10 04:16:09 +08:00
}
2019-10-12 08:50:43 +08:00
else if ( highestPriority > = SERVER_KNOBS - > PRIORITY_TEAM_1_LEFT ) {
2018-08-10 04:16:09 +08:00
stateSectionObj [ " healthy " ] = false ;
stateSectionObj [ " name " ] = " healing " ;
stateSectionObj [ " description " ] = " Only one replica remains of some data " ;
stateSectionObj [ " min_replicas_remaining " ] = 1 ;
2019-10-11 01:36:35 +08:00
replicas = 1 ;
2018-08-10 04:16:09 +08:00
}
2019-10-12 08:50:43 +08:00
else if ( highestPriority > = SERVER_KNOBS - > PRIORITY_TEAM_2_LEFT ) {
2018-08-10 04:16:09 +08:00
stateSectionObj [ " healthy " ] = false ;
stateSectionObj [ " name " ] = " healing " ;
stateSectionObj [ " description " ] = " Only two replicas remain of some data " ;
stateSectionObj [ " min_replicas_remaining " ] = 2 ;
2019-10-11 01:36:35 +08:00
replicas = 2 ;
2017-05-26 04:48:44 +08:00
}
2019-10-12 08:50:43 +08:00
else if ( highestPriority > = SERVER_KNOBS - > PRIORITY_TEAM_UNHEALTHY ) {
2018-08-10 04:16:09 +08:00
stateSectionObj [ " healthy " ] = false ;
stateSectionObj [ " name " ] = " healing " ;
stateSectionObj [ " description " ] = " Restoring replication factor " ;
2019-10-12 08:50:43 +08:00
} else if ( highestPriority > = SERVER_KNOBS - > PRIORITY_MERGE_SHARD ) {
2018-08-10 04:16:09 +08:00
stateSectionObj [ " healthy " ] = true ;
stateSectionObj [ " name " ] = " healthy_repartitioning " ;
stateSectionObj [ " description " ] = " Repartitioning. " ;
2019-10-12 08:50:43 +08:00
} else if ( highestPriority > = SERVER_KNOBS - > PRIORITY_TEAM_REDUNDANT ) {
2019-02-22 06:16:44 +08:00
stateSectionObj [ " healthy " ] = true ;
2019-03-07 07:05:21 +08:00
stateSectionObj [ " name " ] = " optimizing_team_collections " ;
stateSectionObj [ " description " ] = " Optimizing team collections " ;
2019-10-12 08:50:43 +08:00
} else if ( highestPriority > = SERVER_KNOBS - > PRIORITY_TEAM_CONTAINS_UNDESIRED_SERVER ) {
2018-08-10 04:16:09 +08:00
stateSectionObj [ " healthy " ] = true ;
stateSectionObj [ " name " ] = " healthy_removing_server " ;
stateSectionObj [ " description " ] = " Removing storage server " ;
2019-10-12 08:50:43 +08:00
} else if ( highestPriority = = SERVER_KNOBS - > PRIORITY_TEAM_HEALTHY ) {
2019-07-20 09:32:05 +08:00
stateSectionObj [ " healthy " ] = true ;
2019-05-08 08:06:54 +08:00
stateSectionObj [ " name " ] = " healthy " ;
2019-10-12 08:50:43 +08:00
} else if ( highestPriority > = SERVER_KNOBS - > PRIORITY_RECOVER_MOVE ) {
2018-08-10 04:16:09 +08:00
stateSectionObj [ " healthy " ] = true ;
stateSectionObj [ " name " ] = " healthy_rebalancing " ;
stateSectionObj [ " description " ] = " Rebalancing " ;
2019-07-20 09:32:05 +08:00
} else if ( highestPriority > = 0 ) {
2018-08-10 04:16:09 +08:00
stateSectionObj [ " healthy " ] = true ;
stateSectionObj [ " name " ] = " healthy " ;
2017-05-26 04:48:44 +08:00
}
2018-08-10 04:16:09 +08:00
if ( ! stateSectionObj . empty ( ) ) {
team_tracker [ " state " ] = stateSectionObj ;
teamTrackers . push_back ( team_tracker ) ;
if ( primary ) {
statusObjData [ " state " ] = stateSectionObj ;
}
}
2019-10-11 01:36:35 +08:00
if ( primary ) {
* minReplicasRemaining = std : : max ( * minReplicasRemaining , 0 ) + replicas ;
}
else if ( replicas > 0 ) {
* minReplicasRemaining = std : : max ( * minReplicasRemaining , 0 ) + 1 ;
}
2017-05-26 04:48:44 +08:00
}
2018-08-10 04:16:09 +08:00
statusObjData [ " team_trackers " ] = teamTrackers ;
2017-05-26 04:48:44 +08:00
}
catch ( Error & e ) {
if ( e . code ( ) = = error_code_actor_cancelled )
throw ;
// The most likely reason to be here is a timeout, either way we have no idea if the data state is healthy or not
// from the "cluster" perspective - from the client perspective it is not but that is indicated elsewhere.
}
return statusObjData ;
}
ACTOR template < class iface >
2019-02-08 05:39:22 +08:00
static Future < vector < std : : pair < iface , EventMap > > > getServerMetrics ( vector < iface > servers , std : : unordered_map < NetworkAddress , WorkerInterface > address_workers , std : : vector < std : : string > eventNames ) {
2018-05-03 01:44:38 +08:00
state vector < Future < Optional < TraceEventFields > > > futures ;
2017-05-26 04:48:44 +08:00
for ( auto s : servers ) {
2019-02-08 05:39:22 +08:00
for ( auto name : eventNames ) {
futures . push_back ( latestEventOnWorker ( address_workers [ s . address ( ) ] , s . id ( ) . toString ( ) + " / " + name ) ) ;
}
2017-05-26 04:48:44 +08:00
}
2018-08-11 04:57:10 +08:00
wait ( waitForAll ( futures ) ) ;
2017-05-26 04:48:44 +08:00
2019-02-08 05:39:22 +08:00
vector < std : : pair < iface , EventMap > > results ;
auto futureItr = futures . begin ( ) ;
2017-05-26 04:48:44 +08:00
for ( int i = 0 ; i < servers . size ( ) ; i + + ) {
2019-02-08 05:39:22 +08:00
EventMap serverResults ;
for ( auto name : eventNames ) {
ASSERT ( futureItr ! = futures . end ( ) ) ;
serverResults [ name ] = futureItr - > get ( ) . present ( ) ? futureItr - > get ( ) . get ( ) : TraceEventFields ( ) ;
+ + futureItr ;
}
results . push_back ( std : : make_pair ( servers [ i ] , serverResults ) ) ;
2017-05-26 04:48:44 +08:00
}
2019-02-08 05:39:22 +08:00
2017-05-26 04:48:44 +08:00
return results ;
}
2019-02-08 05:39:22 +08:00
ACTOR static Future < vector < std : : pair < StorageServerInterface , EventMap > > > getStorageServersAndMetrics ( Database cx , std : : unordered_map < NetworkAddress , WorkerInterface > address_workers ) {
2017-05-26 04:48:44 +08:00
vector < StorageServerInterface > servers = wait ( timeoutError ( getStorageServers ( cx , true ) , 5.0 ) ) ;
2019-07-12 05:53:00 +08:00
vector < std : : pair < StorageServerInterface , EventMap > > results = wait (
getServerMetrics ( servers , address_workers , std : : vector < std : : string > { " StorageMetrics " , " ReadLatencyMetrics " } ) ) ;
2019-02-08 05:39:22 +08:00
2017-05-26 04:48:44 +08:00
return results ;
}
2019-09-11 05:27:22 +08:00
ACTOR static Future < vector < std : : pair < TLogInterface , EventMap > > > getTLogsAndMetrics ( Reference < AsyncVar < CachedSerialization < ServerDBInfo > > > db , std : : unordered_map < NetworkAddress , WorkerInterface > address_workers ) {
vector < TLogInterface > servers = db - > get ( ) . read ( ) . logSystemConfig . allPresentLogs ( ) ;
2019-07-12 05:53:00 +08:00
vector < std : : pair < TLogInterface , EventMap > > results =
wait ( getServerMetrics ( servers , address_workers , std : : vector < std : : string > { " TLogMetrics " } ) ) ;
2019-02-08 05:39:22 +08:00
2018-12-01 02:46:04 +08:00
return results ;
}
2020-02-08 07:44:02 +08:00
ACTOR static Future < vector < std : : pair < MasterProxyInterface , EventMap > > > getProxiesAndMetrics ( Reference < AsyncVar < CachedSerialization < ServerDBInfo > > > db , std : : unordered_map < NetworkAddress , WorkerInterface > address_workers ) {
2019-07-12 05:53:00 +08:00
vector < std : : pair < MasterProxyInterface , EventMap > > results = wait ( getServerMetrics (
2020-02-08 07:44:02 +08:00
db - > get ( ) . read ( ) . client . proxies , address_workers , std : : vector < std : : string > { " GRVLatencyMetrics " , " CommitLatencyMetrics " } ) ) ;
2019-02-08 05:39:22 +08:00
2017-05-26 04:48:44 +08:00
return results ;
}
2019-07-31 05:02:31 +08:00
static int getExtraTLogEligibleZones ( const vector < WorkerDetails > & workers , const DatabaseConfiguration & configuration ) {
std : : set < StringRef > allZones ;
std : : map < Key , std : : set < StringRef > > dcId_zone ;
2019-02-08 05:39:22 +08:00
for ( auto const & worker : workers ) {
2019-03-09 00:25:07 +08:00
if ( worker . processClass . machineClassFitness ( ProcessClass : : TLog ) < ProcessClass : : NeverAssign
& & ! configuration . isExcludedServer ( worker . interf . address ( ) ) )
2017-05-26 04:48:44 +08:00
{
2019-07-31 05:02:31 +08:00
allZones . insert ( worker . interf . locality . zoneId ( ) . get ( ) ) ;
2019-03-09 00:25:07 +08:00
if ( worker . interf . locality . dcId ( ) . present ( ) ) {
2019-07-31 05:02:31 +08:00
dcId_zone [ worker . interf . locality . dcId ( ) . get ( ) ] . insert ( worker . interf . locality . zoneId ( ) . get ( ) ) ;
2018-06-15 08:58:57 +08:00
}
2017-05-26 04:48:44 +08:00
}
}
2018-06-15 08:58:57 +08:00
if ( configuration . regions . size ( ) = = 0 ) {
2019-07-31 05:02:31 +08:00
return allZones . size ( ) - std : : max ( configuration . tLogReplicationFactor , configuration . storageTeamSize ) ;
2018-08-31 16:21:24 +08:00
}
2019-07-31 05:02:31 +08:00
int extraTlogEligibleZones = configuration . usableRegions = = 1 ? 0 : std : : numeric_limits < int > : : max ( ) ;
2018-06-15 08:58:57 +08:00
for ( auto & region : configuration . regions ) {
2019-07-31 05:02:31 +08:00
int eligible = dcId_zone [ region . dcId ] . size ( ) - std : : max ( configuration . remoteTLogReplicationFactor , std : : max ( configuration . tLogReplicationFactor , configuration . storageTeamSize ) ) ;
2018-06-29 14:15:32 +08:00
//FIXME: does not take into account fallback satellite policies
2018-06-15 08:58:57 +08:00
if ( region . satelliteTLogReplicationFactor > 0 ) {
int totalSatelliteEligible = 0 ;
for ( auto & sat : region . satellites ) {
2019-07-31 05:02:31 +08:00
totalSatelliteEligible + = dcId_zone [ sat . dcId ] . size ( ) ;
2018-06-15 08:58:57 +08:00
}
2018-09-01 04:04:00 +08:00
eligible = std : : min < int > ( eligible , totalSatelliteEligible - region . satelliteTLogReplicationFactor ) ;
}
if ( configuration . usableRegions = = 1 ) {
if ( region . priority > = 0 ) {
2019-07-31 05:02:31 +08:00
extraTlogEligibleZones = std : : max ( extraTlogEligibleZones , eligible ) ;
2018-09-01 04:04:00 +08:00
}
} else {
2019-07-31 05:02:31 +08:00
extraTlogEligibleZones = std : : min ( extraTlogEligibleZones , eligible ) ;
2018-06-15 08:58:57 +08:00
}
}
2019-07-31 05:02:31 +08:00
return extraTlogEligibleZones ;
2017-05-26 04:48:44 +08:00
}
2019-03-01 01:53:16 +08:00
JsonBuilderObject getPerfLimit ( TraceEventFields const & ratekeeper , double transPerSec , double tpsLimit ) {
int reason = ratekeeper . getInt ( " Reason " ) ;
JsonBuilderObject perfLimit ;
if ( transPerSec > tpsLimit * 0.8 ) {
// If reason is known, set qos.performance_limited_by, otherwise omit
if ( reason > = 0 & & reason < limitReasonEnd ) {
perfLimit = JsonString : : makeMessage ( limitReasonName [ reason ] , limitReasonDesc [ reason ] ) ;
std : : string reason_server_id = ratekeeper . getValue ( " ReasonServerID " ) ;
if ( ! reason_server_id . empty ( ) )
perfLimit [ " reason_server_id " ] = reason_server_id ;
}
}
else {
perfLimit = JsonString : : makeMessage ( " workload " , " The database is not being saturated by the workload. " ) ;
}
if ( ! perfLimit . empty ( ) ) {
perfLimit [ " reason_id " ] = reason ;
}
return perfLimit ;
}
2019-09-11 05:27:22 +08:00
ACTOR static Future < JsonBuilderObject > workloadStatusFetcher ( Reference < AsyncVar < CachedSerialization < ServerDBInfo > > > db , vector < WorkerDetails > workers , WorkerDetails mWorker , WorkerDetails rkWorker ,
2019-02-08 05:39:22 +08:00
JsonBuilderObject * qos , JsonBuilderObject * data_overlay , std : : set < std : : string > * incomplete_reasons , Future < ErrorOr < vector < std : : pair < StorageServerInterface , EventMap > > > > storageServerFuture )
2018-05-05 03:01:40 +08:00
{
2018-09-09 06:44:48 +08:00
state JsonBuilderObject statusObj ;
state JsonBuilderObject operationsObj ;
state JsonBuilderObject bytesObj ;
state JsonBuilderObject keysObj ;
2017-05-26 04:48:44 +08:00
// Writes and conflicts
try {
2018-05-03 01:44:38 +08:00
vector < Future < TraceEventFields > > proxyStatFutures ;
2019-03-09 00:25:07 +08:00
std : : map < NetworkAddress , WorkerDetails > workersMap ;
2019-02-08 05:39:22 +08:00
for ( auto const & w : workers ) {
2019-03-09 00:25:07 +08:00
workersMap [ w . interf . address ( ) ] = w ;
2017-05-26 04:48:44 +08:00
}
2019-09-11 05:27:22 +08:00
for ( auto & p : db - > get ( ) . read ( ) . client . proxies ) {
2017-05-26 04:48:44 +08:00
auto worker = getWorker ( workersMap , p . address ( ) ) ;
if ( worker . present ( ) )
2019-03-09 00:25:07 +08:00
proxyStatFutures . push_back ( timeoutError ( worker . get ( ) . interf . eventLogRequest . getReply ( EventLogRequest ( LiteralStringRef ( " ProxyMetrics " ) ) ) , 1.0 ) ) ;
2017-05-26 04:48:44 +08:00
else
throw all_alternatives_failed ( ) ; // We need data from all proxies for this result to be trustworthy
}
2018-05-03 01:44:38 +08:00
vector < TraceEventFields > proxyStats = wait ( getAll ( proxyStatFutures ) ) ;
2017-05-26 04:48:44 +08:00
2019-07-12 04:54:44 +08:00
StatusCounter mutations ;
StatusCounter mutationBytes ;
StatusCounter txnConflicts ;
StatusCounter txnStartOut ;
StatusCounter txnSystemPriorityStartOut ;
StatusCounter txnDefaultPriorityStartOut ;
StatusCounter txnBatchPriorityStartOut ;
StatusCounter txnCommitOutSuccess ;
2017-05-26 04:48:44 +08:00
for ( auto & ps : proxyStats ) {
2018-08-31 16:21:24 +08:00
mutations . updateValues ( StatusCounter ( ps . getValue ( " Mutations " ) ) ) ;
mutationBytes . updateValues ( StatusCounter ( ps . getValue ( " MutationBytes " ) ) ) ;
txnConflicts . updateValues ( StatusCounter ( ps . getValue ( " TxnConflicts " ) ) ) ;
txnStartOut . updateValues ( StatusCounter ( ps . getValue ( " TxnStartOut " ) ) ) ;
2019-07-12 04:54:44 +08:00
txnSystemPriorityStartOut . updateValues ( StatusCounter ( ps . getValue ( " TxnSystemPriorityStartOut " ) ) ) ;
txnDefaultPriorityStartOut . updateValues ( StatusCounter ( ps . getValue ( " TxnDefaultPriorityStartOut " ) ) ) ;
txnBatchPriorityStartOut . updateValues ( StatusCounter ( ps . getValue ( " TxnBatchPriorityStartOut " ) ) ) ;
2018-08-31 16:21:24 +08:00
txnCommitOutSuccess . updateValues ( StatusCounter ( ps . getValue ( " TxnCommitOutSuccess " ) ) ) ;
2017-05-26 04:48:44 +08:00
}
2018-08-31 16:21:24 +08:00
operationsObj [ " writes " ] = mutations . getStatus ( ) ;
bytesObj [ " written " ] = mutationBytes . getStatus ( ) ;
2017-05-26 04:48:44 +08:00
2018-09-09 06:44:48 +08:00
JsonBuilderObject transactions ;
2018-08-31 16:21:24 +08:00
transactions [ " conflicted " ] = txnConflicts . getStatus ( ) ;
transactions [ " started " ] = txnStartOut . getStatus ( ) ;
2019-07-12 04:54:44 +08:00
transactions [ " started_immediate_priority " ] = txnSystemPriorityStartOut . getStatus ( ) ;
transactions [ " started_default_priority " ] = txnDefaultPriorityStartOut . getStatus ( ) ;
transactions [ " started_batch_priority " ] = txnBatchPriorityStartOut . getStatus ( ) ;
2018-08-31 16:21:24 +08:00
transactions [ " committed " ] = txnCommitOutSuccess . getStatus ( ) ;
2017-05-26 04:48:44 +08:00
statusObj [ " transactions " ] = transactions ;
}
catch ( Error & e ) {
if ( e . code ( ) = = error_code_actor_cancelled )
throw ;
incomplete_reasons - > insert ( " Unknown mutations, conflicts, and transactions state. " ) ;
}
2018-05-05 03:01:40 +08:00
// Transactions
2017-05-26 04:48:44 +08:00
try {
2019-03-13 09:31:25 +08:00
state TraceEventFields ratekeeper = wait ( timeoutError ( rkWorker . interf . eventLogRequest . getReply ( EventLogRequest ( LiteralStringRef ( " RkUpdate " ) ) ) , 1.0 ) ) ;
TraceEventFields batchRatekeeper = wait ( timeoutError ( rkWorker . interf . eventLogRequest . getReply ( EventLogRequest ( LiteralStringRef ( " RkUpdateBatch " ) ) ) , 1.0 ) ) ;
2019-03-01 01:53:16 +08:00
double tpsLimit = ratekeeper . getDouble ( " TPSLimit " ) ;
double batchTpsLimit = batchRatekeeper . getDouble ( " TPSLimit " ) ;
double transPerSec = ratekeeper . getDouble ( " ReleasedTPS " ) ;
2019-03-01 04:00:58 +08:00
double batchTransPerSec = ratekeeper . getDouble ( " ReleasedBatchTPS " ) ;
2019-03-01 01:53:16 +08:00
int ssCount = ratekeeper . getInt ( " StorageServers " ) ;
int tlogCount = ratekeeper . getInt ( " TLogs " ) ;
int64_t worstFreeSpaceStorageServer = ratekeeper . getInt64 ( " WorstFreeSpaceStorageServer " ) ;
int64_t worstFreeSpaceTLog = ratekeeper . getInt64 ( " WorstFreeSpaceTLog " ) ;
( * data_overlay ) . setKeyRawNumber ( " total_disk_used_bytes " , ratekeeper . getValue ( " TotalDiskUsageBytes " ) ) ;
2017-05-26 04:48:44 +08:00
if ( ssCount > 0 ) {
( * data_overlay ) [ " least_operating_space_bytes_storage_server " ] = std : : max ( worstFreeSpaceStorageServer , ( int64_t ) 0 ) ;
2019-03-01 01:53:16 +08:00
( * qos ) . setKeyRawNumber ( " worst_queue_bytes_storage_server " , ratekeeper . getValue ( " WorstStorageServerQueue " ) ) ;
( * qos ) . setKeyRawNumber ( " limiting_queue_bytes_storage_server " , ratekeeper . getValue ( " LimitingStorageServerQueue " ) ) ;
2019-08-16 04:42:39 +08:00
// TODO: These can be removed in the next release after 6.2
2019-03-01 01:53:16 +08:00
( * qos ) . setKeyRawNumber ( " worst_version_lag_storage_server " , ratekeeper . getValue ( " WorstStorageServerVersionLag " ) ) ;
( * qos ) . setKeyRawNumber ( " limiting_version_lag_storage_server " , ratekeeper . getValue ( " LimitingStorageServerVersionLag " ) ) ;
2019-08-16 04:42:39 +08:00
( * qos ) [ " worst_data_lag_storage_server " ] = getLagObject ( ratekeeper . getInt64 ( " WorstStorageServerVersionLag " ) ) ;
( * qos ) [ " limiting_data_lag_storage_server " ] = getLagObject ( ratekeeper . getInt64 ( " LimitingStorageServerVersionLag " ) ) ;
( * qos ) [ " worst_durability_lag_storage_server " ] = getLagObject ( ratekeeper . getInt64 ( " WorstStorageServerDurabilityLag " ) ) ;
( * qos ) [ " limiting_durability_lag_storage_server " ] = getLagObject ( ratekeeper . getInt64 ( " LimitingStorageServerDurabilityLag " ) ) ;
2017-05-26 04:48:44 +08:00
}
if ( tlogCount > 0 ) {
( * data_overlay ) [ " least_operating_space_bytes_log_server " ] = std : : max ( worstFreeSpaceTLog , ( int64_t ) 0 ) ;
2019-03-01 01:53:16 +08:00
( * qos ) . setKeyRawNumber ( " worst_queue_bytes_log_server " , ratekeeper . getValue ( " WorstTLogQueue " ) ) ;
2017-05-26 04:48:44 +08:00
}
( * qos ) [ " transactions_per_second_limit " ] = tpsLimit ;
2019-03-01 01:53:16 +08:00
( * qos ) [ " batch_transactions_per_second_limit " ] = batchTpsLimit ;
2017-05-26 04:48:44 +08:00
( * qos ) [ " released_transactions_per_second " ] = transPerSec ;
2019-03-01 04:00:58 +08:00
( * qos ) [ " batch_released_transactions_per_second " ] = batchTransPerSec ;
2017-05-26 04:48:44 +08:00
2019-03-01 01:53:16 +08:00
JsonBuilderObject perfLimit = getPerfLimit ( ratekeeper , transPerSec , tpsLimit ) ;
2017-05-26 04:48:44 +08:00
if ( ! perfLimit . empty ( ) ) {
( * qos ) [ " performance_limited_by " ] = perfLimit ;
}
2019-03-01 01:53:16 +08:00
JsonBuilderObject batchPerfLimit = getPerfLimit ( batchRatekeeper , transPerSec , batchTpsLimit ) ;
if ( ! batchPerfLimit . empty ( ) ) {
( * qos ) [ " batch_performance_limited_by " ] = batchPerfLimit ;
}
2017-05-26 04:48:44 +08:00
} catch ( Error & e ) {
if ( e . code ( ) = = error_code_actor_cancelled )
throw ;
2018-05-05 03:01:40 +08:00
incomplete_reasons - > insert ( " Unknown performance state. " ) ;
}
// Reads
try {
2019-02-08 05:39:22 +08:00
ErrorOr < vector < std : : pair < StorageServerInterface , EventMap > > > storageServers = wait ( storageServerFuture ) ;
2018-05-05 03:01:40 +08:00
if ( ! storageServers . present ( ) ) {
throw storageServers . getError ( ) ;
}
2018-09-28 06:32:39 +08:00
StatusCounter readRequests ;
2018-08-31 16:21:24 +08:00
StatusCounter reads ;
StatusCounter readKeys ;
StatusCounter readBytes ;
2018-05-05 03:01:40 +08:00
for ( auto & ss : storageServers . get ( ) ) {
2019-02-08 05:39:22 +08:00
TraceEventFields const & storageMetrics = ss . second . at ( " StorageMetrics " ) ;
2019-03-23 05:22:22 +08:00
if ( storageMetrics . size ( ) > 0 ) {
readRequests . updateValues ( StatusCounter ( storageMetrics . getValue ( " QueryQueue " ) ) ) ;
reads . updateValues ( StatusCounter ( storageMetrics . getValue ( " FinishedQueries " ) ) ) ;
readKeys . updateValues ( StatusCounter ( storageMetrics . getValue ( " RowsQueried " ) ) ) ;
readBytes . updateValues ( StatusCounter ( storageMetrics . getValue ( " BytesQueried " ) ) ) ;
}
2018-05-05 03:01:40 +08:00
}
2018-09-28 06:32:39 +08:00
operationsObj [ " read_requests " ] = readRequests . getStatus ( ) ;
2018-08-31 16:21:24 +08:00
operationsObj [ " reads " ] = reads . getStatus ( ) ;
keysObj [ " read " ] = readKeys . getStatus ( ) ;
bytesObj [ " read " ] = readBytes . getStatus ( ) ;
2018-05-05 03:01:40 +08:00
2017-05-26 04:48:44 +08:00
}
2018-05-05 03:01:40 +08:00
catch ( Error & e ) {
if ( e . code ( ) = = error_code_actor_cancelled )
throw ;
incomplete_reasons - > insert ( " Unknown read state. " ) ;
}
2017-05-26 04:48:44 +08:00
statusObj [ " operations " ] = operationsObj ;
2018-05-05 03:01:40 +08:00
statusObj [ " keys " ] = keysObj ;
statusObj [ " bytes " ] = bytesObj ;
2017-05-26 04:48:44 +08:00
return statusObj ;
}
2019-07-11 05:43:20 +08:00
ACTOR static Future < JsonBuilderObject > clusterSummaryStatisticsFetcher ( WorkerEvents pMetrics , Future < ErrorOr < vector < std : : pair < StorageServerInterface , EventMap > > > > storageServerFuture ,
Future < ErrorOr < vector < std : : pair < TLogInterface , EventMap > > > > tlogFuture , std : : set < std : : string > * incomplete_reasons )
{
state JsonBuilderObject statusObj ;
try {
state JsonBuilderObject cacheStatistics ;
ErrorOr < vector < std : : pair < StorageServerInterface , EventMap > > > storageServers = wait ( storageServerFuture ) ;
if ( ! storageServers . present ( ) ) {
throw storageServers . getError ( ) ;
}
double storageCacheHitsHz = 0 ;
double storageCacheMissesHz = 0 ;
for ( auto & ss : storageServers . get ( ) ) {
auto processMetrics = pMetrics . find ( ss . first . address ( ) ) ;
if ( processMetrics ! = pMetrics . end ( ) ) {
int64_t hits = processMetrics - > second . getInt64 ( " CacheHits " ) ;
int64_t misses = processMetrics - > second . getInt64 ( " CacheMisses " ) ;
double elapsed = processMetrics - > second . getDouble ( " Elapsed " ) ;
storageCacheHitsHz + = hits / elapsed ;
storageCacheMissesHz + = misses / elapsed ;
}
}
cacheStatistics [ " storage_hit_rate " ] = ( storageCacheMissesHz = = 0 ) ? 1.0 : storageCacheHitsHz / ( storageCacheHitsHz + storageCacheMissesHz ) ;
ErrorOr < vector < std : : pair < TLogInterface , EventMap > > > tlogServers = wait ( tlogFuture ) ;
if ( ! tlogServers . present ( ) ) {
throw tlogServers . getError ( ) ;
}
double logCacheHitsHz = 0 ;
double logCacheMissesHz = 0 ;
for ( auto & log : tlogServers . get ( ) ) {
auto processMetrics = pMetrics . find ( log . first . address ( ) ) ;
if ( processMetrics ! = pMetrics . end ( ) ) {
int64_t hits = processMetrics - > second . getInt64 ( " CacheHits " ) ;
int64_t misses = processMetrics - > second . getInt64 ( " CacheMisses " ) ;
double elapsed = processMetrics - > second . getDouble ( " Elapsed " ) ;
logCacheHitsHz + = hits / elapsed ;
logCacheMissesHz + = misses / elapsed ;
}
}
cacheStatistics [ " log_hit_rate " ] = ( logCacheMissesHz = = 0 ) ? 1.0 : logCacheHitsHz / ( logCacheHitsHz + logCacheMissesHz ) ;
statusObj [ " page_cache " ] = cacheStatistics ;
}
catch ( Error & e ) {
if ( e . code ( ) = = error_code_actor_cancelled )
throw ;
incomplete_reasons - > insert ( " Unknown cache statistics. " ) ;
}
return statusObj ;
}
2019-09-11 05:27:22 +08:00
static JsonBuilderArray oldTlogFetcher ( int * oldLogFaultTolerance , Reference < AsyncVar < CachedSerialization < ServerDBInfo > > > db , std : : unordered_map < NetworkAddress , WorkerInterface > const & address_workers ) {
2018-09-09 06:44:48 +08:00
JsonBuilderArray oldTlogsArray ;
2017-05-26 04:48:44 +08:00
2019-09-11 05:27:22 +08:00
if ( db - > get ( ) . read ( ) . recoveryState > = RecoveryState : : ACCEPTING_COMMITS ) {
for ( auto it : db - > get ( ) . read ( ) . logSystemConfig . oldTLogs ) {
2018-09-09 06:44:48 +08:00
JsonBuilderObject statusObj ;
JsonBuilderArray logsObj ;
2018-09-05 18:50:53 +08:00
Optional < int32_t > sat_log_replication_factor , sat_log_write_anti_quorum , sat_log_fault_tolerance , log_replication_factor , log_write_anti_quorum , log_fault_tolerance , remote_log_replication_factor , remote_log_fault_tolerance ;
2017-07-11 08:41:32 +08:00
int maxFaultTolerance = 0 ;
2018-08-31 16:21:24 +08:00
2017-07-11 08:41:32 +08:00
for ( int i = 0 ; i < it . tLogs . size ( ) ; i + + ) {
int failedLogs = 0 ;
for ( auto & log : it . tLogs [ i ] . tLogs ) {
2018-09-09 06:44:48 +08:00
JsonBuilderObject logObj ;
2017-07-11 08:41:32 +08:00
bool failed = ! log . present ( ) | | ! address_workers . count ( log . interf ( ) . address ( ) ) ;
logObj [ " id " ] = log . id ( ) . shortString ( ) ;
logObj [ " healthy " ] = ! failed ;
if ( log . present ( ) ) {
logObj [ " address " ] = log . interf ( ) . address ( ) . toString ( ) ;
}
logsObj . push_back ( logObj ) ;
if ( failed ) {
failedLogs + + ;
}
2017-05-26 04:48:44 +08:00
}
2017-07-11 08:41:32 +08:00
maxFaultTolerance = std : : max ( maxFaultTolerance , it . tLogs [ i ] . tLogReplicationFactor - 1 - it . tLogs [ i ] . tLogWriteAntiQuorum - failedLogs ) ;
2018-06-16 03:36:19 +08:00
if ( it . tLogs [ i ] . isLocal & & it . tLogs [ i ] . locality = = tagLocalitySatellite ) {
2018-09-05 18:50:53 +08:00
sat_log_replication_factor = it . tLogs [ i ] . tLogReplicationFactor ;
sat_log_write_anti_quorum = it . tLogs [ i ] . tLogWriteAntiQuorum ;
sat_log_fault_tolerance = it . tLogs [ i ] . tLogReplicationFactor - 1 - it . tLogs [ i ] . tLogWriteAntiQuorum - failedLogs ;
2018-06-16 03:36:19 +08:00
}
else if ( it . tLogs [ i ] . isLocal ) {
2018-09-05 18:50:53 +08:00
log_replication_factor = it . tLogs [ i ] . tLogReplicationFactor ;
log_write_anti_quorum = it . tLogs [ i ] . tLogWriteAntiQuorum ;
log_fault_tolerance = it . tLogs [ i ] . tLogReplicationFactor - 1 - it . tLogs [ i ] . tLogWriteAntiQuorum - failedLogs ;
2017-05-26 04:48:44 +08:00
}
2018-06-16 03:36:19 +08:00
else {
2018-09-05 18:50:53 +08:00
remote_log_replication_factor = it . tLogs [ i ] . tLogReplicationFactor ;
remote_log_fault_tolerance = it . tLogs [ i ] . tLogReplicationFactor - 1 - failedLogs ;
2018-06-15 08:58:57 +08:00
}
2017-05-26 04:48:44 +08:00
}
2017-07-11 08:41:32 +08:00
* oldLogFaultTolerance = std : : min ( * oldLogFaultTolerance , maxFaultTolerance ) ;
2017-05-26 04:48:44 +08:00
statusObj [ " logs " ] = logsObj ;
2018-09-05 18:50:53 +08:00
if ( sat_log_replication_factor . present ( ) )
statusObj [ " satellite_log_replication_factor " ] = sat_log_replication_factor . get ( ) ;
if ( sat_log_write_anti_quorum . present ( ) )
statusObj [ " satellite_log_write_anti_quorum " ] = sat_log_write_anti_quorum . get ( ) ;
if ( sat_log_fault_tolerance . present ( ) )
statusObj [ " satellite_log_fault_tolerance " ] = sat_log_fault_tolerance . get ( ) ;
if ( log_replication_factor . present ( ) )
statusObj [ " log_replication_factor " ] = log_replication_factor . get ( ) ;
if ( log_write_anti_quorum . present ( ) )
statusObj [ " log_write_anti_quorum " ] = log_write_anti_quorum . get ( ) ;
if ( log_fault_tolerance . present ( ) )
statusObj [ " log_fault_tolerance " ] = log_fault_tolerance . get ( ) ;
if ( remote_log_replication_factor . present ( ) )
statusObj [ " remote_log_replication_factor " ] = remote_log_replication_factor . get ( ) ;
if ( remote_log_fault_tolerance . present ( ) )
statusObj [ " remote_log_fault_tolerance " ] = remote_log_fault_tolerance . get ( ) ;
2017-05-26 04:48:44 +08:00
oldTlogsArray . push_back ( statusObj ) ;
}
}
return oldTlogsArray ;
}
2019-07-31 05:02:31 +08:00
static JsonBuilderObject faultToleranceStatusFetcher ( DatabaseConfiguration configuration , ServerCoordinators coordinators , std : : vector < WorkerDetails > & workers , int extraTlogEligibleZones , int minReplicasRemaining , bool underMaintenance ) {
2018-09-09 06:44:48 +08:00
JsonBuilderObject statusObj ;
2017-05-26 04:48:44 +08:00
// without losing data
2019-07-31 05:02:31 +08:00
int32_t maxZoneFailures = configuration . maxZoneFailuresTolerated ( ) ;
2019-04-03 05:15:51 +08:00
if ( underMaintenance ) {
2019-07-31 05:02:31 +08:00
maxZoneFailures - - ;
2019-04-03 05:15:51 +08:00
}
2017-05-26 04:48:44 +08:00
int maxCoordinatorFailures = ( coordinators . clientLeaderServers . size ( ) - 1 ) / 2 ;
std : : map < NetworkAddress , StringRef > workerZones ;
for ( auto & worker : workers ) {
2019-03-09 00:25:07 +08:00
workerZones [ worker . interf . address ( ) ] = worker . interf . locality . zoneId ( ) . orDefault ( LiteralStringRef ( " " ) ) ;
2017-05-26 04:48:44 +08:00
}
std : : map < StringRef , int > coordinatorZoneCounts ;
for ( auto & coordinator : coordinators . ccf - > getConnectionString ( ) . coordinators ( ) ) {
auto zone = workerZones [ coordinator ] ;
coordinatorZoneCounts [ zone ] + = 1 ;
}
std : : vector < std : : pair < StringRef , int > > coordinatorZones ( coordinatorZoneCounts . begin ( ) , coordinatorZoneCounts . end ( ) ) ;
std : : sort ( coordinatorZones . begin ( ) , coordinatorZones . end ( ) , [ ] ( const std : : pair < StringRef , int > & lhs , const std : : pair < StringRef , int > & rhs ) {
return lhs . second > rhs . second ;
} ) ;
int lostCoordinators = 0 ;
int maxCoordinatorZoneFailures = 0 ;
for ( auto zone : coordinatorZones ) {
lostCoordinators + = zone . second ;
if ( lostCoordinators > maxCoordinatorFailures ) {
break ;
}
maxCoordinatorZoneFailures + = 1 ;
}
2019-07-31 05:02:31 +08:00
int zoneFailuresWithoutLosingData = std : : min ( maxZoneFailures , maxCoordinatorZoneFailures ) ;
2017-05-26 04:48:44 +08:00
if ( minReplicasRemaining > = 0 ) {
2019-07-31 05:02:31 +08:00
zoneFailuresWithoutLosingData = std : : min ( zoneFailuresWithoutLosingData , minReplicasRemaining - 1 ) ;
2017-05-26 04:48:44 +08:00
}
2019-07-31 05:02:31 +08:00
statusObj [ " max_zone_failures_without_losing_data " ] = std : : max ( zoneFailuresWithoutLosingData , 0 ) ;
2017-05-26 04:48:44 +08:00
// without losing availablity
2019-07-31 05:02:31 +08:00
statusObj [ " max_zone_failures_without_losing_availability " ] = std : : max ( std : : min ( extraTlogEligibleZones , zoneFailuresWithoutLosingData ) , 0 ) ;
2017-05-26 04:48:44 +08:00
return statusObj ;
}
static std : : string getIssueDescription ( std : : string name ) {
2018-01-06 02:29:47 +08:00
if ( name = = " incorrect_cluster_file_contents " ) {
2018-06-13 02:59:47 +08:00
return " Cluster file contents do not match current cluster connection string. Verify the cluster file and its parent directory are writable and that the cluster file has not been overwritten externally. " ;
2017-05-26 04:48:44 +08:00
}
2018-01-06 02:29:47 +08:00
// FIXME: name and description will be the same unless the message is 'incorrect_cluster_file_contents', which is currently the only possible message
2017-05-26 04:48:44 +08:00
return name ;
}
2019-03-22 23:29:41 +08:00
static std : : map < std : : string , std : : vector < JsonBuilderObject > > getProcessIssuesAsMessages (
ProcessIssuesMap const & _issues ) {
std : : map < std : : string , std : : vector < JsonBuilderObject > > issuesMap ;
2017-05-26 04:48:44 +08:00
try {
ProcessIssuesMap issues = _issues ;
2019-03-22 23:29:41 +08:00
for ( auto processIssues : issues ) {
for ( auto issue : processIssues . second . first ) {
std : : string issueStr = issue . toString ( ) ;
issuesMap [ processIssues . first . toString ( ) ] . push_back (
JsonString : : makeMessage ( issueStr . c_str ( ) , getIssueDescription ( issueStr ) . c_str ( ) ) ) ;
}
2017-05-26 04:48:44 +08:00
}
}
catch ( Error & e ) {
TraceEvent ( SevError , " ErrorParsingProcessIssues " ) . error ( e ) ;
// swallow
}
return issuesMap ;
}
2019-07-26 08:15:31 +08:00
static JsonBuilderArray getClientIssuesAsMessages ( std : : map < NetworkAddress , std : : pair < double , OpenDatabaseRequest > > * clientStatusMap ) {
2018-09-09 06:44:48 +08:00
JsonBuilderArray issuesList ;
2017-05-26 04:48:44 +08:00
try {
2019-07-26 08:15:31 +08:00
std : : map < std : : string , std : : pair < int , std : : vector < std : : string > > > deduplicatedIssues ;
2019-09-06 02:36:34 +08:00
for ( auto iter = clientStatusMap - > begin ( ) ; iter ! = clientStatusMap - > end ( ) ; ) {
if ( now ( ) - iter - > second . first < 2 * SERVER_KNOBS - > COORDINATOR_REGISTER_INTERVAL ) {
2019-07-26 08:15:31 +08:00
for ( auto & issue : iter - > second . second . issues ) {
auto & t = deduplicatedIssues [ issue . item . toString ( ) ] ;
t . first + = issue . count ;
for ( auto & example : issue . examples ) {
t . second . push_back ( formatIpPort ( example . first . ip , example . first . port ) ) ;
}
}
2019-09-06 02:36:34 +08:00
+ + iter ;
2019-07-26 08:15:31 +08:00
} else {
iter = clientStatusMap - > erase ( iter ) ;
2019-03-22 23:29:41 +08:00
}
2017-05-26 04:48:44 +08:00
}
2019-07-27 04:23:56 +08:00
//FIXME: add the log_group in addition to the network address
2017-05-26 04:48:44 +08:00
for ( auto i : deduplicatedIssues ) {
2018-09-09 06:44:48 +08:00
JsonBuilderObject message = JsonString : : makeMessage ( i . first . c_str ( ) , getIssueDescription ( i . first ) . c_str ( ) ) ;
JsonBuilderArray addresses ;
2019-07-26 08:15:31 +08:00
for ( auto addr : i . second . second ) {
2018-09-08 22:15:28 +08:00
addresses . push_back ( addr ) ;
2017-05-26 04:48:44 +08:00
}
2019-07-26 08:15:31 +08:00
message [ " count " ] = i . second . first ;
2017-05-26 04:48:44 +08:00
message [ " addresses " ] = addresses ;
issuesList . push_back ( message ) ;
}
}
catch ( Error & e ) {
TraceEvent ( SevError , " ErrorParsingClientIssues " ) . error ( e ) ;
// swallow
}
return issuesList ;
}
2018-09-09 06:44:48 +08:00
ACTOR Future < JsonBuilderObject > layerStatusFetcher ( Database cx , JsonBuilderArray * messages , std : : set < std : : string > * incomplete_reasons ) {
2018-09-05 18:50:53 +08:00
state StatusObject result ;
state JSONDoc json ( result ) ;
state double tStart = now ( ) ;
2017-05-26 04:48:44 +08:00
try {
state ReadYourWritesTransaction tr ( cx ) ;
loop {
try {
tr . setOption ( FDBTransactionOptions : : ACCESS_SYSTEM_KEYS ) ;
int64_t timeout_ms = 3000 ;
tr . setOption ( FDBTransactionOptions : : TIMEOUT , StringRef ( ( uint8_t * ) & timeout_ms , sizeof ( int64_t ) ) ) ;
std : : string jsonPrefix = layerStatusMetaPrefixRange . begin . toString ( ) + " json/ " ;
Standalone < RangeResultRef > jsonLayers = wait ( tr . getRange ( KeyRangeRef ( jsonPrefix , strinc ( jsonPrefix ) ) , 1000 ) ) ;
// TODO: Also fetch other linked subtrees of meta keys
state std : : vector < Future < Standalone < RangeResultRef > > > docFutures ;
state int i ;
for ( i = 0 ; i < jsonLayers . size ( ) ; + + i )
docFutures . push_back ( tr . getRange ( KeyRangeRef ( jsonLayers [ i ] . value , strinc ( jsonLayers [ i ] . value ) ) , 1000 ) ) ;
result . clear ( ) ;
JSONDoc : : expires_reference_version = ( uint64_t ) tr . getReadVersion ( ) . get ( ) ;
for ( i = 0 ; i < docFutures . size ( ) ; + + i ) {
state Standalone < RangeResultRef > docs = wait ( docFutures [ i ] ) ;
state int j ;
for ( j = 0 ; j < docs . size ( ) ; + + j ) {
state json_spirit : : mValue doc ;
try {
json_spirit : : read_string ( docs [ j ] . value . toString ( ) , doc ) ;
2018-08-11 04:57:10 +08:00
wait ( yield ( ) ) ;
2017-05-26 04:48:44 +08:00
json . absorb ( doc . get_obj ( ) ) ;
2018-08-11 04:57:10 +08:00
wait ( yield ( ) ) ;
2017-05-26 04:48:44 +08:00
} catch ( Error & e ) {
2019-03-19 06:03:43 +08:00
TraceEvent ( SevWarn , " LayerStatusBadJSON " ) . detail ( " Key " , docs [ j ] . key ) ;
2017-05-26 04:48:44 +08:00
}
}
}
json . create ( " _valid " ) = true ;
break ;
} catch ( Error & e ) {
2018-08-11 04:57:10 +08:00
wait ( tr . onError ( e ) ) ;
2017-05-26 04:48:44 +08:00
}
}
} catch ( Error & e ) {
TraceEvent ( SevWarn , " LayerStatusError " ) . error ( e ) ;
incomplete_reasons - > insert ( format ( " Unable to retrieve layer status (%s). " , e . what ( ) ) ) ;
json . create ( " _error " ) = format ( " Unable to retrieve layer status (%s). " , e . what ( ) ) ;
json . create ( " _valid " ) = false ;
}
json . cleanOps ( ) ;
2018-09-09 06:44:48 +08:00
JsonBuilderObject statusObj ;
statusObj . addContents ( result ) ;
2018-09-08 22:15:28 +08:00
TraceEvent ( " LayerStatusFetcher " ) . detail ( " Duration " , now ( ) - tStart ) . detail ( " StatusSize " , statusObj . getFinalLength ( ) ) ;
2018-09-05 18:50:53 +08:00
return statusObj ;
2017-05-26 04:48:44 +08:00
}
2019-09-11 05:27:22 +08:00
ACTOR Future < JsonBuilderObject > lockedStatusFetcher ( Reference < AsyncVar < CachedSerialization < ServerDBInfo > > > db , JsonBuilderArray * messages , std : : set < std : : string > * incomplete_reasons ) {
2018-09-09 06:44:48 +08:00
state JsonBuilderObject statusObj ;
2017-05-26 04:48:44 +08:00
2019-06-25 17:47:35 +08:00
state Database cx = openDBOnServer ( db , TaskPriority : : DefaultEndpoint , true , false ) ; // Open a new database connection that isn't lock-aware
2017-05-26 04:48:44 +08:00
state Transaction tr ( cx ) ;
state int timeoutSeconds = 5 ;
state Future < Void > getTimeout = delay ( timeoutSeconds ) ;
loop {
tr . setOption ( FDBTransactionOptions : : PRIORITY_SYSTEM_IMMEDIATE ) ;
2019-09-07 13:11:41 +08:00
tr . setOption ( FDBTransactionOptions : : READ_SYSTEM_KEYS ) ;
2019-09-07 13:12:24 +08:00
tr . setOption ( FDBTransactionOptions : : READ_LOCK_AWARE ) ;
2017-05-26 04:48:44 +08:00
try {
choose {
2019-09-05 06:33:46 +08:00
when ( Optional < Value > lockUID = wait ( tr . get ( databaseLockedKey ) ) ) {
if ( lockUID . present ( ) ) {
statusObj [ " locked " ] = true ;
2019-09-07 12:58:35 +08:00
statusObj [ " lock_uid " ] =
2019-09-05 06:33:46 +08:00
BinaryReader : : fromStringRef < UID > ( lockUID . get ( ) . substr ( 10 ) , Unversioned ( ) ) . toString ( ) ;
} else {
statusObj [ " locked " ] = false ;
}
}
2018-08-11 04:57:10 +08:00
when ( wait ( getTimeout ) ) {
2017-05-26 04:48:44 +08:00
incomplete_reasons - > insert ( format ( " Unable to determine if database is locked after %d seconds. " , timeoutSeconds ) ) ;
}
}
break ;
}
catch ( Error & e ) {
2019-09-05 06:33:46 +08:00
try {
wait ( tr . onError ( e ) ) ;
} catch ( Error & e ) {
incomplete_reasons - > insert ( format ( " Unable to determine if database is locked (%s). " , e . what ( ) ) ) ;
2017-05-26 04:48:44 +08:00
break ;
}
}
}
return statusObj ;
}
// constructs the cluster section of the json status output
ACTOR Future < StatusReply > clusterGetStatus (
2019-09-11 05:27:22 +08:00
Reference < AsyncVar < CachedSerialization < ServerDBInfo > > > db ,
2017-05-26 04:48:44 +08:00
Database cx ,
2019-03-09 00:25:07 +08:00
vector < WorkerDetails > workers ,
2017-05-26 04:48:44 +08:00
ProcessIssuesMap workerIssues ,
2019-07-26 08:15:31 +08:00
std : : map < NetworkAddress , std : : pair < double , OpenDatabaseRequest > > * clientStatus ,
2017-05-26 04:48:44 +08:00
ServerCoordinators coordinators ,
2018-06-15 10:09:25 +08:00
std : : vector < NetworkAddress > incompatibleConnections ,
Version datacenterVersionDifference )
2017-05-26 04:48:44 +08:00
{
2018-09-06 13:51:15 +08:00
state double tStart = timer ( ) ;
2017-05-26 04:48:44 +08:00
// Check if master worker is present
2018-09-09 06:44:48 +08:00
state JsonBuilderArray messages ;
2017-05-26 04:48:44 +08:00
state std : : set < std : : string > status_incomplete_reasons ;
2019-03-09 00:25:07 +08:00
state WorkerDetails mWorker ;
state WorkerDetails ddWorker ; // DataDistributor worker
2019-03-27 23:24:25 +08:00
state WorkerDetails rkWorker ; // Ratekeeper worker
2017-05-26 04:48:44 +08:00
try {
// Get the master Worker interface
2019-09-11 05:27:22 +08:00
Optional < WorkerDetails > _mWorker = getWorker ( workers , db - > get ( ) . read ( ) . master . address ( ) ) ;
2017-05-26 04:48:44 +08:00
if ( _mWorker . present ( ) ) {
mWorker = _mWorker . get ( ) ;
} else {
2018-08-31 16:21:24 +08:00
messages . push_back ( JsonString : : makeMessage ( " unreachable_master_worker " , " Unable to locate the master worker. " ) ) ;
2017-05-26 04:48:44 +08:00
}
2019-02-22 10:05:46 +08:00
// Get the DataDistributor worker interface
2019-03-09 00:25:07 +08:00
Optional < WorkerDetails > _ddWorker ;
2019-09-11 05:27:22 +08:00
if ( db - > get ( ) . read ( ) . distributor . present ( ) ) {
_ddWorker = getWorker ( workers , db - > get ( ) . read ( ) . distributor . get ( ) . address ( ) ) ;
2019-02-22 10:05:46 +08:00
}
2019-09-11 05:27:22 +08:00
if ( ! db - > get ( ) . read ( ) . distributor . present ( ) | | ! _ddWorker . present ( ) ) {
2019-02-22 10:05:46 +08:00
messages . push_back ( JsonString : : makeMessage ( " unreachable_dataDistributor_worker " , " Unable to locate the data distributor worker. " ) ) ;
} else {
ddWorker = _ddWorker . get ( ) ;
}
2017-05-26 04:48:44 +08:00
2019-03-27 23:24:25 +08:00
// Get the Ratekeeper worker interface
2019-03-13 09:31:25 +08:00
Optional < WorkerDetails > _rkWorker ;
2019-09-11 05:27:22 +08:00
if ( db - > get ( ) . read ( ) . ratekeeper . present ( ) ) {
_rkWorker = getWorker ( workers , db - > get ( ) . read ( ) . ratekeeper . get ( ) . address ( ) ) ;
2019-02-23 07:04:38 +08:00
}
2019-09-11 05:27:22 +08:00
if ( ! db - > get ( ) . read ( ) . ratekeeper . present ( ) | | ! _rkWorker . present ( ) ) {
2019-02-23 07:04:38 +08:00
messages . push_back ( JsonString : : makeMessage ( " unreachable_ratekeeper_worker " , " Unable to locate the ratekeeper worker. " ) ) ;
} else {
rkWorker = _rkWorker . get ( ) ;
}
2017-05-26 04:48:44 +08:00
// Get latest events for various event types from ALL workers
// WorkerEvents is a map of worker's NetworkAddress to its event string
// The pair represents worker responses and a set of worker NetworkAddress strings which did not respond
std : : vector < Future < Optional < std : : pair < WorkerEvents , std : : set < std : : string > > > > > futures ;
futures . push_back ( latestEventOnWorkers ( workers , " MachineMetrics " ) ) ;
futures . push_back ( latestEventOnWorkers ( workers , " ProcessMetrics " ) ) ;
2019-06-27 05:03:02 +08:00
futures . push_back ( latestEventOnWorkers ( workers , " NetworkMetrics " ) ) ;
2017-05-26 04:48:44 +08:00
futures . push_back ( latestErrorOnWorkers ( workers ) ) ;
futures . push_back ( latestEventOnWorkers ( workers , " TraceFileOpenError " ) ) ;
futures . push_back ( latestEventOnWorkers ( workers , " ProgramStart " ) ) ;
// Wait for all response pairs.
state std : : vector < Optional < std : : pair < WorkerEvents , std : : set < std : : string > > > > workerEventsVec = wait ( getAll ( futures ) ) ;
// Create a unique set of all workers who were unreachable for 1 or more of the event requests above.
// Since each event request is independent and to all workers, workers can have responded to some
// event requests but still end up in the unreachable set.
std : : set < std : : string > mergeUnreachable ;
// For each (optional) pair, if the pair is present and not empty then add the unreachable workers to the set.
for ( auto pair : workerEventsVec )
{
if ( pair . present ( ) & & pair . get ( ) . second . size ( ) )
mergeUnreachable . insert ( pair . get ( ) . second . begin ( ) , pair . get ( ) . second . end ( ) ) ;
}
// We now have a unique set of workers who were in some way unreachable. If there is anything in that set, create a message
// for it and include the list of unreachable processes.
if ( mergeUnreachable . size ( ) ) {
2018-09-09 06:44:48 +08:00
JsonBuilderObject message = JsonBuilder : : makeMessage ( " unreachable_processes " , " The cluster has some unreachable processes. " ) ;
JsonBuilderArray unreachableProcs ;
2017-05-26 04:48:44 +08:00
for ( auto m : mergeUnreachable ) {
2018-09-09 06:44:48 +08:00
unreachableProcs . push_back ( JsonBuilderObject ( ) . setKey ( " address " , m ) ) ;
2017-05-26 04:48:44 +08:00
}
message [ " unreachable_processes " ] = unreachableProcs ;
messages . push_back ( message ) ;
}
// construct status information for cluster subsections
2018-08-31 16:21:24 +08:00
state int statusCode = ( int ) RecoveryStatus : : END ;
2018-09-09 06:44:48 +08:00
state JsonBuilderObject recoveryStateStatus = wait ( recoveryStateStatusFetcher ( mWorker , workers . size ( ) , & status_incomplete_reasons , & statusCode ) ) ;
2017-05-26 04:48:44 +08:00
// machine metrics
state WorkerEvents mMetrics = workerEventsVec [ 0 ] . present ( ) ? workerEventsVec [ 0 ] . get ( ) . first : WorkerEvents ( ) ;
// process metrics
state WorkerEvents pMetrics = workerEventsVec [ 1 ] . present ( ) ? workerEventsVec [ 1 ] . get ( ) . first : WorkerEvents ( ) ;
2019-06-27 05:03:02 +08:00
state WorkerEvents networkMetrics = workerEventsVec [ 2 ] . present ( ) ? workerEventsVec [ 2 ] . get ( ) . first : WorkerEvents ( ) ;
state WorkerEvents latestError = workerEventsVec [ 3 ] . present ( ) ? workerEventsVec [ 3 ] . get ( ) . first : WorkerEvents ( ) ;
state WorkerEvents traceFileOpenErrors = workerEventsVec [ 4 ] . present ( ) ? workerEventsVec [ 4 ] . get ( ) . first : WorkerEvents ( ) ;
state WorkerEvents programStarts = workerEventsVec [ 5 ] . present ( ) ? workerEventsVec [ 5 ] . get ( ) . first : WorkerEvents ( ) ;
2017-05-26 04:48:44 +08:00
2018-09-09 06:44:48 +08:00
state JsonBuilderObject statusObj ;
2019-09-11 05:27:22 +08:00
if ( db - > get ( ) . read ( ) . recoveryCount > 0 ) {
statusObj [ " generation " ] = db - > get ( ) . read ( ) . recoveryCount ;
2017-05-26 04:48:44 +08:00
}
2019-03-22 23:29:41 +08:00
state std : : map < std : : string , std : : vector < JsonBuilderObject > > processIssues =
getProcessIssuesAsMessages ( workerIssues ) ;
2019-02-08 05:39:22 +08:00
state vector < std : : pair < StorageServerInterface , EventMap > > storageServers ;
state vector < std : : pair < TLogInterface , EventMap > > tLogs ;
state vector < std : : pair < MasterProxyInterface , EventMap > > proxies ;
2018-09-09 06:44:48 +08:00
state JsonBuilderObject qos ;
state JsonBuilderObject data_overlay ;
2017-05-26 04:48:44 +08:00
2019-08-02 01:19:46 +08:00
statusObj [ " protocol_version " ] = format ( " % " PRIx64 , currentProtocolVersion . version ( ) ) ;
2018-07-10 13:11:58 +08:00
statusObj [ " connection_string " ] = coordinators . ccf - > getConnectionString ( ) . toString ( ) ;
2017-05-26 04:48:44 +08:00
2018-06-15 12:14:18 +08:00
state Optional < DatabaseConfiguration > configuration ;
2019-04-03 05:15:51 +08:00
state Optional < LoadConfigurationResult > loadResult ;
2017-05-26 04:48:44 +08:00
2018-09-06 13:51:15 +08:00
if ( statusCode ! = RecoveryStatus : : configuration_missing ) {
2019-04-03 05:15:51 +08:00
std : : pair < Optional < DatabaseConfiguration > , Optional < LoadConfigurationResult > > loadResults = wait ( loadConfiguration ( cx , & messages , & status_incomplete_reasons ) ) ;
2018-06-15 12:14:18 +08:00
configuration = loadResults . first ;
2019-04-03 05:15:51 +08:00
loadResult = loadResults . second ;
2018-06-15 12:14:18 +08:00
}
2019-04-03 05:15:51 +08:00
if ( loadResult . present ( ) ) {
statusObj [ " full_replication " ] = loadResult . get ( ) . fullReplication ;
if ( loadResult . get ( ) . healthyZone . present ( ) ) {
2019-07-19 04:18:36 +08:00
if ( loadResult . get ( ) . healthyZone . get ( ) ! = ignoreSSFailuresZoneString ) {
2019-07-12 05:53:00 +08:00
statusObj [ " maintenance_zone " ] = loadResult . get ( ) . healthyZone . get ( ) . printable ( ) ;
statusObj [ " maintenance_seconds_remaining " ] = loadResult . get ( ) . healthyZoneSeconds ;
} else {
2019-07-17 06:20:58 +08:00
statusObj [ " data_distribution_disabled_for_ss_failures " ] = true ;
2019-07-12 05:53:00 +08:00
}
}
if ( loadResult . get ( ) . rebalanceDDIgnored ) {
2019-07-17 06:20:58 +08:00
statusObj [ " data_distribution_disabled_for_rebalance " ] = true ;
2019-04-03 05:15:51 +08:00
}
2019-08-30 09:41:34 +08:00
if ( loadResult . get ( ) . dataDistributionDisabled ) {
statusObj [ " data_distribution_disabled " ] = true ;
2019-04-03 05:15:51 +08:00
}
2017-05-26 04:48:44 +08:00
}
statusObj [ " machines " ] = machineStatusFetcher ( mMetrics , workers , configuration , & status_incomplete_reasons ) ;
if ( configuration . present ( ) ) {
// Do the latency probe by itself to avoid interference from other status activities
2018-09-06 13:51:15 +08:00
state bool isAvailable = true ;
2018-09-09 06:44:48 +08:00
JsonBuilderObject latencyProbeResults = wait ( latencyProbeFetcher ( cx , & messages , & status_incomplete_reasons , & isAvailable ) ) ;
2017-05-26 04:48:44 +08:00
2018-09-06 13:51:15 +08:00
statusObj [ " database_available " ] = isAvailable ;
2017-05-26 04:48:44 +08:00
if ( ! latencyProbeResults . empty ( ) ) {
statusObj [ " latency_probe " ] = latencyProbeResults ;
}
2019-10-03 08:06:19 +08:00
state std : : vector < Future < Void > > warningFutures ;
if ( isAvailable ) {
warningFutures . push_back ( consistencyCheckStatusFetcher ( cx , & messages , & status_incomplete_reasons ) ) ;
if ( ! SERVER_KNOBS - > DISABLE_DUPLICATE_LOG_WARNING ) {
warningFutures . push_back ( logRangeWarningFetcher ( cx , & messages , & status_incomplete_reasons ) ) ;
}
}
2019-06-21 12:38:45 +08:00
2017-05-26 04:48:44 +08:00
// Start getting storage servers now (using system priority) concurrently. Using sys priority because having storage servers
// in status output is important to give context to error messages in status that reference a storage server role ID.
state std : : unordered_map < NetworkAddress , WorkerInterface > address_workers ;
2019-02-08 05:39:22 +08:00
for ( auto const & worker : workers ) {
2019-03-09 00:25:07 +08:00
address_workers [ worker . interf . address ( ) ] = worker . interf ;
2018-12-01 02:46:04 +08:00
}
2019-02-08 05:39:22 +08:00
state Future < ErrorOr < vector < std : : pair < StorageServerInterface , EventMap > > > > storageServerFuture = errorOr ( getStorageServersAndMetrics ( cx , address_workers ) ) ;
state Future < ErrorOr < vector < std : : pair < TLogInterface , EventMap > > > > tLogFuture = errorOr ( getTLogsAndMetrics ( db , address_workers ) ) ;
2020-02-08 07:44:02 +08:00
state Future < ErrorOr < vector < std : : pair < MasterProxyInterface , EventMap > > > > proxyFuture = errorOr ( getProxiesAndMetrics ( db , address_workers ) ) ;
2017-05-26 04:48:44 +08:00
2018-05-05 03:01:40 +08:00
state int minReplicasRemaining = - 1 ;
2018-09-09 06:44:48 +08:00
std : : vector < Future < JsonBuilderObject > > futures2 ;
2019-10-11 01:36:35 +08:00
futures2 . push_back ( dataStatusFetcher ( ddWorker , configuration . get ( ) , & minReplicasRemaining ) ) ;
2019-02-23 07:04:38 +08:00
futures2 . push_back ( workloadStatusFetcher ( db , workers , mWorker , rkWorker , & qos , & data_overlay , & status_incomplete_reasons , storageServerFuture ) ) ;
2018-05-05 03:01:40 +08:00
futures2 . push_back ( layerStatusFetcher ( cx , & messages , & status_incomplete_reasons ) ) ;
futures2 . push_back ( lockedStatusFetcher ( db , & messages , & status_incomplete_reasons ) ) ;
2019-07-11 05:43:20 +08:00
futures2 . push_back ( clusterSummaryStatisticsFetcher ( pMetrics , storageServerFuture , tLogFuture , & status_incomplete_reasons ) ) ;
2018-09-09 06:44:48 +08:00
state std : : vector < JsonBuilderObject > workerStatuses = wait ( getAll ( futures2 ) ) ;
2017-05-26 04:48:44 +08:00
int oldLogFaultTolerance = 100 ;
2019-09-11 05:27:22 +08:00
if ( db - > get ( ) . read ( ) . recoveryState > = RecoveryState : : ACCEPTING_COMMITS & & db - > get ( ) . read ( ) . logSystemConfig . oldTLogs . size ( ) > 0 ) {
2017-05-26 04:48:44 +08:00
statusObj [ " old_logs " ] = oldTlogFetcher ( & oldLogFaultTolerance , db , address_workers ) ;
}
if ( configuration . present ( ) ) {
2019-07-31 05:02:31 +08:00
int extraTlogEligibleZones = getExtraTLogEligibleZones ( workers , configuration . get ( ) ) ;
statusObj [ " fault_tolerance " ] = faultToleranceStatusFetcher ( configuration . get ( ) , coordinators , workers , extraTlogEligibleZones , minReplicasRemaining , loadResult . present ( ) & & loadResult . get ( ) . healthyZone . present ( ) ) ;
2017-05-26 04:48:44 +08:00
}
2018-09-09 06:44:48 +08:00
JsonBuilderObject configObj = configurationFetcher ( configuration , coordinators , & status_incomplete_reasons ) ;
2017-05-26 04:48:44 +08:00
// configArr could be empty
if ( ! configObj . empty ( ) )
statusObj [ " configuration " ] = configObj ;
// workloadStatusFetcher returns the workload section but also optionally writes the qos section and adds to the data_overlay object
if ( ! workerStatuses [ 1 ] . empty ( ) )
statusObj [ " workload " ] = workerStatuses [ 1 ] ;
statusObj [ " layers " ] = workerStatuses [ 2 ] ;
// Add qos section if it was populated
if ( ! qos . empty ( ) )
statusObj [ " qos " ] = qos ;
// Merge data_overlay into data
2018-09-09 06:44:48 +08:00
JsonBuilderObject & clusterDataSection = workerStatuses [ 0 ] ;
// TODO: This probably is no longer possible as there is no ability to merge json objects with an output-only model
clusterDataSection . addContents ( data_overlay ) ;
2017-05-26 04:48:44 +08:00
// If data section not empty, add it to statusObj
if ( ! clusterDataSection . empty ( ) )
statusObj [ " data " ] = clusterDataSection ;
2019-09-05 06:33:46 +08:00
// Insert database_lock_state section
2017-05-26 04:48:44 +08:00
if ( ! workerStatuses [ 3 ] . empty ( ) ) {
2019-09-05 06:33:46 +08:00
statusObj [ " database_lock_state " ] = workerStatuses [ 3 ] ;
2017-05-26 04:48:44 +08:00
}
2019-07-11 05:43:20 +08:00
// Insert cluster summary statistics
if ( ! workerStatuses [ 4 ] . empty ( ) ) {
statusObj . addContents ( workerStatuses [ 4 ] ) ;
2017-05-26 04:48:44 +08:00
}
// Need storage servers now for processStatusFetcher() below.
2019-02-08 05:39:22 +08:00
ErrorOr < vector < std : : pair < StorageServerInterface , EventMap > > > _storageServers = wait ( storageServerFuture ) ;
2017-05-26 04:48:44 +08:00
if ( _storageServers . present ( ) ) {
storageServers = _storageServers . get ( ) ;
}
2018-12-01 02:46:04 +08:00
else {
2018-09-09 06:44:48 +08:00
messages . push_back ( JsonBuilder : : makeMessage ( " storage_servers_error " , " Timed out trying to retrieve storage servers. " ) ) ;
2018-12-01 02:46:04 +08:00
}
2017-05-26 04:48:44 +08:00
// ...also tlogs
2019-02-08 05:39:22 +08:00
ErrorOr < vector < std : : pair < TLogInterface , EventMap > > > _tLogs = wait ( tLogFuture ) ;
2017-05-26 04:48:44 +08:00
if ( _tLogs . present ( ) ) {
tLogs = _tLogs . get ( ) ;
}
2018-12-01 02:46:04 +08:00
else {
2018-09-09 06:44:48 +08:00
messages . push_back ( JsonBuilder : : makeMessage ( " log_servers_error " , " Timed out trying to retrieve log servers. " ) ) ;
2018-12-01 02:46:04 +08:00
}
// ...also proxies
2019-02-08 05:39:22 +08:00
ErrorOr < vector < std : : pair < MasterProxyInterface , EventMap > > > _proxies = wait ( proxyFuture ) ;
2018-12-01 02:46:04 +08:00
if ( _proxies . present ( ) ) {
proxies = _proxies . get ( ) ;
}
else {
messages . push_back ( JsonBuilder : : makeMessage ( " proxies_error " , " Timed out trying to retrieve proxies. " ) ) ;
}
2019-10-03 08:06:19 +08:00
wait ( waitForAll ( warningFutures ) ) ;
2017-05-26 04:48:44 +08:00
}
else {
// Set layers status to { _valid: false, error: "configurationMissing"}
2018-09-09 06:44:48 +08:00
JsonBuilderObject layers ;
layers [ " _valid " ] = false ;
layers [ " _error " ] = " configurationMissing " ;
2018-09-08 22:15:28 +08:00
statusObj [ " layers " ] = layers ;
2017-05-26 04:48:44 +08:00
}
2019-07-12 05:53:00 +08:00
JsonBuilderObject processStatus = wait ( processStatusFetcher ( db , workers , pMetrics , mMetrics , networkMetrics ,
latestError , traceFileOpenErrors , programStarts ,
2020-02-05 02:26:18 +08:00
processIssues , storageServers , tLogs , proxies ,
coordinators , cx , configuration ,
2019-08-16 05:42:49 +08:00
loadResult . present ( ) ? loadResult . get ( ) . healthyZone : Optional < Key > ( ) ,
2019-06-27 05:03:02 +08:00
& status_incomplete_reasons ) ) ;
2017-05-26 04:48:44 +08:00
statusObj [ " processes " ] = processStatus ;
2019-07-26 08:15:31 +08:00
statusObj [ " clients " ] = clientStatusFetcher ( clientStatus ) ;
2017-05-26 04:48:44 +08:00
2018-09-09 06:44:48 +08:00
JsonBuilderArray incompatibleConnectionsArray ;
2017-05-26 04:48:44 +08:00
for ( auto it : incompatibleConnections ) {
2018-09-08 22:15:28 +08:00
incompatibleConnectionsArray . push_back ( it . toString ( ) ) ;
2017-05-26 04:48:44 +08:00
}
statusObj [ " incompatible_connections " ] = incompatibleConnectionsArray ;
2019-08-16 04:42:39 +08:00
statusObj [ " datacenter_lag " ] = getLagObject ( datacenterVersionDifference ) ;
2017-05-26 04:48:44 +08:00
2019-03-11 13:58:15 +08:00
int totalDegraded = 0 ;
for ( auto & it : workers ) {
if ( it . degraded ) {
totalDegraded + + ;
}
}
statusObj [ " degraded_processes " ] = totalDegraded ;
2017-05-26 04:48:44 +08:00
if ( ! recoveryStateStatus . empty ( ) )
statusObj [ " recovery_state " ] = recoveryStateStatus ;
// cluster messages subsection;
2019-07-26 08:15:31 +08:00
JsonBuilderArray clientIssuesArr = getClientIssuesAsMessages ( clientStatus ) ;
2017-05-26 04:48:44 +08:00
if ( clientIssuesArr . size ( ) > 0 ) {
2018-09-09 06:44:48 +08:00
JsonBuilderObject clientIssueMessage = JsonBuilder : : makeMessage ( " client_issues " , " Some clients of this cluster have issues. " ) ;
2017-05-26 04:48:44 +08:00
clientIssueMessage [ " issues " ] = clientIssuesArr ;
messages . push_back ( clientIssueMessage ) ;
}
// Create the status_incomplete message if there were any reasons that the status is incomplete.
if ( ! status_incomplete_reasons . empty ( ) )
{
2018-09-09 06:44:48 +08:00
JsonBuilderObject incomplete_message = JsonBuilder : : makeMessage ( " status_incomplete " , " Unable to retrieve all status information. " ) ;
2017-05-26 04:48:44 +08:00
// Make a JSON array of all of the reasons in the status_incomplete_reasons set.
2018-09-09 06:44:48 +08:00
JsonBuilderArray reasons ;
2018-08-31 16:21:24 +08:00
for ( auto i : status_incomplete_reasons ) {
2018-09-09 06:44:48 +08:00
reasons . push_back ( JsonBuilderObject ( ) . setKey ( " description " , i ) ) ;
2018-08-31 16:21:24 +08:00
}
2017-05-26 04:48:44 +08:00
incomplete_message [ " reasons " ] = reasons ;
messages . push_back ( incomplete_message ) ;
}
statusObj [ " messages " ] = messages ;
int64_t clusterTime = time ( 0 ) ;
if ( clusterTime ! = - 1 ) {
statusObj [ " cluster_controller_timestamp " ] = clusterTime ;
}
2018-09-08 22:15:28 +08:00
TraceEvent ( " ClusterGetStatus " ) . detail ( " Duration " , timer ( ) - tStart ) . detail ( " StatusSize " , statusObj . getFinalLength ( ) ) ;
2018-09-05 18:50:53 +08:00
2018-09-08 22:15:28 +08:00
return StatusReply ( statusObj . getJson ( ) ) ;
2017-05-26 04:48:44 +08:00
} catch ( Error & e ) {
TraceEvent ( SevError , " StatusError " ) . error ( e ) ;
throw ;
}
}
2018-09-10 18:07:11 +08:00
bool checkAsciiNumber ( const char * s ) {
JsonBuilderObject number ;
number . setKeyRawNumber ( " number " , s ) ;
2018-09-10 18:21:55 +08:00
std : : string js = number . getJson ( ) ;
printf ( " '%s' => %s \n " , s , js . c_str ( ) ) ;
2018-09-10 18:07:11 +08:00
try {
// Make sure it parses as JSON
2018-09-10 18:21:55 +08:00
readJSONStrictly ( js ) ;
} catch ( Error & e ) {
printf ( " error: %s \n " , e . what ( ) ) ;
2018-09-10 18:07:11 +08:00
return false ;
}
2018-09-10 18:21:55 +08:00
2018-09-10 18:07:11 +08:00
return true ;
}
bool checkJson ( const JsonBuilder & j , const char * expected ) {
std : : string js = j . getJson ( ) ;
printf ( " json: '%s' \n " , js . c_str ( ) ) ;
printf ( " expected: '%s' \n \n " , expected ) ;
2018-09-10 18:21:55 +08:00
try {
// Make sure it parses as JSON
readJSONStrictly ( js ) ;
} catch ( Error & e ) {
printf ( " error: %s \n " , e . what ( ) ) ;
return false ;
}
2018-09-10 18:07:11 +08:00
return js = = expected ;
2018-09-09 06:44:48 +08:00
}
2018-10-06 13:09:58 +08:00
TEST_CASE ( " /status/json/builder " ) {
2018-09-09 06:44:48 +08:00
JsonBuilder json ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkJson ( json , " null " ) ) ;
2018-09-09 06:44:48 +08:00
JsonBuilderArray array ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkJson ( array , " [] " ) ) ;
2018-09-09 06:44:48 +08:00
array . push_back ( 1 ) ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkJson ( array , " [1] " ) ) ;
2018-09-09 06:44:48 +08:00
array . push_back ( 2 ) ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkJson ( array , " [1,2] " ) ) ;
2018-09-09 06:44:48 +08:00
array . push_back ( " test " ) ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkJson ( array , " [1,2, \" test \" ] " ) ) ;
2018-09-09 06:44:48 +08:00
JsonBuilderObject object ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkJson ( object , " {} " ) ) ;
2018-09-09 06:44:48 +08:00
object . setKey ( " a " , 5 ) ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkJson ( object , " { \" a \" :5} " ) ) ;
2018-09-09 06:44:48 +08:00
object . setKey ( " b " , " hi " ) ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkJson ( object , " { \" a \" :5, \" b \" : \" hi \" } " ) ) ;
2018-09-09 06:44:48 +08:00
object . setKey ( " c " , array ) ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkJson ( object , " { \" a \" :5, \" b \" : \" hi \" , \" c \" :[1,2, \" test \" ]} " ) ) ;
2018-09-09 06:44:48 +08:00
JsonBuilderArray array2 ;
array2 . push_back ( json ) ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkJson ( array2 , " [null] " ) ) ;
2018-09-09 06:44:48 +08:00
object . setKey ( " d " , array2 ) ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkJson ( object , " { \" a \" :5, \" b \" : \" hi \" , \" c \" :[1,2, \" test \" ], \" d \" :[null]} " ) ) ;
2018-09-05 18:50:53 +08:00
2018-09-10 13:05:04 +08:00
JsonBuilderObject object2 ;
object2 [ " x " ] = 1 ;
object2 [ " y " ] = " why " ;
object2 [ " z " ] = std : : string ( " zee " ) ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkJson ( object2 , " { \" x \" :1, \" y \" : \" why \" , \" z \" : \" zee \" } " ) ) ;
2018-09-10 13:05:04 +08:00
object2 . addContents ( object ) ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkJson ( object2 , " { \" x \" :1, \" y \" : \" why \" , \" z \" : \" zee \" , \" a \" :5, \" b \" : \" hi \" , \" c \" :[1,2, \" test \" ], \" d \" :[null]} " ) ) ;
2018-09-10 13:05:04 +08:00
object2 . addContents ( JsonBuilderObject ( ) ) ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkJson ( object2 , " { \" x \" :1, \" y \" : \" why \" , \" z \" : \" zee \" , \" a \" :5, \" b \" : \" hi \" , \" c \" :[1,2, \" test \" ], \" d \" :[null]} " ) ) ;
2018-09-10 13:05:04 +08:00
array2 . addContents ( array ) ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkJson ( array2 , " [null,1,2, \" test \" ] " ) ) ;
2018-09-10 13:05:04 +08:00
array2 . addContents ( JsonBuilderArray ( ) ) ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkJson ( array2 , " [null,1,2, \" test \" ] " ) ) ;
2018-09-11 04:28:31 +08:00
JsonBuilderObject object3 ;
object3 [ " infinity " ] = std : : numeric_limits < double > : : infinity ( ) ;
object3 [ " nan " ] = std : : numeric_limits < double > : : quiet_NaN ( ) ;
ASSERT ( checkJson ( object3 , " { \" infinity \" :1e99, \" nan \" :-999} " ) ) ;
ASSERT ( checkAsciiNumber ( " inf " ) ) ;
ASSERT ( checkAsciiNumber ( " infA " ) ) ;
ASSERT ( checkAsciiNumber ( " in " ) ) ;
ASSERT ( checkAsciiNumber ( " -inf " ) ) ;
ASSERT ( checkAsciiNumber ( " -infA " ) ) ;
ASSERT ( checkAsciiNumber ( " -in " ) ) ;
2018-09-10 18:07:11 +08:00
ASSERT ( checkAsciiNumber ( " a " ) ) ;
ASSERT ( checkAsciiNumber ( " -1a.0 " ) ) ;
ASSERT ( checkAsciiNumber ( " -01a.0 " ) ) ;
ASSERT ( checkAsciiNumber ( " 01.0a " ) ) ;
ASSERT ( checkAsciiNumber ( " -1.0 " ) ) ;
ASSERT ( checkAsciiNumber ( " -01.0 " ) ) ;
ASSERT ( checkAsciiNumber ( " 01.0 " ) ) ;
ASSERT ( checkAsciiNumber ( " -001 " ) ) ;
ASSERT ( checkAsciiNumber ( " 000. " ) ) ;
ASSERT ( checkAsciiNumber ( " -0001.e- " ) ) ;
ASSERT ( checkAsciiNumber ( " -0001.0e-01 " ) ) ;
ASSERT ( checkAsciiNumber ( " -000123e-234 " ) ) ;
ASSERT ( checkAsciiNumber ( " -09234.12312e-132 " ) ) ;
ASSERT ( checkAsciiNumber ( " -111.e-01 " ) ) ;
ASSERT ( checkAsciiNumber ( " -00111.e-01 " ) ) ;
ASSERT ( checkAsciiNumber ( " -.e " ) ) ;
ASSERT ( checkAsciiNumber ( " -09234.123a12e-132 " ) ) ;
ASSERT ( checkAsciiNumber ( " -11a1.e-01 " ) ) ;
ASSERT ( checkAsciiNumber ( " -00111.ae-01 " ) ) ;
ASSERT ( checkAsciiNumber ( " -.ea " ) ) ;
2018-09-11 01:52:15 +08:00
ASSERT ( checkAsciiNumber ( " -.e+ " ) ) ;
ASSERT ( checkAsciiNumber ( " -.0e+1 " ) ) ;
2018-09-10 13:05:04 +08:00
2018-09-05 18:50:53 +08:00
return Void ( ) ;
}
2018-09-09 09:05:16 +08:00
JsonBuilderObject randomDocument ( const std : : vector < std : : string > & strings , int & limit , int level ) ;
JsonBuilderArray randomArray ( const std : : vector < std : : string > & strings , int & limit , int level ) ;
JsonBuilderArray randomArray ( const std : : vector < std : : string > & strings , int & limit , int level ) {
JsonBuilderArray r ;
2019-05-11 05:01:52 +08:00
int size = deterministicRandom ( ) - > randomInt ( 0 , 50 ) ;
2018-09-09 09:05:16 +08:00
while ( - - size ) {
if ( - - limit < = 0 )
break ;
2019-05-11 05:01:52 +08:00
if ( level > 0 & & deterministicRandom ( ) - > coinflip ( ) ) {
if ( deterministicRandom ( ) - > coinflip ( ) )
2018-09-09 09:05:16 +08:00
r . push_back ( randomDocument ( strings , limit , level - 1 ) ) ;
else
r . push_back ( randomArray ( strings , limit , level - 1 ) ) ;
}
else {
2019-05-11 05:01:52 +08:00
switch ( deterministicRandom ( ) - > randomInt ( 0 , 3 ) ) {
2018-09-10 09:12:41 +08:00
case 0 :
2019-05-11 05:01:52 +08:00
r . push_back ( deterministicRandom ( ) - > randomInt ( 0 , 10000000 ) ) ;
2018-09-10 09:12:41 +08:00
case 1 :
2019-05-11 05:01:52 +08:00
r . push_back ( strings [ deterministicRandom ( ) - > randomInt ( 0 , strings . size ( ) ) ] ) ;
2018-09-10 09:12:41 +08:00
case 2 :
default :
2019-05-11 05:01:52 +08:00
r . push_back ( deterministicRandom ( ) - > random01 ( ) ) ;
2018-09-10 09:12:41 +08:00
}
2018-09-09 09:05:16 +08:00
}
}
return r ;
}
JsonBuilderObject randomDocument ( const std : : vector < std : : string > & strings , int & limit , int level ) {
JsonBuilderObject r ;
2019-05-11 05:01:52 +08:00
int size = deterministicRandom ( ) - > randomInt ( 0 , 300 ) ;
2018-09-09 09:05:16 +08:00
while ( - - size ) {
if ( - - limit < = 0 )
break ;
2019-05-11 05:01:52 +08:00
const std : : string & key = strings [ deterministicRandom ( ) - > randomInt ( 0 , strings . size ( ) ) ] ;
2018-09-09 09:05:16 +08:00
2019-05-11 05:01:52 +08:00
if ( level > 0 & & deterministicRandom ( ) - > coinflip ( ) ) {
if ( deterministicRandom ( ) - > coinflip ( ) )
2018-09-09 09:05:16 +08:00
r [ key ] = randomDocument ( strings , limit , level - 1 ) ;
else
r [ key ] = randomArray ( strings , limit , level - 1 ) ;
}
else {
2019-05-11 05:01:52 +08:00
switch ( deterministicRandom ( ) - > randomInt ( 0 , 3 ) ) {
2018-09-10 09:12:41 +08:00
case 0 :
2019-05-11 05:01:52 +08:00
r [ key ] = deterministicRandom ( ) - > randomInt ( 0 , 10000000 ) ;
2018-09-10 09:12:41 +08:00
case 1 :
2019-05-11 05:01:52 +08:00
r [ key ] = strings [ deterministicRandom ( ) - > randomInt ( 0 , strings . size ( ) ) ] ;
2018-09-10 09:12:41 +08:00
case 2 :
default :
2019-05-11 05:01:52 +08:00
r [ key ] = deterministicRandom ( ) - > random01 ( ) ;
2018-09-10 09:12:41 +08:00
}
2018-09-09 09:05:16 +08:00
}
}
return r ;
}
2018-10-06 13:09:58 +08:00
TEST_CASE ( " /status/json/builderPerf " ) {
2018-09-09 09:05:16 +08:00
std : : vector < std : : string > strings ;
int c = 1000000 ;
printf ( " Generating random strings \n " ) ;
while ( - - c )
2019-05-11 05:01:52 +08:00
strings . push_back ( deterministicRandom ( ) - > randomAlphaNumeric ( deterministicRandom ( ) - > randomInt ( 0 , 50 ) ) ) ;
2018-09-09 09:05:16 +08:00
int elements = 100000 ;
int level = 6 ;
2018-09-10 09:12:41 +08:00
int iterations = 200 ;
2018-09-09 09:05:16 +08:00
printf ( " Generating and serializing random document \n " ) ;
2018-09-10 13:11:11 +08:00
int64_t bytes = 0 ;
2018-09-10 09:12:41 +08:00
double generated = 0 ;
2018-09-11 10:54:27 +08:00
double serialized = 0 ;
2018-09-10 09:12:41 +08:00
for ( int i = 0 ; i < iterations ; i + + ) {
int n = elements ;
2018-09-11 10:54:27 +08:00
double start ;
start = timer ( ) ;
2018-09-10 09:12:41 +08:00
JsonBuilderObject obj = randomDocument ( strings , n , level ) ;
2018-09-11 10:54:27 +08:00
double generate = timer ( ) - start ;
2018-09-09 09:05:16 +08:00
2018-09-11 10:54:27 +08:00
start = timer ( ) ;
2018-09-10 09:12:41 +08:00
std : : string s = obj . getJson ( ) ;
2018-09-11 10:54:27 +08:00
double serialize = timer ( ) - start ;
start = timer ( ) ;
json_spirit : : mValue mv = readJSONStrictly ( s ) ;
double jsParse = timer ( ) - start ;
start = timer ( ) ;
std : : string jsStr = json_spirit : : write_string ( mv ) ;
double jsSerialize = timer ( ) - start ;
printf ( " JsonBuilder: %8lu bytes %-7.5f gen + %-7.5f serialize = %-7.5f \n " , s . size ( ) , generate , serialize , generate + serialize ) ;
printf ( " json_spirit: %8lu bytes %-7.5f parse + %-7.5f serialize = %-7.5f \n " , jsStr . size ( ) , jsParse , jsSerialize , jsParse + jsSerialize ) ;
printf ( " \n " ) ;
2019-07-12 05:53:00 +08:00
2018-09-11 10:54:27 +08:00
generated + = generate ;
serialized + = serialize ;
2018-09-10 09:12:41 +08:00
bytes + = s . size ( ) ;
}
2018-09-09 09:05:16 +08:00
2018-09-11 10:54:27 +08:00
double elapsed = generated + serialized ;
2019-05-05 01:52:02 +08:00
printf ( " RESULT: % " PRId64 " bytes %d elements %d levels %f seconds (%f gen, %f serialize) %f MB/s %f items/s \n " ,
2018-09-10 09:12:41 +08:00
bytes , iterations * elements , level , elapsed , generated , elapsed - generated , bytes / elapsed / 1e6 , iterations * elements / elapsed ) ;
2018-09-09 09:05:16 +08:00
return Void ( ) ;
}
2018-10-06 13:09:58 +08:00
TEST_CASE ( " /status/json/merging " ) {
2017-05-26 04:48:44 +08:00
StatusObject objA , objB , objC ;
JSONDoc a ( objA ) , b ( objB ) , c ( objC ) ;
a . create ( " int_one " ) = 1 ;
a . create ( " int_unmatched " ) = 2 ;
a . create ( " int_total_30.$sum " ) = 10 ;
a . create ( " bool_true.$and " ) = true ;
a . create ( " string " ) = " test " ;
a . create ( " subdoc.int_11 " ) = 11 ;
a . create ( " a " ) = " justA " ;
a . create ( " subdoc.double_max_5.$max " ) = 2.0 ;
a . create ( " subdoc.double_min_2.$min " ) = 2.0 ;
a . create ( " subdoc.obj_count_3.$count_keys.one " ) = 1 ;
a . create ( " subdoc.obj_count_3.$count_keys.two " ) = 2 ;
a . create ( " expired.$expires " ) = " I should have expired. " ;
a . create ( " expired.version " ) = 1 ;
a . create ( " not_expired_and_merged.$expires.seven.$sum " ) = 1 ;
a . create ( " not_expired_and_merged.$expires.one.$min " ) = 3 ;
a . create ( " not_expired_and_merged.version " ) = 3 ;
2018-02-07 05:44:04 +08:00
a . create ( " mixed_numeric_sum_6.$sum " ) = 0.5 ;
a . create ( " mixed_numeric_min_0.$min " ) = 1.5 ;
2017-05-26 04:48:44 +08:00
b . create ( " int_one " ) = 1 ;
b . create ( " int_unmatched " ) = 3 ;
b . create ( " int_total_30.$sum " ) = 20 ;
b . create ( " bool_true.$and " ) = true ;
b . create ( " string " ) = " test " ;
b . create ( " subdoc.int_11 " ) = 11 ;
b . create ( " b " ) = " justB " ;
b . create ( " subdoc.double_max_5.$max " ) = 5.0 ;
b . create ( " subdoc.double_min_2.$min " ) = 5.0 ;
b . create ( " subdoc.obj_count_3.$count_keys.three " ) = 3 ;
b . create ( " expired.$expires " ) = " I should have also expired. " ;
b . create ( " expired.version " ) = 1 ;
b . create ( " not_expired_and_merged.$expires.seven.$sum " ) = 2 ;
b . create ( " not_expired_and_merged.$expires.one.$min " ) = 1 ;
b . create ( " not_expired_and_merged.version " ) = 3 ;
b . create ( " last_hello.$last " ) = " blah " ;
b . create ( " latest_obj.$latest.a " ) = 0 ;
b . create ( " latest_obj.$latest.b " ) = 0 ;
b . create ( " latest_obj.$latest.c " ) = 0 ;
b . create ( " latest_obj.timestamp " ) = 2 ;
b . create ( " latest_int_5.$latest " ) = 7 ;
b . create ( " latest_int_5.timestamp " ) = 2 ;
2018-02-07 05:44:04 +08:00
b . create ( " mixed_numeric_sum_6.$sum " ) = 1 ;
b . create ( " mixed_numeric_min_0.$min " ) = 4.5 ;
2017-05-26 04:48:44 +08:00
c . create ( " int_total_30.$sum " ) = 0 ;
c . create ( " not_expired.$expires " ) = " I am still valid " ;
c . create ( " not_expired.version " ) = 3 ;
c . create ( " not_expired_and_merged.$expires.seven.$sum " ) = 4 ;
c . create ( " not_expired_and_merged.$expires.one.$min " ) = 2 ;
c . create ( " not_expired_and_merged.version " ) = 3 ;
c . create ( " last_hello.$last " ) = " hello " ;
c . create ( " latest_obj.$latest.a.$max " ) = " a " ;
c . create ( " latest_obj.$latest.b.$min " ) = " b " ;
c . create ( " latest_obj.$latest.expired.$expires " ) = " I should not be here. " ;
c . create ( " latest_obj.$latest.expired.version " ) = 1 ;
c . create ( " latest_obj.$latest.not_expired.$expires " ) = " Still alive. " ;
c . create ( " latest_obj.$latest.not_expired.version " ) = 3 ;
c . create ( " latest_obj.timestamp " ) = 3 ;
2018-02-07 05:44:04 +08:00
c . create ( " latest_int_5.$latest " ) = 5 ;
c . create ( " latest_int_5.timestamp " ) = 3 ;
c . create ( " mixed_numeric_sum_6.$sum " ) = 4.5 ;
c . create ( " mixed_numeric_min_0.$min " ) = ( double ) 0.0 ;
printf ( " a = \n %s \n " , json_spirit : : write_string ( json_spirit : : mValue ( objA ) , json_spirit : : pretty_print ) . c_str ( ) ) ;
printf ( " b = \n %s \n " , json_spirit : : write_string ( json_spirit : : mValue ( objB ) , json_spirit : : pretty_print ) . c_str ( ) ) ;
printf ( " c = \n %s \n " , json_spirit : : write_string ( json_spirit : : mValue ( objC ) , json_spirit : : pretty_print ) . c_str ( ) ) ;
2017-05-26 04:48:44 +08:00
JSONDoc : : expires_reference_version = 2 ;
a . absorb ( b ) ;
a . absorb ( c ) ;
a . cleanOps ( ) ;
2018-02-07 05:44:04 +08:00
printf ( " result = \n %s \n " , json_spirit : : write_string ( json_spirit : : mValue ( objA ) , json_spirit : : pretty_print ) . c_str ( ) ) ;
2017-05-26 04:48:44 +08:00
std : : string result = json_spirit : : write_string ( json_spirit : : mValue ( objA ) ) ;
2018-02-07 05:44:04 +08:00
std : : string expected = " { \" a \" : \" justA \" , \" b \" : \" justB \" , \" bool_true \" :true, \" expired \" :null, \" int_one \" :1, \" int_total_30 \" :30, \" int_unmatched \" :{ \" ERROR \" : \" Values do not match. \" , \" a \" :2, \" b \" :3}, \" last_hello \" : \" hello \" , \" latest_int_5 \" :5, \" latest_obj \" :{ \" a \" : \" a \" , \" b \" : \" b \" , \" not_expired \" : \" Still alive. \" }, \" mixed_numeric_min_0 \" :0, \" mixed_numeric_sum_6 \" :6, \" not_expired \" : \" I am still valid \" , \" not_expired_and_merged \" :{ \" one \" :1, \" seven \" :7}, \" string \" : \" test \" , \" subdoc \" :{ \" double_max_5 \" :5, \" double_min_2 \" :2, \" int_11 \" :11, \" obj_count_3 \" :3}} " ;
2017-05-26 04:48:44 +08:00
if ( result ! = expected ) {
2018-02-07 05:44:04 +08:00
printf ( " ERROR: Combined doc does not match expected. \n expected: \n \n %s \n result: \n %s \n " , expected . c_str ( ) , result . c_str ( ) ) ;
2017-05-26 04:48:44 +08:00
ASSERT ( false ) ;
}
return Void ( ) ;
}